Browse Source

HDFS-8489. Subclass BlockInfo to represent contiguous blocks. Contributed by Zhe Zhang.

(cherry picked from commit cdc13efb1af54d931585d25c5ba696a012412828)
Jing Zhao 10 years ago
parent
commit
640ea994ed
20 changed files with 185 additions and 99 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 16 58
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  3. 123 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
  4. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
  5. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  6. 1 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  7. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  8. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  9. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  10. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  11. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  12. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
  13. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
  16. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  17. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
  18. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  19. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java
  20. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -248,6 +248,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8443. Document dfs.namenode.service.handler.count in hdfs-site.xml.
     (J.Andreina via aajisaka)
 
+    HDFS-8489. Subclass BlockInfo to represent contiguous blocks.
+    (Zhe Zhang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 16 - 58
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.util.LightWeightGSet;
  * the block are stored.
  */
 @InterfaceAudience.Private
-public class BlockInfo extends Block
+public abstract class  BlockInfo extends Block
     implements LightWeightGSet.LinkedElement {
   public static final BlockInfo[] EMPTY_ARRAY = {};
 
@@ -54,7 +54,7 @@ public class BlockInfo extends Block
    * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
    * bytes using the triplets.
    */
-  private Object[] triplets;
+  protected Object[] triplets;
 
   /**
    * Construct an entry for blocksmap
@@ -105,7 +105,7 @@ public class BlockInfo extends Block
     return (DatanodeStorageInfo)triplets[index*3];
   }
 
-  private BlockInfo getPrevious(int index) {
+  BlockInfo getPrevious(int index) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
@@ -125,7 +125,7 @@ public class BlockInfo extends Block
     return info;
   }
 
-  private void setStorageInfo(int index, DatanodeStorageInfo storage) {
+  void setStorageInfo(int index, DatanodeStorageInfo storage) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
     triplets[index*3] = storage;
@@ -139,7 +139,7 @@ public class BlockInfo extends Block
    * @param to - block to be set to previous on the list of blocks
    * @return current previous block on the list of blocks
    */
-  private BlockInfo setPrevious(int index, BlockInfo to) {
+  BlockInfo setPrevious(int index, BlockInfo to) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+1];
@@ -155,7 +155,7 @@ public class BlockInfo extends Block
    * @param to - block to be set to next on the list of blocks
    *    * @return current next block on the list of blocks
    */
-  private BlockInfo setNext(int index, BlockInfo to) {
+  BlockInfo setNext(int index, BlockInfo to) {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound";
     BlockInfo info = (BlockInfo)triplets[index*3+2];
@@ -170,68 +170,26 @@ public class BlockInfo extends Block
   }
 
   /**
-   * Ensure that there is enough  space to include num more triplets.
-   * @return first free triplet index.
+   * Count the number of data-nodes the block belongs to.
    */
-  private int ensureCapacity(int num) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    int last = numNodes();
-    if(triplets.length >= (last+num)*3)
-      return last;
-    /* Not enough space left. Create a new array. Should normally
-     * happen only when replication is manually increased by the user. */
-    Object[] old = triplets;
-    triplets = new Object[(last+num)*3];
-    System.arraycopy(old, 0, triplets, 0, last*3);
-    return last;
-  }
+  public abstract int numNodes();
 
   /**
-   * Count the number of data-nodes the block belongs to.
+   * Add a {@link DatanodeStorageInfo} location for a block.
    */
-  public int numNodes() {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-    for(int idx = getCapacity()-1; idx >= 0; idx--) {
-      if(getDatanode(idx) != null)
-        return idx+1;
-    }
-    return 0;
-  }
+  abstract boolean addStorage(DatanodeStorageInfo storage);
 
   /**
-   * Add a {@link DatanodeStorageInfo} location for a block
+   * Remove {@link DatanodeStorageInfo} location for a block
    */
-  boolean addStorage(DatanodeStorageInfo storage) {
-    // find the last null node
-    int lastNode = ensureCapacity(1);
-    setStorageInfo(lastNode, storage);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
-  }
+  abstract boolean removeStorage(DatanodeStorageInfo storage);
+
 
   /**
-   * Remove {@link DatanodeStorageInfo} location for a block
+   * Replace the current BlockInfo with the new one in corresponding
+   * DatanodeStorageInfo's linked list
    */
-  boolean removeStorage(DatanodeStorageInfo storage) {
-    int dnIndex = findStorageInfo(storage);
-    if(dnIndex < 0) // the node is not found
-      return false;
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
-      "Block is still in the list and must be removed first.";
-    // find the last not null node
-    int lastNode = numNodes()-1;
-    // replace current node triplet by the lastNode one
-    setStorageInfo(dnIndex, getStorageInfo(lastNode));
-    setNext(dnIndex, getNext(lastNode));
-    setPrevious(dnIndex, getPrevious(lastNode));
-    // set the last triplet to null
-    setStorageInfo(lastNode, null);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
-  }
+  abstract void replaceBlock(BlockInfo newBlock);
 
   /**
    * Find specified DatanodeStorageInfo.

+ 123 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java

@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.blockmanagement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Subclass of {@link BlockInfo}, used for a block with replication scheme.
+ */
+@InterfaceAudience.Private
+public class BlockInfoContiguous extends BlockInfo {
+  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
+
+  public BlockInfoContiguous(short size) {
+    super(size);
+  }
+
+  public BlockInfoContiguous(Block blk, short size) {
+    super(blk, size);
+  }
+
+  /**
+   * Copy construction.
+   * This is used to convert BlockReplicationInfoUnderConstruction
+   * @param from BlockReplicationInfo to copy from.
+   */
+  protected BlockInfoContiguous(BlockInfoContiguous from) {
+    super(from);
+  }
+
+  /**
+   * Ensure that there is enough  space to include num more triplets.
+   * @return first free triplet index.
+   */
+  private int ensureCapacity(int num) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    int last = numNodes();
+    if (triplets.length >= (last+num)*3) {
+      return last;
+    }
+    /* Not enough space left. Create a new array. Should normally
+     * happen only when replication is manually increased by the user. */
+    Object[] old = triplets;
+    triplets = new Object[(last+num)*3];
+    System.arraycopy(old, 0, triplets, 0, last * 3);
+    return last;
+  }
+
+  @Override
+  boolean addStorage(DatanodeStorageInfo storage) {
+    // find the last null node
+    int lastNode = ensureCapacity(1);
+    setStorageInfo(lastNode, storage);
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
+    return true;
+  }
+
+  @Override
+  boolean removeStorage(DatanodeStorageInfo storage) {
+    int dnIndex = findStorageInfo(storage);
+    if (dnIndex < 0) { // the node is not found
+      return false;
+    }
+    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
+        "Block is still in the list and must be removed first.";
+    // find the last not null node
+    int lastNode = numNodes()-1;
+    // replace current node triplet by the lastNode one
+    setStorageInfo(dnIndex, getStorageInfo(lastNode));
+    setNext(dnIndex, getNext(lastNode));
+    setPrevious(dnIndex, getPrevious(lastNode));
+    // set the last triplet to null
+    setStorageInfo(lastNode, null);
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
+    return true;
+  }
+
+  @Override
+  public int numNodes() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+
+    for (int idx = getCapacity()-1; idx >= 0; idx--) {
+      if (getDatanode(idx) != null) {
+        return idx + 1;
+      }
+    }
+    return 0;
+  }
+
+  @Override
+  void replaceBlock(BlockInfo newBlock) {
+    assert newBlock instanceof BlockInfoContiguous;
+    for (int i = this.numNodes() - 1; i >= 0; i--) {
+      final DatanodeStorageInfo storage = this.getStorageInfo(i);
+      final boolean removed = storage.removeBlock(this);
+      assert removed : "currentBlock not found.";
+
+      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
+          newBlock);
+      assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
+          "newBlock already exists.";
+    }
+  }
+}

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

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1767,7 +1767,8 @@ public class BlockManager {
 
     BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
         Reason reasonCode) {
-      this(new BlockInfo(stored), stored, reason, reasonCode);
+      this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
+          reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -2146,7 +2147,7 @@ public class BlockManager {
 
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
-    BlockInfo delimiter = new BlockInfo(new Block(), (short) 1);
+    BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1);
     AddBlockResult result = storageInfo.addBlock(delimiter);
     assert result == AddBlockResult.ADDED 
         : "Delimiting block cannot be present in the node";

+ 1 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java

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

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -1058,7 +1059,7 @@ public class FSEditLogLoader {
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
-          newBI = new BlockInfo(newBlock,
+          newBI = new BlockInfoContiguous(newBlock,
               file.getPreferredBlockReplication());
         }
         fsNamesys.getBlockManager().addBlockCollection(newBI, file);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -755,7 +756,7 @@ public class FSImageFormat {
       // read blocks
       BlockInfo[] blocks = new BlockInfo[numBlocks];
       for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfo(replication);
+        blocks[j] = new BlockInfoContiguous(replication);
         blocks[j].readFields(in);
       }
 

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
@@ -324,7 +325,8 @@ public final class FSImageFormatPBINode {
 
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0, e = bp.size(); i < e; ++i) {
-        blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
+        blocks[i] =
+            new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
       }
       final PermissionStatus permissions = loadPermission(f.getPermission(),
           parent.getLoaderContext().getStringTable());

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
@@ -131,7 +132,7 @@ public class FSImageSerialization {
     int i = 0;
     for (; i < numBlocks-1; i++) {
       blk.readFields(in);
-      blocks[i] = new BlockInfo(blk, blockReplication);
+      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
     }
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.AclEntryStatusFormat;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -246,7 +247,7 @@ public class FSImageFormatPBSnapshot {
           BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = fsn.getBlockManager().addBlockCollection(
-                new BlockInfo(blk, copy.getFileReplication()), file);
+                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;
         }

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java

@@ -49,7 +49,7 @@ public class TestBlockInfo {
 
   @Test
   public void testIsDeleted() {
-    BlockInfo blockInfo = new BlockInfo((short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     blockInfo.setBlockCollection(bc);
     Assert.assertFalse(blockInfo.isDeleted());
@@ -59,7 +59,7 @@ public class TestBlockInfo {
 
   @Test
   public void testAddStorage() throws Exception {
-    BlockInfo blockInfo = new BlockInfo((short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
 
     final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1");
 
@@ -71,9 +71,9 @@ public class TestBlockInfo {
 
   @Test
   public void testCopyConstructor() {
-    BlockInfo old = new BlockInfo((short) 3);
+    BlockInfo old = new BlockInfoContiguous((short) 3);
     try {
-      BlockInfo copy = new BlockInfo(old);
+      BlockInfo copy = new BlockInfoContiguous((BlockInfoContiguous)old);
       assertEquals(old.getBlockCollection(), copy.getBlockCollection());
       assertEquals(old.getCapacity(), copy.getCapacity());
     } catch (Exception e) {
@@ -92,7 +92,7 @@ public class TestBlockInfo {
 
     // Create a few dummy blocks and add them to the first storage.
     for (int i = 0; i < NUM_BLOCKS; ++i) {
-      blockInfos[i] = new BlockInfo((short) 3);
+      blockInfos[i] = new BlockInfoContiguous((short) 3);
       storage1.addBlock(blockInfos[i]);
     }
 
@@ -118,7 +118,7 @@ public class TestBlockInfo {
     LOG.info("Building block list...");
     for (int i = 0; i < MAX_BLOCKS; i++) {
       blockList.add(new Block(i, 0, GenerationStamp.LAST_RESERVED_STAMP));
-      blockInfoList.add(new BlockInfo(blockList.get(i), (short) 3));
+      blockInfoList.add(new BlockInfoContiguous(blockList.get(i), (short) 3));
       dd.addBlock(blockInfoList.get(i));
 
       // index of the datanode should be 0
@@ -182,4 +182,4 @@ public class TestBlockInfo {
           blockInfoList.get(j), dd.getBlockListHeadForTesting());
     }
   }
-}
+}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -389,7 +389,7 @@ public class TestBlockManager {
 
   private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
     Block block = new Block(blkId);
-    BlockInfo blockInfo = new BlockInfo(block, (short) 3);
+    BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
@@ -738,7 +738,7 @@ public class TestBlockManager {
   private BlockInfo addBlockToBM(long blkId) {
     Block block = new Block(blkId);
     BlockInfo blockInfo =
-        new BlockInfo(block, (short) 3);
+        new BlockInfoContiguous(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);

+ 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());
-    BlockInfo blk = new BlockInfo(new Block(1L), (short) 1);
-    BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2);
+    BlockInfo blk = new BlockInfoContiguous(new Block(1L), (short) 1);
+    BlockInfo blk1 = new BlockInfoContiguous(new Block(2L), (short) 2);
     DatanodeStorageInfo[] storages = dd.getStorageInfos();
     assertTrue(storages.length > 0);
     // add first block

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java

@@ -178,7 +178,7 @@ public class TestPendingReplication {
       //
 
       block = new Block(1, 1, 0);
-      blockInfo = new BlockInfo(block, (short) 3);
+      blockInfo = new BlockInfoContiguous(block, (short) 3);
 
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 
@@ -69,8 +70,8 @@ public class CreateEditsLog {
     BlockInfo[] blocks = new BlockInfo[blocksPerFile];
     for (int iB = 0; iB < blocksPerFile; ++iB) {
       blocks[iB] = 
-       new BlockInfo(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
-                               replication);
+       new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
+           replication);
     }
     
     long currentBlockId = startingBlockId;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -106,7 +107,7 @@ public class TestCommitBlockSynchronization {
         lastBlock, genStamp, length, false, false, newTargets, null);
 
     // Simulate 'completing' the block.
-    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
+    BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)
@@ -178,7 +179,7 @@ public class TestCommitBlockSynchronization {
     namesystemSpy.commitBlockSynchronization(
         lastBlock, genStamp, length, true, false, newTargets, null);
 
-    BlockInfo completedBlockInfo = new BlockInfo(block, (short) 1);
+    BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     completedBlockInfo.setBlockCollection(file);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.IOUtils;
@@ -289,7 +290,7 @@ public class TestINodeFile {
       iNodes[i] = new INodeFile(i, null, perm, 0L, 0L, null, replication,
           preferredBlockSize, (byte)0);
       iNodes[i].setLocalName(DFSUtil.string2Bytes(fileNamePrefix + i));
-      BlockInfo newblock = new BlockInfo(replication);
+      BlockInfo newblock = new BlockInfoContiguous(replication);
       iNodes[i].addBlock(newblock);
     }
     

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -148,7 +149,7 @@ public class TestTruncateQuotaUpdate {
 
   private BlockInfo newBlock(long size, short replication) {
     Block b = new Block(++nextMockBlockId, size, ++nextMockGenstamp);
-    return new BlockInfo(b, replication);
+    return new BlockInfoContiguous(b, replication);
   }
 
   private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFileWithSnapshotFeature.java

@@ -21,6 +21,7 @@ import com.google.common.collect.Lists;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -50,7 +51,7 @@ public class TestFileWithSnapshotFeature {
     BlockStoragePolicySuite bsps = mock(BlockStoragePolicySuite.class);
     BlockStoragePolicy bsp = mock(BlockStoragePolicy.class);
     BlockInfo[] blocks = new BlockInfo[] {
-        new BlockInfo(new Block(1, BLOCK_SIZE, 1), REPL_1)
+        new BlockInfoContiguous(new Block(1, BLOCK_SIZE, 1), REPL_1)
     };
 
     // No snapshot