Browse Source

HDFS-5782. Change BlockListAsLongs constructor to take Replica as parameter type instead of concrete classes Block and ReplicaInfo. Contributed by David Powell and Joe Pallas

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Tsz-Wo Nicholas Sze 10 years ago
parent
commit
2ecfd018f4

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

@@ -284,6 +284,10 @@ Release 2.7.0 - UNRELEASED
     class and constructor to public; and fix FsDatasetSpi to use generic type
     class and constructor to public; and fix FsDatasetSpi to use generic type
     instead of FsVolumeImpl.  (David Powell and Joe Pallas via szetszwo)
     instead of FsVolumeImpl.  (David Powell and Joe Pallas via szetszwo)
 
 
+    HDFS-5782. Change BlockListAsLongs constructor to take Replica as parameter
+    type instead of concrete classes Block and ReplicaInfo.  (David Powell
+    and Joe Pallas via szetszwo)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

+ 49 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java

@@ -25,7 +25,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.Replica;
 
 
 /**
 /**
  * This class provides an interface for accessing list of blocks that
  * This class provides an interface for accessing list of blocks that
@@ -85,8 +85,8 @@ public class BlockListAsLongs implements Iterable<Block> {
    * @param finalized - list of finalized blocks
    * @param finalized - list of finalized blocks
    * @param uc - list of under construction blocks
    * @param uc - list of under construction blocks
    */
    */
-  public BlockListAsLongs(final List<? extends Block> finalized,
-                          final List<ReplicaInfo> uc) {
+  public BlockListAsLongs(final List<? extends Replica> finalized,
+                          final List<? extends Replica> uc) {
     int finalizedSize = finalized == null ? 0 : finalized.size();
     int finalizedSize = finalized == null ? 0 : finalized.size();
     int ucSize = uc == null ? 0 : uc.size();
     int ucSize = uc == null ? 0 : uc.size();
     int len = HEADER_SIZE
     int len = HEADER_SIZE
@@ -113,8 +113,34 @@ public class BlockListAsLongs implements Iterable<Block> {
     }
     }
   }
   }
 
 
+  /**
+   * Create block report from a list of finalized blocks.  Used by
+   * NNThroughputBenchmark.
+   *
+   * @param blocks - list of finalized blocks
+   */
+  public BlockListAsLongs(final List<? extends Block> blocks) {
+    int finalizedSize = blocks == null ? 0 : blocks.size();
+    int len = HEADER_SIZE
+              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK;
+
+    blockList = new long[len];
+
+    // set the header
+    blockList[0] = finalizedSize;
+    blockList[1] = 0;
+
+    // set finalized blocks
+    for (int i = 0; i < finalizedSize; i++) {
+      setBlock(i, blocks.get(i));
+    }
+
+    // set invalid delimiting block
+    setDelimitingBlock(finalizedSize);
+  }
+
   public BlockListAsLongs() {
   public BlockListAsLongs() {
-    this(null);
+    this((long[])null);
   }
   }
 
 
   /**
   /**
@@ -279,18 +305,30 @@ public class BlockListAsLongs implements Iterable<Block> {
   /**
   /**
    * Set the indexTh block
    * Set the indexTh block
    * @param index - the index of the block to set
    * @param index - the index of the block to set
-   * @param b - the block is set to the value of the this block
+   * @param r - the block is set to the value of the this Replica
    */
    */
-  private <T extends Block> void setBlock(final int index, final T b) {
+  private void setBlock(final int index, final Replica r) {
     int pos = index2BlockId(index);
     int pos = index2BlockId(index);
-    blockList[pos] = b.getBlockId();
-    blockList[pos + 1] = b.getNumBytes();
-    blockList[pos + 2] = b.getGenerationStamp();
+    blockList[pos] = r.getBlockId();
+    blockList[pos + 1] = r.getNumBytes();
+    blockList[pos + 2] = r.getGenerationStamp();
     if(index < getNumberOfFinalizedReplicas())
     if(index < getNumberOfFinalizedReplicas())
       return;
       return;
-    assert ((ReplicaInfo)b).getState() != ReplicaState.FINALIZED :
+    assert r.getState() != ReplicaState.FINALIZED :
       "Must be under-construction replica.";
       "Must be under-construction replica.";
-    blockList[pos + 3] = ((ReplicaInfo)b).getState().getValue();
+    blockList[pos + 3] = r.getState().getValue();
+  }
+
+  /**
+   * Set the indexTh block
+   * @param index - the index of the block to set
+   * @param b - the block is set to the value of the this Block
+   */
+  private void setBlock(final int index, final Block b) {
+    int pos = index2BlockId(index);
+    blockList[pos] = b.getBlockId();
+    blockList[pos + 1] = b.getNumBytes();
+    blockList[pos + 2] = b.getGenerationStamp();
   }
   }
 
 
   /**
   /**

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -563,12 +563,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
   }
 
 
   synchronized BlockListAsLongs getBlockReport(String bpid) {
   synchronized BlockListAsLongs getBlockReport(String bpid) {
-    final List<Block> blocks = new ArrayList<Block>();
+    final List<Replica> blocks = new ArrayList<Replica>();
     final Map<Block, BInfo> map = blockMap.get(bpid);
     final Map<Block, BInfo> map = blockMap.get(bpid);
     if (map != null) {
     if (map != null) {
       for (BInfo b : map.values()) {
       for (BInfo b : map.values()) {
         if (b.isFinalized()) {
         if (b.isFinalized()) {
-          blocks.add(b.theBlock);
+          blocks.add(b);
         }
         }
       }
       }
     }
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java

@@ -114,7 +114,7 @@ public class TestBlockHasMultipleReplicasOnSameDN {
     }
     }
 
 
     for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
     for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
-      BlockListAsLongs bll = new BlockListAsLongs(blocks, null);
+      BlockListAsLongs bll = new BlockListAsLongs(blocks);
       FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
       FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
       DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
       DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
       reports[i] = new StorageBlockReport(dns, bll.getBlockListAsLongs());
       reports[i] = new StorageBlockReport(dns, bll.getBlockListAsLongs());

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

@@ -977,7 +977,7 @@ public class NNThroughputBenchmark implements Tool {
       // fill remaining slots with blocks that do not exist
       // fill remaining slots with blocks that do not exist
       for(int idx = blocks.size()-1; idx >= nrBlocks; idx--)
       for(int idx = blocks.size()-1; idx >= nrBlocks; idx--)
         blocks.set(idx, new Block(blocks.size() - idx, 0, 0));
         blocks.set(idx, new Block(blocks.size() - idx, 0, 0));
-      blockReportList = new BlockListAsLongs(blocks,null).getBlockListAsLongs();
+      blockReportList = new BlockListAsLongs(blocks).getBlockListAsLongs();
     }
     }
 
 
     long[] getBlockReportList() {
     long[] getBlockReportList() {