Преглед изворни кода

HDFS-5323. Remove some deadcode in BlockManager (Colin Patrick McCabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1530814 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe пре 11 година
родитељ
комит
b8ae4e2a8e

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

@@ -306,6 +306,8 @@ Release 2.3.0 - UNRELEASED
     HDFS-4510. Cover classes ClusterJspHelper/NamenodeJspHelper with unit
     tests. (Andrey Klochkov via kihwal)
 
+    HDFS-5323.  Remove some deadcode in BlockManager (Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

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

@@ -75,6 +75,7 @@ import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -90,9 +91,6 @@ public class BlockManager {
   static final Log LOG = LogFactory.getLog(BlockManager.class);
   public static final Log blockLog = NameNode.blockStateChangeLog;
 
-  /** Default load factor of map */
-  public static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
-
   private static final String QUEUE_REASON_CORRUPT_STATE =
     "it has the wrong state or generation stamp";
 
@@ -244,7 +242,8 @@ public class BlockManager {
     invalidateBlocks = new InvalidateBlocks(datanodeManager);
 
     // Compute the map capacity by allocating 2% of total memory
-    blocksMap = new BlocksMap(DEFAULT_MAP_LOAD_FACTOR);
+    blocksMap = new BlocksMap(
+        LightWeightGSet.computeCapacity(2.0, "BlocksMap"));
     blockplacement = BlockPlacementPolicy.getInstance(
         conf, stats, datanodeManager.getNetworkTopology());
     pendingReplications = new PendingReplicationBlocks(conf.getInt(

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

@@ -57,11 +57,11 @@ class BlocksMap {
   /** Constant {@link LightWeightGSet} capacity. */
   private final int capacity;
   
-  private volatile GSet<Block, BlockInfo> blocks;
+  private GSet<Block, BlockInfo> blocks;
 
-  BlocksMap(final float loadFactor) {
+  BlocksMap(int capacity) {
     // Use 2% of total memory to size the GSet capacity
-    this.capacity = LightWeightGSet.computeCapacity(2.0, "BlocksMap");
+    this.capacity = capacity;
     this.blocks = new LightWeightGSet<Block, BlockInfo>(capacity);
   }
 

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -192,17 +192,10 @@ public class TestNameNodeMetrics {
     assertCounter("CreateFileOps", 1L, rb);
     assertCounter("FilesCreated", (long)file.depth(), rb);
 
-    // Blocks are stored in a hashmap. Compute its capacity, which
-    // doubles every time the number of entries reach the threshold.
-    int threshold = (int)(blockCapacity * BlockManager.DEFAULT_MAP_LOAD_FACTOR);
-    while (threshold < blockCount) {
-      blockCapacity <<= 1;
-    }
     long filesTotal = file.depth() + 1; // Add 1 for root
     rb = getMetrics(NS_METRICS);
     assertGauge("FilesTotal", filesTotal, rb);
     assertGauge("BlocksTotal", blockCount, rb);
-    assertGauge("BlockCapacity", blockCapacity, rb);
     fs.delete(file, true);
     filesTotal--; // reduce the filecount for deleted file