You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2015/05/28 00:42:17 UTC

[1/3] hadoop git commit: HDFS-8482. Rename BlockInfoContiguous to BlockInfo. Contributed by Zhe Zhang.

Repository: hadoop
Updated Branches:
  refs/heads/trunk cab7674e5 -> 4928f5473


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
index f6b18e6..0d726e9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -100,7 +100,7 @@ public class TestTruncateQuotaUpdate {
   @Test
   public void testTruncateWithSnapshotAndDivergence() {
     INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous
+    BlockInfo[] blocks = new BlockInfo
         [file.getBlocks().length];
     System.arraycopy(file.getBlocks(), 0, blocks, 0, blocks.length);
     addSnapshotFeature(file, blocks);
@@ -130,11 +130,11 @@ public class TestTruncateQuotaUpdate {
   }
 
   private INodeFile createMockFile(long size, short replication) {
-    ArrayList<BlockInfoContiguous> blocks = new ArrayList<>();
+    ArrayList<BlockInfo> blocks = new ArrayList<>();
     long createdSize = 0;
     while (createdSize < size) {
       long blockSize = Math.min(BLOCKSIZE, size - createdSize);
-      BlockInfoContiguous bi = newBlock(blockSize, replication);
+      BlockInfo bi = newBlock(blockSize, replication);
       blocks.add(bi);
       createdSize += BLOCKSIZE;
     }
@@ -142,16 +142,16 @@ public class TestTruncateQuotaUpdate {
         .createImmutable((short) 0x1ff));
     return new INodeFile(
         ++nextMockINodeId, new byte[0], perm, 0, 0,
-        blocks.toArray(new BlockInfoContiguous[blocks.size()]), replication,
+        blocks.toArray(new BlockInfo[blocks.size()]), replication,
         BLOCKSIZE);
   }
 
-  private BlockInfoContiguous newBlock(long size, short replication) {
+  private BlockInfo newBlock(long size, short replication) {
     Block b = new Block(++nextMockBlockId, size, ++nextMockGenstamp);
-    return new BlockInfoContiguous(b, replication);
+    return new BlockInfo(b, replication);
   }
 
-  private static void addSnapshotFeature(INodeFile file, BlockInfoContiguous[] blocks) {
+  private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {
     FileDiff diff = mock(FileDiff.class);
     when(diff.getBlocks()).thenReturn(blocks);
     FileDiffList diffList = new FileDiffList();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 11b19f3..a1abd08 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
@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
@@ -176,7 +176,7 @@ public class SnapshotTestHelper {
    * 
    * Specific information for different types of INode: 
    * {@link INodeDirectory}:childrenSize 
-   * {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()}
+   * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
    * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
    * {@link FileWithSnapshot}: next link
    * </pre>

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 85072d1..7bffb33 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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(BlockInfoContiguous b : file.getBlocks()) {
+    for(BlockInfo b : file.getBlocks()) {
       assertBlockCollection(blkManager, file, b);
     }
     return file;
   }
 
   static void assertBlockCollection(final BlockManager blkManager,
-      final INodeFile file, final BlockInfoContiguous b) {
+      final INodeFile file, final BlockInfo 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);
-      BlockInfoContiguous[] blocks = f2.getBlocks();
+      BlockInfo[] blocks = f2.getBlocks();
       hdfs.delete(sub2, true);
       // The INode should have been removed from the blocksMap
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo 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);
-    BlockInfoContiguous[] blocks0 = f0.getBlocks();
+    BlockInfo[] 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(BlockInfoContiguous b : blocks0) {
+    for(BlockInfo 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(BlockInfoContiguous b : blocks0) {
+    for(BlockInfo 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();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] 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();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] 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();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] 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();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(testData.length, blks[0].getNumBytes());
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 cdd655e..139a37e 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -262,12 +262,12 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
     final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
         tempFile.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks = temp.getBlocks();
+    BlockInfo[] blocks = temp.getBlocks();
     hdfs.delete(tempDir, true);
     // check dir's quota usage
     checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     // check blocks of tempFile
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -344,7 +344,7 @@ public class TestSnapshotDeletion {
     // while deletion, we add diff for subsub and metaChangeFile1, and remove
     // newFile
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -481,7 +481,7 @@ public class TestSnapshotDeletion {
     
     final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
         .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks = toDeleteFileNode.getBlocks();
+    BlockInfo[] blocks = toDeleteFileNode.getBlocks();
     
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
@@ -507,7 +507,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 (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -801,7 +801,7 @@ public class TestSnapshotDeletion {
     FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
     INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
         file14_s2.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks_14 = file14Node.getBlocks();
+    BlockInfo[] blocks_14 = file14Node.getBlocks();
     TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
         blockmanager);
     
@@ -838,7 +838,7 @@ public class TestSnapshotDeletion {
         modDirStr + "file15");
     assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file15_s1));
-    for (BlockInfoContiguous b : blocks_14) {
+    for (BlockInfo b : blocks_14) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     


[3/3] hadoop git commit: HDFS-8482. Rename BlockInfoContiguous to BlockInfo. Contributed by Zhe Zhang.

Posted by wa...@apache.org.
HDFS-8482. Rename BlockInfoContiguous to BlockInfo. Contributed by Zhe Zhang.


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

Branch: refs/heads/trunk
Commit: 4928f5473394981829e5ffd4b16ea0801baf5c45
Parents: cab7674
Author: Andrew Wang <wa...@apache.org>
Authored: Wed May 27 15:37:01 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed May 27 15:42:07 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/blockmanagement/BlockCollection.java |   8 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  | 399 +++++++++++++++++++
 .../blockmanagement/BlockInfoContiguous.java    | 399 -------------------
 .../BlockInfoContiguousUnderConstruction.java   |   6 +-
 .../server/blockmanagement/BlockManager.java    | 124 +++---
 .../hdfs/server/blockmanagement/BlocksMap.java  |  32 +-
 .../CacheReplicationMonitor.java                |  14 +-
 .../blockmanagement/DatanodeDescriptor.java     |  16 +-
 .../blockmanagement/DatanodeStorageInfo.java    |  25 +-
 .../blockmanagement/DecommissionManager.java    |  22 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  14 +-
 .../hdfs/server/namenode/FSDirectory.java       |   6 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   8 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  14 +-
 .../hdfs/server/namenode/FSImageFormat.java     |  14 +-
 .../server/namenode/FSImageFormatPBINode.java   |  11 +-
 .../server/namenode/FSImageSerialization.java   |   6 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  42 +-
 .../namenode/FileUnderConstructionFeature.java  |   6 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  74 ++--
 .../hdfs/server/namenode/LeaseManager.java      |   6 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   8 +-
 .../hdfs/server/namenode/snapshot/FileDiff.java |  10 +-
 .../server/namenode/snapshot/FileDiffList.java  |  20 +-
 .../snapshot/FileWithSnapshotFeature.java       |   4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   4 +-
 .../apache/hadoop/hdfs/TestDecommission.java    |   4 +-
 .../server/blockmanagement/TestBlockInfo.java   |  20 +-
 .../blockmanagement/TestBlockManager.java       |  36 +-
 .../blockmanagement/TestDatanodeDescriptor.java |   4 +-
 .../blockmanagement/TestPendingReplication.java |   4 +-
 .../blockmanagement/TestReplicationPolicy.java  |   9 +-
 .../hdfs/server/namenode/CreateEditsLog.java    |   8 +-
 .../hdfs/server/namenode/TestAddBlock.java      |  12 +-
 .../namenode/TestBlockUnderConstruction.java    |   6 +-
 .../TestCommitBlockSynchronization.java         |   8 +-
 .../hdfs/server/namenode/TestEditLog.java       |   4 +-
 .../hdfs/server/namenode/TestFSImage.java       |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   4 +-
 .../server/namenode/TestGetBlockLocations.java  |   4 +-
 .../hdfs/server/namenode/TestINodeFile.java     |   4 +-
 .../namenode/TestTruncateQuotaUpdate.java       |  16 +-
 .../namenode/snapshot/SnapshotTestHelper.java   |   4 +-
 .../snapshot/TestFileWithSnapshotFeature.java   |   6 +-
 .../snapshot/TestSnapshotBlocksMap.java         |  24 +-
 .../namenode/snapshot/TestSnapshotDeletion.java |  16 +-
 48 files changed, 745 insertions(+), 750 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 ce97552..948a516 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -580,6 +580,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8377. Support HTTP/2 in datanode. (Duo Zhang via wheat9)
 
+    HDFS-8482. Rename BlockInfoContiguous to BlockInfo. (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 c0a959c..02a1d05 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 BlockInfoContiguous getLastBlock();
+  public BlockInfo getLastBlock();
 
   /** 
    * Get content summary.
@@ -46,7 +46,7 @@ public interface BlockCollection {
   /**
    * Get the blocks.
    */
-  public BlockInfoContiguous[] getBlocks();
+  public BlockInfo[] 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, BlockInfoContiguous blk);
+  public void setBlock(int index, BlockInfo blk);
 
   /**
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfoContiguous lastBlock,
+  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfo lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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
new file mode 100644
index 0000000..0370c57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -0,0 +1,399 @@
+/**
+ * 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) {
+    super(from);
+    this.triplets = new Object[from.triplets.length];
+    this.bc = from.bc;
+  }
+
+  public BlockCollection getBlockCollection() {
+    return bc;
+  }
+
+  public void setBlockCollection(BlockCollection bc) {
+    this.bc = bc;
+  }
+
+  public boolean isDeleted() {
+    return (bc == null);
+  }
+
+  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 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 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().getPreferredBlockReplication(), s, targets);
+      ucBlock.setBlockCollection(getBlockCollection());
+      return ucBlock;
+    }
+    // the block is already under construction
+    BlockInfoContiguousUnderConstruction ucBlock =
+        (BlockInfoContiguousUnderConstruction)this;
+    ucBlock.setBlockUCState(s);
+    ucBlock.setExpectedLocations(targets);
+    ucBlock.setBlockCollection(getBlockCollection());
+    return ucBlock;
+  }
+
+  @Override
+  public 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/4928f547/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
deleted file mode 100644
index 769046b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ /dev/null
@@ -1,399 +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 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) {
-    super(from);
-    this.triplets = new Object[from.triplets.length];
-    this.bc = from.bc;
-  }
-
-  public BlockCollection getBlockCollection() {
-    return bc;
-  }
-
-  public void setBlockCollection(BlockCollection bc) {
-    this.bc = bc;
-  }
-
-  public boolean isDeleted() {
-    return (bc == null);
-  }
-
-  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 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().getPreferredBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
-    }
-    // the block is already under construction
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)this;
-    ucBlock.setBlockUCState(s);
-    ucBlock.setExpectedLocations(targets);
-    ucBlock.setBlockCollection(getBlockCollection());
-    return ucBlock;
-  }
-
-  @Override
-  public 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/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
index 92153ab..58020a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  * Represents a block that is currently being constructed.<br>
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
+public class BlockInfoContiguousUnderConstruction extends BlockInfo {
   /** Block state. See {@link BlockUCState} */
   private BlockUCState blockUCState;
 
@@ -182,10 +182,10 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
    * the client or it does not have at least a minimal number of replicas 
    * reported from data-nodes. 
    */
-  BlockInfoContiguous convertToCompleteBlock() throws IOException {
+  BlockInfo convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != BlockUCState.COMPLETE :
       "Trying to convert a COMPLETE block";
-    return new BlockInfoContiguous(this);
+    return new BlockInfo(this);
   }
 
   /** Set expected locations */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 54981fb..45a485b 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
@@ -543,8 +543,8 @@ public class BlockManager {
     int usableReplicas = numReplicas.liveReplicas() +
                          numReplicas.decommissionedAndDecommissioning();
     
-    if (block instanceof BlockInfoContiguous) {
-      BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
+    if (block instanceof BlockInfo) {
+      BlockCollection bc = ((BlockInfo) block).getBlockCollection();
       String fileName = (bc == null) ? "[orphaned]" : bc.getName();
       out.print(fileName + ": ");
     }
@@ -624,7 +624,7 @@ public class BlockManager {
       Block commitBlock) throws IOException {
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
-    BlockInfoContiguous lastBlock = bc.getLastBlock();
+    BlockInfo lastBlock = bc.getLastBlock();
     if(lastBlock == null)
       return false; // no blocks in file yet
     if(lastBlock.isComplete())
@@ -644,11 +644,11 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private BlockInfoContiguous completeBlock(final BlockCollection bc,
+  private BlockInfo completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
     if(blkIndex < 0)
       return null;
-    BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
+    BlockInfo curBlock = bc.getBlocks()[blkIndex];
     if(curBlock.isComplete())
       return curBlock;
     BlockInfoContiguousUnderConstruction ucBlock =
@@ -660,7 +660,7 @@ public class BlockManager {
     if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
-    BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
+    BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
     
@@ -678,9 +678,9 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
   }
 
-  private BlockInfoContiguous completeBlock(final BlockCollection bc,
-      final BlockInfoContiguous block, boolean force) throws IOException {
-    BlockInfoContiguous[] fileBlocks = bc.getBlocks();
+  private BlockInfo completeBlock(final BlockCollection bc,
+      final BlockInfo block, boolean force) throws IOException {
+    BlockInfo[] fileBlocks = bc.getBlocks();
     for(int idx = 0; idx < fileBlocks.length; idx++)
       if(fileBlocks[idx] == block) {
         return completeBlock(bc, idx, force);
@@ -693,7 +693,7 @@ public class BlockManager {
    * regardless of whether enough replicas are present. This is necessary
    * when tailing edit logs as a Standby.
    */
-  public BlockInfoContiguous forceCompleteBlock(final BlockCollection bc,
+  public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfoContiguousUnderConstruction block) throws IOException {
     block.commitBlock(block);
     return completeBlock(bc, block, true);
@@ -716,7 +716,7 @@ public class BlockManager {
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
       BlockCollection bc, long bytesToRemove) throws IOException {
-    BlockInfoContiguous oldBlock = bc.getLastBlock();
+    BlockInfo oldBlock = bc.getLastBlock();
     if(oldBlock == null ||
        bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
       return null;
@@ -769,7 +769,7 @@ public class BlockManager {
   }
   
   private List<LocatedBlock> createLocatedBlockList(
-      final BlockInfoContiguous[] blocks,
+      final BlockInfo[] blocks,
       final long offset, final long length, final int nrBlocksToReturn,
       final AccessMode mode) throws IOException {
     int curBlk = 0;
@@ -799,7 +799,7 @@ public class BlockManager {
     return results;
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks,
+  private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
     int curBlk = 0;
     long curPos = 0;
@@ -815,7 +815,7 @@ public class BlockManager {
     return createLocatedBlock(blocks[curBlk], curPos, mode);
   }
   
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
     final AccessMode mode) throws IOException {
     final LocatedBlock lb = createLocatedBlock(blk, pos);
     if (mode != null) {
@@ -825,7 +825,7 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
       ) throws IOException {
     if (blk instanceof BlockInfoContiguousUnderConstruction) {
       if (blk.isComplete()) {
@@ -874,7 +874,7 @@ public class BlockManager {
   }
 
   /** Create a LocatedBlocks. */
-  public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks,
+  public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
       final long fileSizeExcludeBlocksUnderConstruction,
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
@@ -897,7 +897,7 @@ public class BlockManager {
       final LocatedBlock lastlb;
       final boolean isComplete;
       if (!inSnapshot) {
-        final BlockInfoContiguous last = blocks[blocks.length - 1];
+        final BlockInfo last = blocks[blocks.length - 1];
         final long lastPos = last.isComplete()?
             fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
             : fileSizeExcludeBlocksUnderConstruction;
@@ -986,7 +986,7 @@ public class BlockManager {
   /**
    * Check if a block is replicated to at least the minimum replication.
    */
-  public boolean isSufficientlyReplicated(BlockInfoContiguous b) {
+  public boolean isSufficientlyReplicated(BlockInfo b) {
     // Compare against the lesser of the minReplication and number of live DNs.
     final int replication =
         Math.min(minReplication, getDatanodeManager().getNumLiveDataNodes());
@@ -1027,7 +1027,7 @@ public class BlockManager {
     if(numBlocks == 0) {
       return new BlocksWithLocations(new BlockWithLocations[0]);
     }
-    Iterator<BlockInfoContiguous> iter = node.getBlockIterator();
+    Iterator<BlockInfo> iter = node.getBlockIterator();
     // starting from a random block
     int startBlock = ThreadLocalRandom.current().nextInt(numBlocks);
     // skip blocks
@@ -1036,7 +1036,7 @@ public class BlockManager {
     }
     List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
     long totalSize = 0;
-    BlockInfoContiguous curBlock;
+    BlockInfo curBlock;
     while(totalSize<size && iter.hasNext()) {
       curBlock = iter.next();
       if(!curBlock.isComplete())  continue;
@@ -1135,7 +1135,7 @@ public class BlockManager {
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
       final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
-    final BlockInfoContiguous storedBlock = getStoredBlock(blk.getLocalBlock());
+    final BlockInfo 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
@@ -1694,7 +1694,7 @@ public class BlockManager {
            * Use the blockinfo from the blocksmap to be certain we're working
            * with the most up-to-date block information (e.g. genstamp).
            */
-          BlockInfoContiguous bi = blocksMap.getStoredBlock(timedOutItems[i]);
+          BlockInfo bi = blocksMap.getStoredBlock(timedOutItems[i]);
           if (bi == null) {
             continue;
           }
@@ -1738,16 +1738,16 @@ public class BlockManager {
    */
   private static class BlockToMarkCorrupt {
     /** The corrupted block in a datanode. */
-    final BlockInfoContiguous corrupted;
+    final BlockInfo corrupted;
     /** The corresponding block stored in the BlockManager. */
-    final BlockInfoContiguous stored;
+    final BlockInfo stored;
     /** The reason to mark corrupt. */
     final String reason;
     /** The reason code to be stored */
     final Reason reasonCode;
 
-    BlockToMarkCorrupt(BlockInfoContiguous corrupted,
-        BlockInfoContiguous stored, String reason,
+    BlockToMarkCorrupt(BlockInfo corrupted,
+        BlockInfo stored, String reason,
         Reason reasonCode) {
       Preconditions.checkNotNull(corrupted, "corrupted is null");
       Preconditions.checkNotNull(stored, "stored is null");
@@ -1758,14 +1758,14 @@ public class BlockManager {
       this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(BlockInfoContiguous stored, String reason,
+    BlockToMarkCorrupt(BlockInfo stored, String reason,
         Reason reasonCode) {
       this(stored, stored, reason, reasonCode);
     }
 
-    BlockToMarkCorrupt(BlockInfoContiguous stored, long gs, String reason,
+    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
         Reason reasonCode) {
-      this(new BlockInfoContiguous(stored), stored, reason, reasonCode);
+      this(new BlockInfo(stored), stored, reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -1882,10 +1882,10 @@ public class BlockManager {
              "longer exists on the DataNode.",
               Long.toHexString(context.getReportId()), zombie.getStorageID());
     assert(namesystem.hasWriteLock());
-    Iterator<BlockInfoContiguous> iter = zombie.getBlockIterator();
+    Iterator<BlockInfo> iter = zombie.getBlockIterator();
     int prevBlocks = zombie.numBlocks();
     while (iter.hasNext()) {
-      BlockInfoContiguous block = iter.next();
+      BlockInfo block = iter.next();
       // We assume that a block can be on only one storage in a DataNode.
       // That's why we pass in the DatanodeDescriptor rather than the
       // DatanodeStorageInfo.
@@ -1943,7 +1943,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfoContiguous bi = blocksMap.getStoredBlock(b);
+        BlockInfo bi = blocksMap.getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -1983,7 +1983,7 @@ public class BlockManager {
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     //
-    Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
+    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
     Collection<Block> toRemove = new TreeSet<Block>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
@@ -2000,7 +2000,7 @@ public class BlockManager {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
-    for (BlockInfoContiguous b : toAdd) {
+    for (BlockInfo b : toAdd) {
       addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
@@ -2022,7 +2022,7 @@ public class BlockManager {
    * Mark block replicas as corrupt except those on the storages in 
    * newStorages list.
    */
-  public void markBlockReplicasAsCorrupt(BlockInfoContiguous block, 
+  public void markBlockReplicasAsCorrupt(BlockInfo block,
       long oldGenerationStamp, long oldNumBytes, 
       DatanodeStorageInfo[] newStorages) throws IOException {
     assert namesystem.hasWriteLock();
@@ -2091,7 +2091,7 @@ public class BlockManager {
         continue;
       }
       
-      BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(iblk);
+      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -2136,7 +2136,7 @@ public class BlockManager {
 
   private void reportDiff(DatanodeStorageInfo storageInfo, 
       BlockListAsLongs newReport, 
-      Collection<BlockInfoContiguous> toAdd,              // add to DatanodeDescriptor
+      Collection<BlockInfo> 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
@@ -2144,7 +2144,7 @@ public class BlockManager {
 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
-    BlockInfoContiguous delimiter = new BlockInfoContiguous(new Block(), (short) 1);
+    BlockInfo delimiter = new BlockInfo(new Block(), (short) 1);
     AddBlockResult result = storageInfo.addBlock(delimiter);
     assert result == AddBlockResult.ADDED 
         : "Delimiting block cannot be present in the node";
@@ -2157,7 +2157,7 @@ public class BlockManager {
     // scan the report and process newly reported blocks
     for (BlockReportReplica iblk : newReport) {
       ReplicaState iState = iblk.getState();
-      BlockInfoContiguous storedBlock = processReportedBlock(storageInfo,
+      BlockInfo storedBlock = processReportedBlock(storageInfo,
           iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
 
       // move block to the head of the list
@@ -2169,7 +2169,7 @@ public class BlockManager {
 
     // collect blocks that have not been reported
     // all of them are next to the delimiter
-    Iterator<BlockInfoContiguous> it =
+    Iterator<BlockInfo> it =
         storageInfo.new BlockIterator(delimiter.getNext(0));
     while(it.hasNext())
       toRemove.add(it.next());
@@ -2207,10 +2207,10 @@ public class BlockManager {
    * @return the up-to-date stored block, if it should be kept.
    *         Otherwise, null.
    */
-  private BlockInfoContiguous processReportedBlock(
+  private BlockInfo processReportedBlock(
       final DatanodeStorageInfo storageInfo,
       final Block block, final ReplicaState reportedState, 
-      final Collection<BlockInfoContiguous> toAdd,
+      final Collection<BlockInfo> toAdd,
       final Collection<Block> toInvalidate, 
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
@@ -2231,7 +2231,7 @@ public class BlockManager {
     }
     
     // find block by blockId
-    BlockInfoContiguous storedBlock = blocksMap.getStoredBlock(block);
+    BlockInfo storedBlock = blocksMap.getStoredBlock(block);
     if(storedBlock == null) {
       // If blocksMap does not contain reported block id,
       // the replica should be removed from the data-node.
@@ -2370,7 +2370,7 @@ public class BlockManager {
    */
   private BlockToMarkCorrupt checkReplicaCorrupt(
       Block reported, ReplicaState reportedState, 
-      BlockInfoContiguous storedBlock, BlockUCState ucState,
+      BlockInfo storedBlock, BlockUCState ucState,
       DatanodeDescriptor dn) {
     switch(reportedState) {
     case FINALIZED:
@@ -2443,7 +2443,7 @@ public class BlockManager {
     }
   }
 
-  private boolean isBlockUnderConstruction(BlockInfoContiguous storedBlock,
+  private boolean isBlockUnderConstruction(BlockInfo storedBlock,
       BlockUCState ucState, ReplicaState reportedState) {
     switch(reportedState) {
     case FINALIZED:
@@ -2487,7 +2487,7 @@ public class BlockManager {
    * 
    * @throws IOException
    */
-  private void addStoredBlockImmediate(BlockInfoContiguous storedBlock,
+  private void addStoredBlockImmediate(BlockInfo storedBlock,
       DatanodeStorageInfo storageInfo)
   throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
@@ -2519,13 +2519,13 @@ public class BlockManager {
    * needed replications if this takes care of the problem.
    * @return the block that is stored in blockMap.
    */
-  private Block addStoredBlock(final BlockInfoContiguous block,
+  private Block addStoredBlock(final BlockInfo block,
                                DatanodeStorageInfo storageInfo,
                                DatanodeDescriptor delNodeHint,
                                boolean logEveryBlock)
   throws IOException {
     assert block != null && namesystem.hasWriteLock();
-    BlockInfoContiguous storedBlock;
+    BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (block instanceof BlockInfoContiguousUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
@@ -2623,7 +2623,7 @@ public class BlockManager {
     return storedBlock;
   }
 
-  private void logAddStoredBlock(BlockInfoContiguous storedBlock,
+  private void logAddStoredBlock(BlockInfo storedBlock,
       DatanodeDescriptor node) {
     if (!blockLog.isInfoEnabled()) {
       return;
@@ -2651,7 +2651,7 @@ public class BlockManager {
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfoContiguous blk) {
+  private void invalidateCorruptReplicas(BlockInfo blk) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
@@ -2730,7 +2730,7 @@ public class BlockManager {
     long nrInvalid = 0, nrOverReplicated = 0;
     long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
     long startTimeMisReplicatedScan = Time.monotonicNow();
-    Iterator<BlockInfoContiguous> blocksItr = blocksMap.getBlocks().iterator();
+    Iterator<BlockInfo> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
     replicationQueuesInitProgress = 0;
     long totalProcessed = 0;
@@ -2742,7 +2742,7 @@ public class BlockManager {
       namesystem.writeLockInterruptibly();
       try {
         while (processed < numBlocksPerIteration && blocksItr.hasNext()) {
-          BlockInfoContiguous block = blocksItr.next();
+          BlockInfo block = blocksItr.next();
           MisReplicationResult res = processMisReplicatedBlock(block);
           if (LOG.isTraceEnabled()) {
             LOG.trace("block " + block + ": " + res);
@@ -2817,7 +2817,7 @@ public class BlockManager {
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
    */
-  private MisReplicationResult processMisReplicatedBlock(BlockInfoContiguous block) {
+  private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
     if (block.isDeleted()) {
       // block does not belong to any file
       addToInvalidates(block);
@@ -3157,7 +3157,7 @@ public class BlockManager {
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
-    Collection<BlockInfoContiguous> toAdd = new LinkedList<BlockInfoContiguous>();
+    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
     Collection<Block> toInvalidate = new LinkedList<Block>();
     Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
     Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
@@ -3174,7 +3174,7 @@ public class BlockManager {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
-    for (BlockInfoContiguous b : toAdd) {
+    for (BlockInfo b : toAdd) {
       addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
@@ -3301,7 +3301,7 @@ public class BlockManager {
    * @param b - the block being tested
    * @return count of live nodes for this block
    */
-  int countLiveNodes(BlockInfoContiguous b) {
+  int countLiveNodes(BlockInfo b) {
     if (!namesystem.isInStartupSafeMode()) {
       return countNodes(b).liveReplicas();
     }
@@ -3380,7 +3380,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public DatanodeStorageInfo[] getStorages(BlockInfoContiguous block) {
+  public DatanodeStorageInfo[] getStorages(BlockInfo block) {
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
     int i = 0;
     for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
@@ -3409,7 +3409,7 @@ public class BlockManager {
     }
   }
 
-  public BlockInfoContiguous getStoredBlock(Block block) {
+  public BlockInfo getStoredBlock(Block block) {
     return blocksMap.getStoredBlock(block);
   }
 
@@ -3462,8 +3462,8 @@ public class BlockManager {
    * replicated.
    */
   public boolean checkBlocksProperlyReplicated(
-      String src, BlockInfoContiguous[] blocks) {
-    for (BlockInfoContiguous b: blocks) {
+      String src, BlockInfo[] blocks) {
+    for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
         final BlockInfoContiguousUnderConstruction uc =
             (BlockInfoContiguousUnderConstruction)b;
@@ -3578,7 +3578,7 @@ public class BlockManager {
     return this.neededReplications.getCorruptReplOneBlockSize();
   }
 
-  public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
+  public BlockInfo addBlockCollection(BlockInfo block,
       BlockCollection bc) {
     return blocksMap.addBlockCollection(block, bc);
   }
@@ -3818,7 +3818,7 @@ public class BlockManager {
 
   /**
    * A simple result enum for the result of
-   * {@link BlockManager#processMisReplicatedBlock(BlockInfoContiguous)}.
+   * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
    */
   enum MisReplicationResult {
     /** The block should be invalidated since it belongs to a deleted file. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 5e7d34f..d06d503 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 BlockInfoContiguous blockInfo;
+    private final BlockInfo blockInfo;
     private int nextIdx = 0;
       
-    StorageIterator(BlockInfoContiguous blkInfo) {
+    StorageIterator(BlockInfo blkInfo) {
       this.blockInfo = blkInfo;
     }
 
@@ -63,14 +63,14 @@ class BlocksMap {
   /** Constant {@link LightWeightGSet} capacity. */
   private final int capacity;
   
-  private GSet<Block, BlockInfoContiguous> blocks;
+  private GSet<Block, BlockInfo> blocks;
 
   BlocksMap(int capacity) {
     // Use 2% of total memory to size the GSet capacity
     this.capacity = capacity;
-    this.blocks = new LightWeightGSet<Block, BlockInfoContiguous>(capacity) {
+    this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity) {
       @Override
-      public Iterator<BlockInfoContiguous> iterator() {
+      public Iterator<BlockInfo> 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) {
-    BlockInfoContiguous info = blocks.get(b);
+    BlockInfo info = blocks.get(b);
     return (info != null) ? info.getBlockCollection() : null;
   }
 
   /**
    * Add block b belonging to the specified block collection to the map.
    */
-  BlockInfoContiguous addBlockCollection(BlockInfoContiguous b, BlockCollection bc) {
-    BlockInfoContiguous info = blocks.get(b);
+  BlockInfo addBlockCollection(BlockInfo b, BlockCollection bc) {
+    BlockInfo 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) {
-    BlockInfoContiguous blockInfo = blocks.remove(block);
+    BlockInfo blockInfo = blocks.remove(block);
     if (blockInfo == null)
       return;
 
@@ -132,7 +132,7 @@ class BlocksMap {
   }
   
   /** Returns the block object it it exists in the map. */
-  BlockInfoContiguous getStoredBlock(Block b) {
+  BlockInfo 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 BlockInfoContiguous storedBlock) {
+  Iterable<DatanodeStorageInfo> getStorages(final BlockInfo 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) {
-    BlockInfoContiguous info = blocks.get(b);
+    BlockInfo 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) {
-    BlockInfoContiguous info = blocks.get(b);
+    BlockInfo info = blocks.get(b);
     if (info == null)
       return false;
 
@@ -203,7 +203,7 @@ class BlocksMap {
     return blocks.size();
   }
 
-  Iterable<BlockInfoContiguous> getBlocks() {
+  Iterable<BlockInfo> getBlocks() {
     return blocks;
   }
   
@@ -218,8 +218,8 @@ class BlocksMap {
    * @param newBlock - block for replacement
    * @return new block
    */
-  BlockInfoContiguous replaceBlock(BlockInfoContiguous newBlock) {
-    BlockInfoContiguous currentBlock = blocks.get(newBlock);
+  BlockInfo replaceBlock(BlockInfo newBlock) {
+    BlockInfo currentBlock = blocks.get(newBlock);
     assert currentBlock != null : "the block if not in blocksMap";
     // replace block in data-node lists
     for (int i = currentBlock.numNodes() - 1; i >= 0; i--) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 bf5ece9..a0f3503 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) {
-    BlockInfoContiguous[] blockInfos = file.getBlocks();
+    BlockInfo[] 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 (BlockInfoContiguous blockInfo : blockInfos) {
+    for (BlockInfo 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, 
-          BlockInfoContiguous blockInfo) {
+          BlockInfo 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();
         }
       }
-      BlockInfoContiguous blockInfo = blockManager.
+      BlockInfo 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.
-    BlockInfoContiguous blockInfo = blockManager.
+    BlockInfo 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();
-        BlockInfoContiguous info =
+        BlockInfo 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();
-        BlockInfoContiguous info =
+        BlockInfo info =
             blockManager.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes += info.getNumBytes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 4731ad4..02abc1d 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
@@ -335,7 +335,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(BlockInfoContiguous b) {
+  boolean removeBlock(BlockInfo b) {
     final DatanodeStorageInfo s = b.findStorageInfo(this);
     // if block exists on this datanode
     if (s != null) {
@@ -348,7 +348,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, BlockInfoContiguous b) {
+  boolean removeBlock(String storageID, BlockInfo b) {
     DatanodeStorageInfo s = getStorageInfo(storageID);
     if (s != null) {
       return s.removeBlock(b);
@@ -537,12 +537,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  private static class BlockIterator implements Iterator<BlockInfoContiguous> {
+  private static class BlockIterator implements Iterator<BlockInfo> {
     private int index = 0;
-    private final List<Iterator<BlockInfoContiguous>> iterators;
+    private final List<Iterator<BlockInfo>> iterators;
     
     private BlockIterator(final DatanodeStorageInfo... storages) {
-      List<Iterator<BlockInfoContiguous>> iterators = new ArrayList<Iterator<BlockInfoContiguous>>();
+      List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
       for (DatanodeStorageInfo e : storages) {
         iterators.add(e.getBlockIterator());
       }
@@ -556,7 +556,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
 
     @Override
-    public BlockInfoContiguous next() {
+    public BlockInfo next() {
       update();
       return iterators.get(index).next();
     }
@@ -573,10 +573,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  Iterator<BlockInfoContiguous> getBlockIterator() {
+  Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(getStorageInfos());
   }
-  Iterator<BlockInfoContiguous> getBlockIterator(final String storageID) {
+  Iterator<BlockInfo> getBlockIterator(final String storageID) {
     return new BlockIterator(getStorageInfo(storageID));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 c6c9001..57e81b4 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
@@ -25,9 +25,6 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -83,10 +80,10 @@ public class DatanodeStorageInfo {
   /**
    * Iterates over the list of blocks belonging to the data-node.
    */
-  class BlockIterator implements Iterator<BlockInfoContiguous> {
-    private BlockInfoContiguous current;
+  class BlockIterator implements Iterator<BlockInfo> {
+    private BlockInfo current;
 
-    BlockIterator(BlockInfoContiguous head) {
+    BlockIterator(BlockInfo head) {
       this.current = head;
     }
 
@@ -94,8 +91,8 @@ public class DatanodeStorageInfo {
       return current != null;
     }
 
-    public BlockInfoContiguous next() {
-      BlockInfoContiguous res = current;
+    public BlockInfo next() {
+      BlockInfo res = current;
       current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
       return res;
     }
@@ -115,7 +112,7 @@ public class DatanodeStorageInfo {
   private volatile long remaining;
   private long blockPoolUsed;
 
-  private volatile BlockInfoContiguous blockList = null;
+  private volatile BlockInfo blockList = null;
   private int numBlocks = 0;
 
   // The ID of the last full block report which updated this storage.
@@ -229,7 +226,7 @@ public class DatanodeStorageInfo {
     return blockPoolUsed;
   }
 
-  public AddBlockResult addBlock(BlockInfoContiguous b) {
+  public AddBlockResult addBlock(BlockInfo b) {
     // First check whether the block belongs to a different storage
     // on the same DN.
     AddBlockResult result = AddBlockResult.ADDED;
@@ -254,7 +251,7 @@ public class DatanodeStorageInfo {
     return result;
   }
 
-  public boolean removeBlock(BlockInfoContiguous b) {
+  public boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
     if (b.removeStorage(this)) {
       numBlocks--;
@@ -268,7 +265,7 @@ public class DatanodeStorageInfo {
     return numBlocks;
   }
   
-  Iterator<BlockInfoContiguous> getBlockIterator() {
+  Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(blockList);
 
   }
@@ -277,7 +274,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(BlockInfoContiguous b, int curIndex, int headIndex) {
+  int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
     blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
     return curIndex;
   }
@@ -287,7 +284,7 @@ public class DatanodeStorageInfo {
    * @return the head of the blockList
    */
   @VisibleForTesting
-  BlockInfoContiguous getBlockListHeadForTesting(){
+  BlockInfo getBlockListHeadForTesting(){
     return blockList;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 5f7366e..639d2b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -100,7 +100,7 @@ public class DecommissionManager {
    * reports or other events. Before being finally marking as decommissioned,
    * another check is done with the actual block map.
    */
-  private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfoContiguous>>
+  private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfo>>
       decomNodeBlocks;
 
   /**
@@ -248,7 +248,7 @@ public class DecommissionManager {
    * Full-strength replication is not always necessary, hence "sufficient".
    * @return true if sufficient, else false.
    */
-  private boolean isSufficientlyReplicated(BlockInfoContiguous block, 
+  private boolean isSufficientlyReplicated(BlockInfo block,
       BlockCollection bc,
       NumberReplicas numberReplicas) {
     final int numExpected = bc.getPreferredBlockReplication();
@@ -412,7 +412,7 @@ public class DecommissionManager {
     }
 
     private void check() {
-      final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfoContiguous>>>
+      final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
           it = new CyclicIteration<>(decomNodeBlocks, iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
@@ -420,10 +420,10 @@ public class DecommissionManager {
           && !exceededNumBlocksPerCheck()
           && !exceededNumNodesPerCheck()) {
         numNodesChecked++;
-        final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfoContiguous>>
+        final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
             entry = it.next();
         final DatanodeDescriptor dn = entry.getKey();
-        AbstractList<BlockInfoContiguous> blocks = entry.getValue();
+        AbstractList<BlockInfo> blocks = entry.getValue();
         boolean fullScan = false;
         if (blocks == null) {
           // This is a newly added datanode, run through its list to schedule 
@@ -495,7 +495,7 @@ public class DecommissionManager {
      * datanode.
      */
     private void pruneSufficientlyReplicated(final DatanodeDescriptor datanode,
-        AbstractList<BlockInfoContiguous> blocks) {
+        AbstractList<BlockInfo> blocks) {
       processBlocksForDecomInternal(datanode, blocks.iterator(), null, true);
     }
 
@@ -509,9 +509,9 @@ public class DecommissionManager {
      * @param datanode
      * @return List of insufficiently replicated blocks 
      */
-    private AbstractList<BlockInfoContiguous> handleInsufficientlyReplicated(
+    private AbstractList<BlockInfo> handleInsufficientlyReplicated(
         final DatanodeDescriptor datanode) {
-      AbstractList<BlockInfoContiguous> insufficient = new ChunkedArrayList<>();
+      AbstractList<BlockInfo> insufficient = new ChunkedArrayList<>();
       processBlocksForDecomInternal(datanode, datanode.getBlockIterator(),
           insufficient, false);
       return insufficient;
@@ -535,8 +535,8 @@ public class DecommissionManager {
      */
     private void processBlocksForDecomInternal(
         final DatanodeDescriptor datanode,
-        final Iterator<BlockInfoContiguous> it,
-        final List<BlockInfoContiguous> insufficientlyReplicated,
+        final Iterator<BlockInfo> it,
+        final List<BlockInfo> insufficientlyReplicated,
         boolean pruneSufficientlyReplicated) {
       boolean firstReplicationLog = true;
       int underReplicatedBlocks = 0;
@@ -544,7 +544,7 @@ public class DecommissionManager {
       int underReplicatedInOpenFiles = 0;
       while (it.hasNext()) {
         numBlocksChecked++;
-        final BlockInfoContiguous block = it.next();
+        final BlockInfo block = it.next();
         // Remove the block from the list if it's no longer in the block map,
         // e.g. the containing file has been deleted
         if (blockManager.blocksMap.getStoredBlock(block) == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 307bd59..3b5f973 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -235,7 +235,7 @@ class FSDirWriteFileOp {
         return onRetryBlock[0];
       } else {
         // add new chosen targets to already allocated block and return
-        BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
+        BlockInfo lastBlockInFile = pendingFile.getLastBlock();
         ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
             .setExpectedLocations(targets);
         offset = pendingFile.computeFileSize();
@@ -507,7 +507,7 @@ class FSDirWriteFileOp {
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  private static BlockInfoContiguous addBlock(
+  private static BlockInfo addBlock(
       FSDirectory fsd, String path, INodesInPath inodesInPath, Block block,
       DatanodeStorageInfo[] targets) throws IOException {
     fsd.writeLock();
@@ -608,7 +608,7 @@ class FSDirWriteFileOp {
       }
     }
     final INodeFile file = fsn.checkLease(src, clientName, inode, fileId);
-    BlockInfoContiguous lastBlockInFile = file.getLastBlock();
+    BlockInfo lastBlockInFile = file.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
@@ -636,7 +636,7 @@ class FSDirWriteFileOp {
       //    changed the namesystem state yet.
       //    We run this analysis again in Part II where case 4 is impossible.
 
-      BlockInfoContiguous penultimateBlock = file.getPenultimateBlock();
+      BlockInfo penultimateBlock = file.getPenultimateBlock();
       if (previous == null &&
           lastBlockInFile != null &&
           lastBlockInFile.getNumBytes() >= file.getPreferredBlockSize() &&
@@ -762,7 +762,7 @@ class FSDirWriteFileOp {
       long id, PermissionStatus permissions, long mtime, long atime,
       short replication, long preferredBlockSize, byte storagePolicyId) {
     return new INodeFile(id, null, permissions, mtime, atime,
-        BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize,
+        BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
         storagePolicyId);
   }
 
@@ -802,7 +802,7 @@ class FSDirWriteFileOp {
       DatanodeStorageInfo[] targets)
       throws IOException {
     assert fsn.hasWriteLock();
-    BlockInfoContiguous b = addBlock(fsn.dir, src, inodesInPath, newBlock,
+    BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock,
                                      targets);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 8fdd2d7..5b8650e 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
@@ -33,10 +33,8 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.XAttrHelper;
@@ -51,7 +49,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -919,7 +917,7 @@ public class FSDirectory implements Closeable {
         unprotectedTruncate(iip, newLength, collectedBlocks, mtime, null);
 
     if(! onBlockBoundary) {
-      BlockInfoContiguous oldBlock = file.getLastBlock();
+      BlockInfo oldBlock = file.getLastBlock();
       Block tBlk =
       getFSNamesystem().prepareFileForTruncate(iip,
           clientName, clientMachine, file.computeFileSize() - newLength,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 83e52bc..dec9709 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.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
@@ -773,10 +773,10 @@ public class FSEditLog implements LogsPurgeable {
   
   public void logAddBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
-    BlockInfoContiguous[] blocks = file.getBlocks();
+    BlockInfo[] blocks = file.getBlocks();
     Preconditions.checkState(blocks != null && blocks.length > 0);
-    BlockInfoContiguous pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
-    BlockInfoContiguous lastBlock = blocks[blocks.length - 1];
+    BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
+    BlockInfo lastBlock = blocks[blocks.length - 1];
     AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
         .setPenultimateBlock(pBlock).setLastBlock(lastBlock);
     logEdit(op);


[2/3] hadoop git commit: HDFS-8482. Rename BlockInfoContiguous to BlockInfo. Contributed by Zhe Zhang.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 476ff36..9acd81f 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
@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -942,7 +942,7 @@ public class FSEditLogLoader {
    */
   private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
       throws IOException {
-    BlockInfoContiguous[] oldBlocks = file.getBlocks();
+    BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
     
@@ -968,7 +968,7 @@ public class FSEditLogLoader {
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
+    BlockInfo newBI = new BlockInfoContiguousUnderConstruction(
           newBlock, file.getPreferredBlockReplication());
     fsNamesys.getBlockManager().addBlockCollection(newBI, file);
     file.addBlock(newBI);
@@ -982,7 +982,7 @@ public class FSEditLogLoader {
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
       INodesInPath iip, INodeFile file) throws IOException {
     // Update its block list
-    BlockInfoContiguous[] oldBlocks = file.getBlocks();
+    BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
     String path = op.getPath();
     
@@ -991,7 +991,7 @@ public class FSEditLogLoader {
     
     // First, update blocks in common
     for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) {
-      BlockInfoContiguous oldBlock = oldBlocks[i];
+      BlockInfo oldBlock = oldBlocks[i];
       Block newBlock = newBlocks[i];
       
       boolean isLastBlock = i == newBlocks.length - 1;
@@ -1043,7 +1043,7 @@ public class FSEditLogLoader {
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
-        BlockInfoContiguous newBI;
+        BlockInfo 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
@@ -1055,7 +1055,7 @@ public class FSEditLogLoader {
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          newBI = new BlockInfoContiguous(newBlock,
+          newBI = new BlockInfo(newBlock,
               file.getPreferredBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollection(newBI, file);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 ec2babd..a46f2b6 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
@@ -52,7 +52,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -686,7 +686,7 @@ public class FSImageFormat {
 
     public void updateBlocksMap(INodeFile file) {
       // Add file->block mapping
-      final BlockInfoContiguous[] blocks = file.getBlocks();
+      final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
@@ -753,9 +753,9 @@ public class FSImageFormat {
       // file
       
       // read blocks
-      BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
+      BlockInfo[] blocks = new BlockInfo[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfoContiguous(replication);
+        blocks[j] = new BlockInfo(replication);
         blocks[j].readFields(in);
       }
 
@@ -775,7 +775,7 @@ public class FSImageFormat {
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
             if (blocks.length > 0) {
-              BlockInfoContiguous lastBlk = blocks[blocks.length - 1];
+              BlockInfo lastBlk = blocks[blocks.length - 1];
               blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
                   lastBlk, replication);
             }
@@ -958,9 +958,9 @@ public class FSImageFormat {
         FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
         oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (oldnode.numBlocks() > 0) {
-          BlockInfoContiguous ucBlock = cons.getLastBlock();
+          BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfoContiguous info = namesystem.getBlockManager().addBlockCollection(
+          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
               ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 d966c69..29713a2 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
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -43,7 +42,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
@@ -211,7 +210,7 @@ public final class FSImageFormatPBINode {
 
     public static void updateBlocksMap(INodeFile file, BlockManager bm) {
       // Add file->block mapping
-      final BlockInfoContiguous[] blocks = file.getBlocks();
+      final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
           file.setBlock(i, bm.addBlockCollection(blocks[i], file));
@@ -324,9 +323,9 @@ public final class FSImageFormatPBINode {
       short replication = (short) f.getReplication();
       LoaderContext state = parent.getLoaderContext();
 
-      BlockInfoContiguous[] blocks = new BlockInfoContiguous[bp.size()];
+      BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0, e = bp.size(); i < e; ++i) {
-        blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
+        blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
       }
       final PermissionStatus permissions = loadPermission(f.getPermission(),
           parent.getLoaderContext().getStringTable());
@@ -352,7 +351,7 @@ public final class FSImageFormatPBINode {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (blocks.length > 0) {
-          BlockInfoContiguous lastBlk = file.getLastBlock();
+          BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
           file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
               lastBlk, replication));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 1888d87..d790b8f 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,7 +33,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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;
@@ -126,12 +126,12 @@ public class FSImageSerialization {
     long preferredBlockSize = in.readLong();
   
     int numBlocks = in.readInt();
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
+    BlockInfo[] blocks = new BlockInfo[numBlocks];
     Block blk = new Block();
     int i = 0;
     for (; i < numBlocks-1; i++) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
+      blocks[i] = new BlockInfo(blk, blockReplication);
     }
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 bfd6eba..798f8d5 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
@@ -203,7 +203,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -2016,7 +2016,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Check if the file is already being truncated with the same length
-    final BlockInfoContiguous last = file.getLastBlock();
+    final BlockInfo last = file.getLastBlock();
     if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
       final Block truncateBlock
           = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();
@@ -2090,7 +2090,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.addLease(
         file.getFileUnderConstructionFeature().getClientName(), file.getId());
     boolean shouldRecoverNow = (newBlock == null);
-    BlockInfoContiguous oldBlock = file.getLastBlock();
+    BlockInfo oldBlock = file.getLastBlock();
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
     if(newBlock == null) {
       newBlock = (shouldCopyOnTruncate) ? createNewBlock() :
@@ -2141,7 +2141,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, BlockInfoContiguous blk) {
+  boolean shouldCopyOnTruncate(INodeFile file, BlockInfo blk) {
     if(!isUpgradeFinalized()) {
       return true;
     }
@@ -2505,7 +2505,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, src, holder,
                            clientMachine, false);
       
-      final BlockInfoContiguous lastBlock = myFile.getLastBlock();
+      final BlockInfo lastBlock = myFile.getLastBlock();
       // Check that the block has at least minimum replication.
       if(lastBlock != null && lastBlock.isComplete() &&
           !getBlockManager().isSufficientlyReplicated(lastBlock)) {
@@ -2561,7 +2561,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         }
       }
     } else {
-      BlockInfoContiguous lastBlock = file.getLastBlock();
+      BlockInfo lastBlock = file.getLastBlock();
       if (lastBlock != null) {
         ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
         ret = new LocatedBlock(blk, new DatanodeInfo[0]);
@@ -2603,7 +2603,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /** Compute quota change for converting a complete block to a UC block */
   private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) {
     final QuotaCounts delta = new QuotaCounts.Builder().build();
-    final BlockInfoContiguous lastBlock = file.getLastBlock();
+    final BlockInfo lastBlock = file.getLastBlock();
     if (lastBlock != null) {
       final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
       final short repl = file.getPreferredBlockReplication();
@@ -2740,7 +2740,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 op.getExceptionMessage(src, holder, clientMachine,
                     "lease recovery is in progress. Try again later."));
         } else {
-          final BlockInfoContiguous lastBlock = file.getLastBlock();
+          final BlockInfo lastBlock = file.getLastBlock();
           if (lastBlock != null
               && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
             throw new RecoveryInProgressException(
@@ -3065,10 +3065,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           .getBlocks());
     } else {
       // check the penultimate block of this file
-      BlockInfoContiguous b = v.getPenultimateBlock();
+      BlockInfo b = v.getPenultimateBlock();
       return b == null ||
           blockManager.checkBlocksProperlyReplicated(
-              src, new BlockInfoContiguous[] { b });
+              src, new BlockInfo[] { b });
     }
   }
 
@@ -3242,7 +3242,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     for (Block b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfoContiguous bi = getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
           if (bi.numNodes() >= blockManager.minReplication) {
@@ -3466,10 +3466,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     final INodeFile pendingFile = iip.getLastINode().asFile();
     int nrBlocks = pendingFile.numBlocks();
-    BlockInfoContiguous[] blocks = pendingFile.getBlocks();
+    BlockInfo[] blocks = pendingFile.getBlocks();
 
     int nrCompleteBlocks;
-    BlockInfoContiguous curBlock = null;
+    BlockInfo curBlock = null;
     for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
@@ -3504,9 +3504,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 BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
+    final BlockInfo lastBlock = pendingFile.getLastBlock();
     BlockUCState lastBlockState = lastBlock.getBlockUCState();
-    BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
+    BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
     boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
@@ -3650,7 +3650,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @VisibleForTesting
-  BlockInfoContiguous getStoredBlock(Block block) {
+  BlockInfo getStoredBlock(Block block) {
     return blockManager.getStoredBlock(block);
   }
   
@@ -3710,7 +3710,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   
       checkNameNodeSafeMode(
           "Cannot commitBlockSynchronization while in safe mode");
-      final BlockInfoContiguous storedBlock = getStoredBlock(
+      final BlockInfo storedBlock = getStoredBlock(
           ExtendedBlock.getLocalBlock(oldBlock));
       if (storedBlock == null) {
         if (deleteblock) {
@@ -3871,7 +3871,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException on error
    */
   @VisibleForTesting
-  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock)
+  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
       throws IOException {
     final INodesInPath iip = INodesInPath.fromINode(pendingFile);
     final String src = iip.getPath();
@@ -4162,7 +4162,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
         while (it.hasNext()) {
           Block b = it.next();
-          BlockInfoContiguous blockInfo = blockManager.getStoredBlock(b);
+          BlockInfo blockInfo = blockManager.getStoredBlock(b);
           if (blockInfo.getBlockCollection().getStoragePolicyID()
               == lpPolicy.getId()) {
             filesToDelete.add(blockInfo.getBlockCollection());
@@ -5105,7 +5105,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
       return;
-    BlockInfoContiguous storedBlock = getStoredBlock(b);
+    BlockInfo storedBlock = getStoredBlock(b);
     if (storedBlock.isComplete()) {
       safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
     }
@@ -5665,7 +5665,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         + "access token for block " + block);
     
     // check stored block state
-    BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
+    BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
     if (storedBlock == null || 
         storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
         throw new IOException(block + 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 1ebdde6..d07ae1f 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,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
@@ -58,7 +58,7 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void updateLengthOfLastBlock(INodeFile f, long lastBlockLength)
       throws IOException {
-    BlockInfoContiguous lastBlock = f.getLastBlock();
+    BlockInfo lastBlock = f.getLastBlock();
     assert (lastBlock != null) : "The last block for path "
         + f.getFullPathName() + " is null when updating its length";
     assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
@@ -74,7 +74,7 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void cleanZeroSizeBlock(final INodeFile f,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfoContiguous[] blocks = f.getBlocks();
+    final BlockInfo[] blocks = f.getBlocks();
     if (blocks != null && blocks.length > 0
         && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
       BlockInfoContiguousUnderConstruction lastUC =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 a6ff6fb..294323c 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
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -121,17 +121,17 @@ public class INodeFile extends INodeWithAdditionalFields
 
   private long header = 0L;
 
-  private BlockInfoContiguous[] blocks;
+  private BlockInfo[] blocks;
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
-            long atime, BlockInfoContiguous[] blklist, short replication,
+            long atime, BlockInfo[] 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, BlockInfoContiguous[] blklist, short replication,
+      long atime, BlockInfo[] blklist, short replication,
       long preferredBlockSize, byte storagePolicyID) {
     super(id, name, permissions, mtime, atime);
     header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
@@ -226,13 +226,13 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection
-  public void setBlock(int index, BlockInfoContiguous blk) {
+  public void setBlock(int index, BlockInfo blk) {
     this.blocks[index] = blk;
   }
 
   @Override // BlockCollection, the file should be under construction
   public BlockInfoContiguousUnderConstruction setLastBlock(
-      BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations)
+      BlockInfo lastBlock, DatanodeStorageInfo[] locations)
       throws IOException {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
@@ -265,7 +265,7 @@ public class INodeFile extends INodeWithAdditionalFields
     BlockInfoContiguousUnderConstruction uc =
         (BlockInfoContiguousUnderConstruction)blocks[size_1];
     //copy to a new list
-    BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
+    BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     setBlocks(newlist);
     return uc;
@@ -420,16 +420,16 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /** @return the blocks of the file. */
   @Override
-  public BlockInfoContiguous[] getBlocks() {
+  public BlockInfo[] getBlocks() {
     return this.blocks;
   }
 
   /** @return blocks of the file corresponding to the snapshot. */
-  public BlockInfoContiguous[] getBlocks(int snapshot) {
+  public BlockInfo[] getBlocks(int snapshot) {
     if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
       return getBlocks();
     FileDiff diff = getDiffs().getDiffById(snapshot);
-    BlockInfoContiguous[] snapshotBlocks =
+    BlockInfo[] snapshotBlocks =
         diff == null ? getBlocks() : diff.getBlocks();
     if(snapshotBlocks != null)
       return snapshotBlocks;
@@ -441,7 +441,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
   void updateBlockCollection() {
     if (blocks != null) {
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo b : blocks) {
         b.setBlockCollection(this);
       }
     }
@@ -457,8 +457,8 @@ public class INodeFile extends INodeWithAdditionalFields
       totalAddedBlocks += f.blocks.length;
     }
     
-    BlockInfoContiguous[] newlist =
-        new BlockInfoContiguous[size + totalAddedBlocks];
+    BlockInfo[] newlist =
+        new BlockInfo[size + totalAddedBlocks];
     System.arraycopy(this.blocks, 0, newlist, 0, size);
     
     for(INodeFile in: inodes) {
@@ -473,12 +473,12 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * add a block to the block list
    */
-  void addBlock(BlockInfoContiguous newblock) {
+  void addBlock(BlockInfo newblock) {
     if (this.blocks == null) {
-      this.setBlocks(new BlockInfoContiguous[]{newblock});
+      this.setBlocks(new BlockInfo[]{newblock});
     } else {
       int size = this.blocks.length;
-      BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
+      BlockInfo[] newlist = new BlockInfo[size + 1];
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       newlist[size] = newblock;
       this.setBlocks(newlist);
@@ -486,7 +486,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Set the blocks. */
-  public void setBlocks(BlockInfoContiguous[] blocks) {
+  public void setBlocks(BlockInfo[] blocks) {
     this.blocks = blocks;
   }
 
@@ -538,7 +538,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
   public void clearFile(ReclaimContext reclaimContext) {
     if (blocks != null && reclaimContext.collectedBlocks != null) {
-      for (BlockInfoContiguous blk : blocks) {
+      for (BlockInfo blk : blocks) {
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
       }
@@ -710,16 +710,16 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
     QuotaCounts counts = new QuotaCounts.Builder().build();
-    final Iterable<BlockInfoContiguous> blocks;
+    final Iterable<BlockInfo> blocks;
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf == null) {
       blocks = Arrays.asList(getBlocks());
     } else {
       // Collect all distinct blocks
-      Set<BlockInfoContiguous> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
+      Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
       List<FileDiff> diffs = sf.getDiffs().asList();
       for(FileDiff diff : diffs) {
-        BlockInfoContiguous[] diffBlocks = diff.getBlocks();
+        BlockInfo[] diffBlocks = diff.getBlocks();
         if (diffBlocks != null) {
           allBlocks.addAll(Arrays.asList(diffBlocks));
         }
@@ -728,7 +728,7 @@ public class INodeFile extends INodeWithAdditionalFields
     }
 
     final short replication = getPreferredBlockReplication();
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       long blockSize = b.isComplete() ? b.getNumBytes() :
           getPreferredBlockSize();
       counts.addStorageSpace(blockSize * replication);
@@ -747,7 +747,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * Return the penultimate allocated block for this file.
    */
-  BlockInfoContiguous getPenultimateBlock() {
+  BlockInfo getPenultimateBlock() {
     if (blocks == null || blocks.length <= 1) {
       return null;
     }
@@ -755,7 +755,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public BlockInfoContiguous getLastBlock() {
+  public BlockInfo getLastBlock() {
     return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
   }
 
@@ -782,7 +782,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public long collectBlocksBeyondMax(final long max,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfoContiguous[] oldBlocks = getBlocks();
+    final BlockInfo[] oldBlocks = getBlocks();
     if (oldBlocks == null)
       return 0;
     // find the minimum n such that the size of the first n blocks > max
@@ -814,17 +814,17 @@ public class INodeFile extends INodeWithAdditionalFields
   void computeQuotaDeltaForTruncate(
       long newLength, BlockStoragePolicy bsps,
       QuotaCounts delta) {
-    final BlockInfoContiguous[] blocks = getBlocks();
+    final BlockInfo[] blocks = getBlocks();
     if (blocks == null || blocks.length == 0) {
       return;
     }
 
     long size = 0;
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       size += b.getNumBytes();
     }
 
-    BlockInfoContiguous[] sblocks = null;
+    BlockInfo[] sblocks = null;
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       FileDiff diff = sf.getDiffs().getLast();
@@ -833,7 +833,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
     for (int i = blocks.length - 1; i >= 0 && size > newLength;
          size -= blocks[i].getNumBytes(), --i) {
-      BlockInfoContiguous bi = blocks[i];
+      BlockInfo bi = blocks[i];
       long truncatedBytes;
       if (size - newLength < bi.getNumBytes()) {
         // Record a full block as the last block will be copied during
@@ -863,20 +863,20 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   void truncateBlocksTo(int n) {
-    final BlockInfoContiguous[] newBlocks;
+    final BlockInfo[] newBlocks;
     if (n == 0) {
-      newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
+      newBlocks = BlockInfo.EMPTY_ARRAY;
     } else {
-      newBlocks = new BlockInfoContiguous[n];
+      newBlocks = new BlockInfo[n];
       System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
     }
     // set new blocks
     setBlocks(newBlocks);
   }
 
-  public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
+  public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
-    BlockInfoContiguous[] oldBlocks = getBlocks();
+    BlockInfo[] oldBlocks = getBlocks();
     if(snapshotBlocks == null || oldBlocks == null)
       return;
     // Skip blocks in common between the file and the snapshot
@@ -900,7 +900,7 @@ public class INodeFile extends INodeWithAdditionalFields
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if(sf == null)
       return;
-    BlockInfoContiguous[] snapshotBlocks =
+    BlockInfo[] snapshotBlocks =
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
@@ -914,12 +914,12 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * @return true if the block is contained in a snapshot or false otherwise.
    */
-  boolean isBlockInLatestSnapshot(BlockInfoContiguous block) {
+  boolean isBlockInLatestSnapshot(BlockInfo block) {
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     if (sf == null || sf.getDiffs() == null) {
       return false;
     }
-    BlockInfoContiguous[] snapshotBlocks = getDiffs()
+    BlockInfo[] snapshotBlocks = getDiffs()
         .findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
     return snapshotBlocks != null &&
         Arrays.asList(snapshotBlocks).contains(block);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 0806f82..f954a58 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
@@ -33,7 +33,7 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
 
@@ -108,11 +108,11 @@ public class LeaseManager {
     for (Long id : getINodeIdWithLeases()) {
       final INodeFile cons = fsnamesystem.getFSDirectory().getInode(id).asFile();
       Preconditions.checkState(cons.isUnderConstruction());
-      BlockInfoContiguous[] blocks = cons.getBlocks();
+      BlockInfo[] blocks = cons.getBlocks();
       if(blocks == null) {
         continue;
       }
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo b : blocks) {
         if(!b.isComplete())
           numUCBlocks++;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 0daf367..39180b1 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
@@ -64,7 +64,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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
@@ -243,7 +243,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       //get blockInfo
       Block block = new Block(Block.getBlockId(blockId));
       //find which file this block belongs to
-      BlockInfoContiguous blockInfo = bm.getStoredBlock(block);
+      BlockInfo 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/4928f547/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 c4cbbc1..363c208 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
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -239,13 +239,13 @@ public class FSImageFormatPBSnapshot {
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
             pbf.getFileSize());
         List<BlockProto> bpl = pbf.getBlocksList();
-        BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
+        BlockInfo[] blocks = new BlockInfo[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
           Block blk = PBHelper.convert(bpl.get(j));
-          BlockInfoContiguous storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
+          BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = fsn.getBlockManager().addBlockCollection(
-                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
+                new BlockInfo(blk, copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 48c5f33..52bf58e 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
@@ -21,7 +21,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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;
@@ -38,7 +38,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 BlockInfoContiguous[] blocks;
+  private BlockInfo[] blocks;
 
   FileDiff(int snapshotId, INodeFile file) {
     super(snapshotId, null, null);
@@ -64,7 +64,7 @@ public class FileDiff extends
    * up to the current {@link #fileSize}.
    * Should be done only once.
    */
-  public void setBlocks(BlockInfoContiguous[] blocks) {
+  public void setBlocks(BlockInfo[] blocks) {
     if(this.blocks != null)
       return;
     int numBlocks = 0;
@@ -73,7 +73,7 @@ public class FileDiff extends
     this.blocks = Arrays.copyOf(blocks, numBlocks);
   }
 
-  public BlockInfoContiguous[] getBlocks() {
+  public BlockInfo[] getBlocks() {
     return blocks;
   }
 
@@ -118,7 +118,7 @@ public class FileDiff extends
     if (blocks == null || collectedBlocks == null) {
       return;
     }
-    for (BlockInfoContiguous blk : blocks) {
+    for (BlockInfo blk : blocks) {
       collectedBlocks.addDeleteBlock(blk);
     }
     blocks = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 62aaccd..6b8388e 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
@@ -21,7 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -59,14 +59,14 @@ public class FileDiffList extends
     }
   }
 
-  public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
+  public BlockInfo[] 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);
-    BlockInfoContiguous[] blocks = null;
+    BlockInfo[] blocks = null;
     for(i = i >= 0 ? i : -i-2; i >= 0; i--) {
       blocks = diffs.get(i).getBlocks();
       if(blocks != null) {
@@ -76,14 +76,14 @@ public class FileDiffList extends
     return blocks;
   }
 
-  public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
+  public BlockInfo[] 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);
-    BlockInfoContiguous[] blocks = null;
+    BlockInfo[] blocks = null;
     for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();
       if (blocks != null) {
@@ -100,7 +100,7 @@ public class FileDiffList extends
    */
   void combineAndCollectSnapshotBlocks(
       INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
-    BlockInfoContiguous[] removedBlocks = removed.getBlocks();
+    BlockInfo[] removedBlocks = removed.getBlocks();
     if (removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
@@ -114,10 +114,10 @@ public class FileDiffList extends
     if (earlierDiff != null) {
       earlierDiff.setBlocks(removedBlocks);
     }
-    BlockInfoContiguous[] earlierBlocks =
-        (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
+    BlockInfo[] earlierBlocks =
+        (earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
-    BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
+    BlockInfo[] 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;
@@ -129,7 +129,7 @@ public class FileDiffList extends
       break;
     }
     // Check if last block is part of truncate recovery
-    BlockInfoContiguous lastBlock = file.getLastBlock();
+    BlockInfo lastBlock = file.getLastBlock();
     Block dontRemoveBlock = null;
     if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 555a662..f017d4b 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
@@ -22,7 +22,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
@@ -208,7 +208,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
 
     // Collect blocks that should be deleted
     FileDiff last = diffs.getLast();
-    BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
+    BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
     if(snapshotBlocks == null)
       file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
     else

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 a88a459..50b85c0 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
@@ -108,7 +108,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -1609,7 +1609,7 @@ public class DFSTestUtil {
   public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
       ExtendedBlock blk) {
     BlockManager bm0 = nn.getNamesystem().getBlockManager();
-    BlockInfoContiguous storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
+    BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
         storedBlock instanceof BlockInfoContiguousUnderConstruction);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 1ab7427..1f18014 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -671,7 +671,7 @@ public class TestDecommission {
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
         public Boolean get() {
-          BlockInfoContiguous info =
+          BlockInfo info =
               blockManager.getStoredBlock(b.getLocalBlock());
           int count = 0;
           StringBuilder sb = new StringBuilder("Replica locations: ");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 c5662ca..96d4d8b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
@@ -49,7 +49,7 @@ public class TestBlockInfo {
 
   @Test
   public void testIsDeleted() {
-    BlockInfoContiguous blockInfo = new BlockInfoContiguous((short) 3);
+    BlockInfo blockInfo = new BlockInfo((short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     blockInfo.setBlockCollection(bc);
     Assert.assertFalse(blockInfo.isDeleted());
@@ -59,7 +59,7 @@ public class TestBlockInfo {
 
   @Test
   public void testAddStorage() throws Exception {
-    BlockInfoContiguous blockInfo = new BlockInfoContiguous((short) 3);
+    BlockInfo blockInfo = new BlockInfo((short) 3);
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
@@ -71,9 +71,9 @@ public class TestBlockInfo {
 
   @Test
   public void testCopyConstructor() {
-    BlockInfoContiguous old = new BlockInfoContiguous((short) 3);
+    BlockInfo old = new BlockInfo((short) 3);
     try {
-      BlockInfoContiguous copy = new BlockInfoContiguous(old);
+      BlockInfo copy = new BlockInfo(old);
       assertEquals(old.getBlockCollection(), copy.getBlockCollection());
       assertEquals(old.getCapacity(), copy.getCapacity());
     } catch (Exception e) {
@@ -88,11 +88,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;
-    BlockInfoContiguous[] blockInfos = new BlockInfoContiguous[NUM_BLOCKS];
+    BlockInfo[] blockInfos = new BlockInfo[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 BlockInfoContiguous((short) 3);
+      blockInfos[i] = new BlockInfo((short) 3);
       storage1.addBlock(blockInfos[i]);
     }
 
@@ -111,14 +111,14 @@ public class TestBlockInfo {
 
     DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1");
     ArrayList<Block> blockList = new ArrayList<Block>(MAX_BLOCKS);
-    ArrayList<BlockInfoContiguous> blockInfoList = new ArrayList<BlockInfoContiguous>();
+    ArrayList<BlockInfo> blockInfoList = new ArrayList<BlockInfo>();
     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 BlockInfoContiguous(blockList.get(i), (short) 3));
+      blockInfoList.add(new BlockInfo(blockList.get(i), (short) 3));
       dd.addBlock(blockInfoList.get(i));
 
       // index of the datanode should be 0
@@ -129,7 +129,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<BlockInfoContiguous> it = dd.getBlockIterator();
+    Iterator<BlockInfo> it = dd.getBlockIterator();
     int len = 0;
     while (it.hasNext()) {
       it.next();
@@ -151,7 +151,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...");
 
-    BlockInfoContiguous temp = dd.getBlockListHeadForTesting();
+    BlockInfo temp = dd.getBlockListHeadForTesting();
     curIndex = 0;
     headIndex = 0;
     dd.moveBlockToHead(temp, curIndex, headIndex);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 58210c1..bf46ed7 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
@@ -147,7 +147,7 @@ public class TestBlockManager {
   private void doBasicTest(int testIndex) {
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
 
     DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
     assertEquals(2, pipeline.length);
@@ -179,7 +179,7 @@ public class TestBlockManager {
     // Block originally on A1, A2, B1
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     // Decommission two of the nodes (A1, A2)
     List<DatanodeDescriptor> decomNodes = startDecommission(0, 1);
@@ -223,7 +223,7 @@ public class TestBlockManager {
     // Block originally on A1, A2, B1
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     // Decommission all of the nodes
     List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 3);
@@ -276,7 +276,7 @@ public class TestBlockManager {
     // Block originally on A1, A2, B1
     List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
     List<DatanodeDescriptor> origNodes = getNodes(origStorages);
-    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     
     // Decommission all of the nodes in rack A
     List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 2);
@@ -335,7 +335,7 @@ public class TestBlockManager {
   private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) {
     // Originally on only nodes in rack A.
     List<DatanodeDescriptor> origNodes = rackA;
-    BlockInfoContiguous blockInfo = addBlockOnNodes(testIndex, origNodes);
+    BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
     DatanodeStorageInfo pipeline[] = scheduleSingleReplication(blockInfo);
     
     assertEquals(2, pipeline.length); // single new copy
@@ -378,7 +378,7 @@ public class TestBlockManager {
    * Tell the block manager that replication is completed for the given
    * pipeline.
    */
-  private void fulfillPipeline(BlockInfoContiguous blockInfo,
+  private void fulfillPipeline(BlockInfo blockInfo,
       DatanodeStorageInfo[] pipeline) throws IOException {
     for (int i = 1; i < pipeline.length; i++) {
       DatanodeStorageInfo storage = pipeline[i];
@@ -387,9 +387,9 @@ public class TestBlockManager {
     }
   }
 
-  private BlockInfoContiguous blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
+  private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
     Block block = new Block(blkId);
-    BlockInfoContiguous blockInfo = new BlockInfoContiguous(block, (short) 3);
+    BlockInfo blockInfo = new BlockInfo(block, (short) 3);
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
@@ -431,10 +431,10 @@ public class TestBlockManager {
     return nodes;
   }
   
-  private BlockInfoContiguous addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
+  private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short)3).when(bc).getPreferredBlockReplication();
-    BlockInfoContiguous blockInfo = blockOnNodes(blockId, nodes);
+    BlockInfo blockInfo = blockOnNodes(blockId, nodes);
 
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
@@ -677,21 +677,21 @@ public class TestBlockManager {
 
     // blk_42 is finalized.
     long receivedBlockId = 42;  // arbitrary
-    BlockInfoContiguous receivedBlock = addBlockToBM(receivedBlockId);
+    BlockInfo receivedBlock = addBlockToBM(receivedBlockId);
     rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivedBlock),
         ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, null));
     builder.add(new FinalizedReplica(receivedBlock, null, null));
 
     // blk_43 is under construction.
     long receivingBlockId = 43;
-    BlockInfoContiguous receivingBlock = addUcBlockToBM(receivingBlockId);
+    BlockInfo receivingBlock = addUcBlockToBM(receivingBlockId);
     rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivingBlock),
         ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, null));
     builder.add(new ReplicaBeingWritten(receivingBlock, null, null, null));
 
     // blk_44 has 2 records in IBR. It's finalized. So full BR has 1 record.
     long receivingReceivedBlockId = 44;
-    BlockInfoContiguous receivingReceivedBlock = addBlockToBM(receivingReceivedBlockId);
+    BlockInfo receivingReceivedBlock = addBlockToBM(receivingReceivedBlockId);
     rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivingReceivedBlock),
         ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, null));
     rdbiList.add(new ReceivedDeletedBlockInfo(new Block(receivingReceivedBlock),
@@ -709,7 +709,7 @@ public class TestBlockManager {
 
     // blk_46 exists in DN for a long time, so it's in full BR, but not in IBR.
     long existedBlockId = 46;
-    BlockInfoContiguous existedBlock = addBlockToBM(existedBlockId);
+    BlockInfo existedBlock = addBlockToBM(existedBlockId);
     builder.add(new FinalizedReplica(existedBlock, null, null));
 
     // process IBR and full BR
@@ -735,17 +735,17 @@ public class TestBlockManager {
         (ds) >= 0);
   }
 
-  private BlockInfoContiguous addBlockToBM(long blkId) {
+  private BlockInfo addBlockToBM(long blkId) {
     Block block = new Block(blkId);
-    BlockInfoContiguous blockInfo =
-        new BlockInfoContiguous(block, (short) 3);
+    BlockInfo blockInfo =
+        new BlockInfo(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
   }
 
-  private BlockInfoContiguous addUcBlockToBM(long blkId) {
+  private BlockInfo addUcBlockToBM(long blkId) {
     Block block = new Block(blkId);
     BlockInfoContiguousUnderConstruction blockInfo =
         new BlockInfoContiguousUnderConstruction(block, (short) 3);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 7cdb423..fe639e4 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());
-    BlockInfoContiguous blk = new BlockInfoContiguous(new Block(1L), (short) 1);
-    BlockInfoContiguous blk1 = new BlockInfoContiguous(new Block(2L), (short) 2);
+    BlockInfo blk = new BlockInfo(new Block(1L), (short) 1);
+    BlockInfo blk1 = new BlockInfo(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/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index 259404e..844c5d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -159,7 +159,7 @@ public class TestPendingReplication {
         DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, TIMEOUT);
     MiniDFSCluster cluster = null;
     Block block;
-    BlockInfoContiguous blockInfo;
+    BlockInfo blockInfo;
     try {
       cluster =
           new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_COUNT).build();
@@ -178,7 +178,7 @@ public class TestPendingReplication {
       //
 
       block = new Block(1, 1, 0);
-      blockInfo = new BlockInfoContiguous(block, (short) 3);
+      blockInfo = new BlockInfo(block, (short) 3);
 
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 3226578..f317723 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
@@ -34,7 +34,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
 
@@ -1219,7 +1218,7 @@ public class TestReplicationPolicy {
     chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
+    final BlockInfo info = new BlockInfo(block1, (short) 1);
     final BlockCollection mbc = mock(BlockCollection.class);
     when(mbc.getLastBlock()).thenReturn(info);
     when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
@@ -1241,12 +1240,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(BlockInfoContiguous.class))).thenReturn(true);
-    when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn
+    when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
+    when(storage.addBlock(any(BlockInfo.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
     ucBlock.addStorage(storage);
 
-    when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any()))
+    when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
     .thenReturn(ucBlock);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 0349251..3f96c0c 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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);
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile];
+    BlockInfo[] blocks = new BlockInfo[blocksPerFile];
     for (int iB = 0; iB < blocksPerFile; ++iB) {
       blocks[iB] = 
-       new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
+       new BlockInfo(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, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
+          p, 0L, 0L, BlockInfo.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/4928f547/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 a417c3d..301ee25 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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();
-    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
+    BlockInfo[] 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();
-    BlockInfoContiguous[] file2Blocks = file2Node.getBlocks();
+    BlockInfo[] 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();
-    BlockInfoContiguous[] file3Blocks = file3Node.getBlocks();
+    BlockInfo[] 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();
-    BlockInfoContiguous[] file4Blocks = file4Node.getBlocks();
+    BlockInfo[] 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();
-      BlockInfoContiguous[] fileBlocks = fileNode.getBlocks();
+      BlockInfo[] 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/4928f547/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 1fbe160..f372bec 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,7 +35,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 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;
@@ -91,12 +91,12 @@ public class TestBlockUnderConstruction {
         " isUnderConstruction = " + inode.isUnderConstruction() +
         " expected to be " + isFileOpen,
         inode.isUnderConstruction() == isFileOpen);
-    BlockInfoContiguous[] blocks = inode.getBlocks();
+    BlockInfo[] blocks = inode.getBlocks();
     assertTrue("File does not have blocks: " + inode.toString(),
         blocks != null && blocks.length > 0);
     
     int idx = 0;
-    BlockInfoContiguous curBlock;
+    BlockInfo curBlock;
     // all blocks but the last two should be regular blocks
     for(; idx < blocks.length - 2; idx++) {
       curBlock = blocks[idx];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 ea560fe..8c9da01 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,7 +22,7 @@ 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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -78,7 +78,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(BlockInfoContiguous.class));
+        any(INodeFile.class), any(BlockInfo.class));
     doReturn(mock(FSEditLog.class)).when(namesystemSpy).getEditLog();
 
     return namesystemSpy;
@@ -106,7 +106,7 @@ public class TestCommitBlockSynchronization {
         lastBlock, genStamp, length, false, false, newTargets, null);
 
     // Simulate 'completing' the block.
-    BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
+    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)
@@ -178,7 +178,7 @@ public class TestCommitBlockSynchronization {
     namesystemSpy.commitBlockSynchronization(
         lastBlock, genStamp, length, true, false, newTargets, null);
 
-    BlockInfoContiguous completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
+    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 1e42e34..f654107 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
@@ -69,7 +69,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -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, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
+            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
         inode.toUnderConstruction("", "");
 
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 5653df5..27a1bd3 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
@@ -39,7 +39,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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
@@ -105,7 +105,7 @@ public class TestFSImage {
       INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile();
       assertEquals("hello".length(), file2Node.computeFileSize());
       assertTrue(file2Node.isUnderConstruction());
-      BlockInfoContiguous[] blks = file2Node.getBlocks();
+      BlockInfo[] 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/4928f547/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 7f31f84..eabd0c8 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
@@ -79,7 +79,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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -885,7 +885,7 @@ public class TestFsck {
       // intentionally corrupt NN data structure
       INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode
           (fileName, true);
-      final BlockInfoContiguous[] blocks = node.getBlocks();
+      final BlockInfo[] 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/4928f547/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
index 4d0f994..7a3a8d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
@@ -21,7 +21,7 @@ import org.apache.commons.io.Charsets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -125,7 +125,7 @@ public class TestGetBlockLocations {
         FsPermission.createImmutable((short) 0x1ff));
     final INodeFile file = new INodeFile(
         MOCK_INODE_ID, FILE_NAME.getBytes(Charsets.UTF_8),
-        perm, 1, 1, new BlockInfoContiguous[] {}, (short) 1,
+        perm, 1, 1, new BlockInfo[] {}, (short) 1,
         DFS_BLOCK_SIZE_DEFAULT);
     fsn.getFSDirectory().addINode(iip, file);
     return fsn;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4928f547/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 b45d2f6..21b67a7 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
@@ -60,7 +60,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.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.IOUtils;
@@ -290,7 +290,7 @@ public class TestINodeFile {
       iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
           preferredBlockSize);
       iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
-      BlockInfoContiguous newblock = new BlockInfoContiguous(replication);
+      BlockInfo newblock = new BlockInfo(replication);
       iNodes[i].addBlock(newblock);
     }