Browse Source

HDFS-8482. Rename BlockInfoContiguous to BlockInfo. Contributed by Zhe Zhang.

Andrew Wang 10 năm trước cách đây
mục cha
commit
4928f54733
47 tập tin đã thay đổi với 386 bổ sung391 xóa
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  3. 40 40
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  4. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
  5. 62 62
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  6. 16 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  7. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
  8. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  9. 11 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  10. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
  11. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  12. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  13. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  14. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  15. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  16. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  17. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  18. 21 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  19. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
  20. 37 37
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  21. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  22. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  23. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  24. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
  25. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  26. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  28. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  29. 10 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
  30. 18 18
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
  32. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
  33. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  34. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  35. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
  36. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  37. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
  38. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  39. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  41. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
  42. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  43. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
  44. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
  45. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java
  46. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
  47. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

+ 2 - 0
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

+ 4 - 4
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;
 
   /**

+ 40 - 40
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java → hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -26,13 +26,13 @@ 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 {@link BlockCollection} it is part of and datanodes where the replicas of
  * the block are stored.
  */
 @InterfaceAudience.Private
-public class BlockInfoContiguous extends Block
+public class BlockInfo extends Block
     implements LightWeightGSet.LinkedElement {
-  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
+  public static final BlockInfo[] EMPTY_ARRAY = {};
 
   private BlockCollection bc;
 
@@ -45,7 +45,7 @@ public class BlockInfoContiguous extends Block
    * {@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
@@ -57,12 +57,12 @@ public class BlockInfoContiguous extends Block
    * Construct an entry for blocksmap
    * @param replication the block's replication factor
    */
-  public BlockInfoContiguous(short replication) {
+  public BlockInfo(short replication) {
     this.triplets = new Object[3*replication];
     this.bc = null;
   }
-  
-  public BlockInfoContiguous(Block blk, short replication) {
+
+  public BlockInfo(Block blk, short replication) {
     super(blk);
     this.triplets = new Object[3*replication];
     this.bc = null;
@@ -73,7 +73,7 @@ public class BlockInfoContiguous extends Block
    * This is used to convert BlockInfoUnderConstruction
    * @param from BlockInfo to copy from.
    */
-  protected BlockInfoContiguous(BlockInfoContiguous from) {
+  protected BlockInfo(BlockInfo from) {
     super(from);
     this.triplets = new Object[from.triplets.length];
     this.bc = from.bc;
@@ -102,22 +102,22 @@ public class BlockInfoContiguous extends Block
     return (DatanodeStorageInfo)triplets[index*3];
   }
 
-  private BlockInfoContiguous getPrevious(int index) {
+  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";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
-    assert info == null || 
-        info.getClass().getName().startsWith(BlockInfoContiguous.class.getName()) :
+    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;
   }
 
-  BlockInfoContiguous getNext(int index) {
+  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";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
+    BlockInfo info = (BlockInfo)triplets[index*3+2];
     assert info == null || info.getClass().getName().startsWith(
-        BlockInfoContiguous.class.getName()) :
+        BlockInfo.class.getName()) :
         "BlockInfo is expected at " + index*3;
     return info;
   }
@@ -136,10 +136,10 @@ public class BlockInfoContiguous extends Block
    * @param to - block to be set to previous on the list of blocks
    * @return current previous block on the list of blocks
    */
-  private BlockInfoContiguous setPrevious(int index, BlockInfoContiguous to) {
+  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";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+1];
+    BlockInfo info = (BlockInfo)triplets[index*3+1];
     triplets[index*3+1] = to;
     return info;
   }
@@ -152,10 +152,10 @@ public class BlockInfoContiguous extends Block
    * @param to - block to be set to next on the list of blocks
    *    * @return current next block on the list of blocks
    */
-  private BlockInfoContiguous setNext(int index, BlockInfoContiguous to) {
+  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";
-    BlockInfoContiguous info = (BlockInfoContiguous)triplets[index*3+2];
+    BlockInfo info = (BlockInfo)triplets[index*3+2];
     triplets[index*3+2] = to;
     return info;
   }
@@ -175,7 +175,7 @@ public class BlockInfoContiguous extends Block
     int last = numNodes();
     if(triplets.length >= (last+num)*3)
       return last;
-    /* Not enough space left. Create a new array. Should normally 
+    /* 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];
@@ -215,18 +215,18 @@ public class BlockInfoContiguous extends Block
     int dnIndex = findStorageInfo(storage);
     if(dnIndex < 0) // the node is not found
       return false;
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : 
+    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 
+    int lastNode = numNodes()-1;
+    // replace current node triplet by the lastNode one
     setStorageInfo(dnIndex, getStorageInfo(lastNode));
-    setNext(dnIndex, getNext(lastNode)); 
-    setPrevious(dnIndex, getPrevious(lastNode)); 
+    setNext(dnIndex, getNext(lastNode));
+    setPrevious(dnIndex, getPrevious(lastNode));
     // set the last triplet to null
     setStorageInfo(lastNode, null);
-    setNext(lastNode, null); 
-    setPrevious(lastNode, null); 
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
     return true;
   }
 
@@ -245,7 +245,7 @@ public class BlockInfoContiguous extends Block
     }
     return null;
   }
-  
+
   /**
    * Find specified DatanodeStorageInfo.
    * @return index or -1 if not found.
@@ -265,16 +265,16 @@ public class BlockInfoContiguous extends Block
   }
 
   /**
-   * Insert this block into the head of the list of blocks 
+   * 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,
+  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 : 
+    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);
@@ -284,14 +284,14 @@ public class BlockInfoContiguous extends Block
   }
 
   /**
-   * Remove this block from the list of blocks 
+   * 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 
+   * 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,
+  BlockInfo listRemove(BlockInfo head,
       DatanodeStorageInfo storage) {
     if(head == null)
       return null;
@@ -299,8 +299,8 @@ public class BlockInfoContiguous extends Block
     if(dnIndex < 0) // this block is not on the data-node list
       return head;
 
-    BlockInfoContiguous next = this.getNext(dnIndex);
-    BlockInfoContiguous prev = this.getPrevious(dnIndex);
+    BlockInfo next = this.getNext(dnIndex);
+    BlockInfo prev = this.getPrevious(dnIndex);
     this.setNext(dnIndex, null);
     this.setPrevious(dnIndex, null);
     if(prev != null)
@@ -318,13 +318,13 @@ public class BlockInfoContiguous extends Block
    *
    * @return the new head of the list.
    */
-  public BlockInfoContiguous moveBlockToHead(BlockInfoContiguous head,
+  public BlockInfo moveBlockToHead(BlockInfo head,
       DatanodeStorageInfo storage, int curIndex, int headIndex) {
     if (head == this) {
       return this;
     }
-    BlockInfoContiguous next = this.setNext(curIndex, head);
-    BlockInfoContiguous prev = this.setPrevious(curIndex, null);
+    BlockInfo next = this.setNext(curIndex, head);
+    BlockInfo prev = this.setPrevious(curIndex, null);
 
     head.setPrevious(headIndex, this);
     prev.setNext(prev.findStorageInfo(storage), next);
@@ -346,7 +346,7 @@ public class BlockInfoContiguous extends Block
 
   /**
    * Is this block complete?
-   * 
+   *
    * @return true if the state of the block is {@link BlockUCState#COMPLETE}
    */
   public boolean isComplete() {

+ 3 - 3
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 */

+ 62 - 62
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. */

+ 16 - 16
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--) {

+ 7 - 7
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();

+ 8 - 8
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));
   }
 

+ 11 - 14
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;
   }
 

+ 11 - 11
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) {

+ 7 - 7
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);

+ 2 - 4
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,

+ 4 - 4
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);

+ 7 - 7
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);

+ 7 - 7
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);
         }

+ 5 - 6
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));

+ 3 - 3
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) {

+ 21 - 21
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 + 

+ 3 - 3
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 =

+ 37 - 37
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);

+ 3 - 3
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++;
       }

+ 2 - 2
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);

+ 4 - 4
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;
         }

+ 5 - 5
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;

+ 10 - 10
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)) {

+ 2 - 2
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

+ 2 - 2
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);

+ 2 - 2
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: ");

+ 10 - 10
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);

+ 18 - 18
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);

+ 2 - 2
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

+ 2 - 2
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(

+ 4 - 5
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);

+ 4 - 4
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);

+ 6 - 6
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());

+ 3 - 3
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];

+ 4 - 4
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)

+ 2 - 2
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);

+ 2 - 2
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

+ 2 - 2
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
       

+ 2 - 2
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;

+ 2 - 2
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);
     }
     

+ 8 - 8
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();

+ 2 - 2
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>

+ 3 - 3
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

+ 12 - 12
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());
     

+ 8 - 8
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));
     }