You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:22:04 UTC

svn commit: r1077509 - /hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

Author: omalley
Date: Fri Mar  4 04:22:04 2011
New Revision: 1077509

URL: http://svn.apache.org/viewvc?rev=1077509&view=rev
Log:
commit 03f9108b2912008a59d6a39f17f3fb7c55c669d4
Author: Tsz Wo Wo Sze <ts...@ucdev29.inktomisearch.com>
Date:   Wed Jun 23 16:34:06 2010 +0000

    Missed two files for the previous commit.
    
    +++ b/YAHOO-CHANGES.txt
    +    HDFS-1119. Introduce a GSet interface to BlocksMap.  (szetszwo)
    +

Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java?rev=1077509&r1=1077508&r2=1077509&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java Fri Mar  4 04:22:04 2011
@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.*;
+import java.util.Iterator;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.util.GSet;
+import org.apache.hadoop.hdfs.util.GSetByHashMap;
 
 /**
  * This class maintains the map from a block to its metadata.
@@ -294,7 +296,7 @@ class BlocksMap {
   private int capacity;
   private final float loadFactor;
   
-  private Map<BlockInfo, BlockInfo> map;
+  private GSet<Block, BlockInfo> blocks;
 
   BlocksMap(int initialCapacity, float loadFactor) {
     this.capacity = 1;
@@ -302,23 +304,24 @@ class BlocksMap {
     while (this.capacity < initialCapacity)
       this.capacity <<= 1;
     this.loadFactor = loadFactor;
-    this.map = new HashMap<BlockInfo, BlockInfo>(initialCapacity, loadFactor);
+    this.blocks = new GSetByHashMap<Block, BlockInfo>(
+        initialCapacity, loadFactor);
   }
 
   /**
    * Add BlockInfo if mapping does not exist.
    */
   private BlockInfo checkBlockInfo(Block b, int replication) {
-    BlockInfo info = map.get(b);
+    BlockInfo info = blocks.get(b);
     if (info == null) {
       info = new BlockInfo(b, replication);
-      map.put(info, info);
+      blocks.put(info);
     }
     return info;
   }
 
   INodeFile getINode(Block b) {
-    BlockInfo info = map.get(b);
+    BlockInfo info = blocks.get(b);
     return (info != null) ? info.inode : null;
   }
 
@@ -337,11 +340,11 @@ class BlocksMap {
    * then remove the block from the block map.
    */
   void removeINode(Block b) {
-    BlockInfo info = map.get(b);
+    BlockInfo info = blocks.get(b);
     if (info != null) {
       info.inode = null;
       if (info.getDatanode(0) == null) {  // no datanodes left
-        map.remove(b);  // remove block from the map
+        blocks.remove(b);  // remove block from the map
       }
     }
   }
@@ -359,22 +362,22 @@ class BlocksMap {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
     }
-    map.remove(blockInfo);  // remove block from the map
+    blocks.remove(blockInfo);  // remove block from the map
   }
 
   /** Returns the block object it it exists in the map. */
   BlockInfo getStoredBlock(Block b) {
-    return map.get(b);
+    return blocks.get(b);
   }
 
   /** Returned Iterator does not support. */
   Iterator<DatanodeDescriptor> nodeIterator(Block b) {
-    return new NodeIterator(map.get(b));
+    return new NodeIterator(blocks.get(b));
   }
 
   /** counts number of containing nodes. Better than using iterator. */
   int numNodes(Block b) {
-    BlockInfo info = map.get(b);
+    BlockInfo info = blocks.get(b);
     return info == null ? 0 : info.numNodes();
   }
 
@@ -393,7 +396,7 @@ class BlocksMap {
    * only if it does not belong to any file and data-nodes.
    */
   boolean removeNode(Block b, DatanodeDescriptor node) {
-    BlockInfo info = map.get(b);
+    BlockInfo info = blocks.get(b);
     if (info == null)
       return false;
 
@@ -402,30 +405,30 @@ class BlocksMap {
 
     if (info.getDatanode(0) == null     // no datanodes left
               && info.inode == null) {  // does not belong to a file
-      map.remove(b);  // remove block from the map
+      blocks.remove(b);  // remove block from the map
     }
     return removed;
   }
 
   int size() {
-    return map.size();
+    return blocks.size();
   }
 
-  Collection<BlockInfo> getBlocks() {
-    return map.values();
+  Iterable<BlockInfo> getBlocks() {
+    return blocks;
   }
   /**
    * Check if the block exists in map
    */
   boolean contains(Block block) {
-    return map.containsKey(block);
+    return blocks.contains(block);
   }
   
   /**
    * Check if the replica at the given datanode exists in map
    */
   boolean contains(Block block, DatanodeDescriptor datanode) {
-    BlockInfo info = map.get(block);
+    BlockInfo info = blocks.get(block);
     if (info == null)
       return false;
     
@@ -438,14 +441,9 @@ class BlocksMap {
   /** Get the capacity of the HashMap that stores blocks */
   public int getCapacity() {
     // Capacity doubles every time the map size reaches the threshold
-    while (map.size() > (int)(capacity * loadFactor)) {
+    while (blocks.size() > (int)(capacity * loadFactor)) {
       capacity <<= 1;
     }
     return capacity;
   }
-  
-  /** Get the load factor of the map */
-  public float getLoadFactor() {
-    return loadFactor;
-  }
 }