Browse Source

HDFS-508. Factor out BlockInfo from BlocksMap. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@798736 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 16 years ago
parent
commit
9c3e5bd441

+ 3 - 1
CHANGES.txt

@@ -48,10 +48,12 @@ Trunk (unreleased changes)
     HDFS-493. Change build.xml so that the fault-injected tests are executed
     HDFS-493. Change build.xml so that the fault-injected tests are executed
     only by the run-test-*-faul-inject targets.  (Konstantin Boudnik via
     only by the run-test-*-faul-inject targets.  (Konstantin Boudnik via
     szetszwo)
     szetszwo)
-   
+
     HADOOP-6160. Fix releaseaudit target to run on specific directories.
     HADOOP-6160. Fix releaseaudit target to run on specific directories.
     (gkesavan)
     (gkesavan)
 
 
+    HDFS-508. Factor out BlockInfo from BlocksMap. (shv)
+
   BUG FIXES
   BUG FIXES
     HDFS-76. Better error message to users when commands fail because of 
     HDFS-76. Better error message to users when commands fail because of 
     lack of quota. Allow quota to be set even if the limit is lower than
     lack of quota. Allow quota to be set even if the limit is lower than

+ 276 - 0
src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java

@@ -0,0 +1,276 @@
+/**
+ * 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.namenode;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/**
+ * Internal class for block metadata.
+ */
+class BlockInfo extends Block {
+  private INodeFile inode;
+
+  /**
+   * This array contains triplets of references.
+   * For each i-th datanode the block belongs to
+   * triplets[3*i] is the reference to the DatanodeDescriptor
+   * 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 data-node.
+   */
+  private Object[] triplets;
+
+  BlockInfo(Block blk, int replication) {
+    super(blk);
+    this.triplets = new Object[3*replication];
+    this.inode = null;
+  }
+
+  INodeFile getINode() {
+    return inode;
+  }
+
+  void setINode(INodeFile inode) {
+    this.inode = inode;
+  }
+
+  DatanodeDescriptor getDatanode(int index) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    DatanodeDescriptor node = (DatanodeDescriptor)triplets[index*3];
+    assert node == null || 
+        DatanodeDescriptor.class.getName().equals(node.getClass().getName()) : 
+              "DatanodeDescriptor is expected at " + index*3;
+    return node;
+  }
+
+  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];
+    assert info == null || 
+        BlockInfo.class.getName().equals(info.getClass().getName()) : 
+              "BlockInfo is expected at " + index*3;
+    return info;
+  }
+
+  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";
+    BlockInfo info = (BlockInfo)triplets[index*3+2];
+    assert info == null || 
+        BlockInfo.class.getName().equals(info.getClass().getName()) : 
+              "BlockInfo is expected at " + index*3;
+    return info;
+  }
+
+  void setDatanode(int index, DatanodeDescriptor node) {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
+    triplets[index*3] = node;
+  }
+
+  void 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";
+    triplets[index*3+1] = to;
+  }
+
+  void 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";
+    triplets[index*3+2] = to;
+  }
+
+  int getCapacity() {
+    assert this.triplets != null : "BlockInfo is not initialized";
+    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
+    return triplets.length / 3;
+  }
+
+  /**
+   * 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];
+    for(int i=0; i < last*3; i++) {
+      triplets[i] = old[i];
+    }
+    return last;
+  }
+
+  /**
+   * Count the number of data-nodes the block belongs to.
+   */
+  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;
+  }
+
+  /**
+   * Add data-node this block belongs to.
+   */
+  boolean addNode(DatanodeDescriptor node) {
+    if(findDatanode(node) >= 0) // the node is already there
+      return false;
+    // find the last null node
+    int lastNode = ensureCapacity(1);
+    setDatanode(lastNode, node);
+    setNext(lastNode, null);
+    setPrevious(lastNode, null);
+    return true;
+  }
+
+  /**
+   * Remove data-node from the block.
+   */
+  boolean removeNode(DatanodeDescriptor node) {
+    int dnIndex = findDatanode(node);
+    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 
+    setDatanode(dnIndex, getDatanode(lastNode));
+    setNext(dnIndex, getNext(lastNode)); 
+    setPrevious(dnIndex, getPrevious(lastNode)); 
+    // set the last triplet to null
+    setDatanode(lastNode, null);
+    setNext(lastNode, null); 
+    setPrevious(lastNode, null); 
+    return true;
+  }
+
+  /**
+   * Find specified DatanodeDescriptor.
+   * @param dn
+   * @return index or -1 if not found.
+   */
+  int findDatanode(DatanodeDescriptor dn) {
+    int len = getCapacity();
+    for(int idx = 0; idx < len; idx++) {
+      DatanodeDescriptor cur = getDatanode(idx);
+      if(cur == dn)
+        return idx;
+      if(cur == null)
+        break;
+    }
+    return -1;
+  }
+
+  /**
+   * Insert this block into the head of the list of blocks 
+   * related to the specified DatanodeDescriptor.
+   * If the head is null then form a new list.
+   * @return current block as the new head of the list.
+   */
+  BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
+    int dnIndex = this.findDatanode(dn);
+    assert dnIndex >= 0 : "Data node is not found: current";
+    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);
+    if(head != null)
+      head.setPrevious(head.findDatanode(dn), this);
+    return this;
+  }
+
+  /**
+   * Remove this block from the list of blocks 
+   * related to the specified DatanodeDescriptor.
+   * 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
+   * empty after deletion.
+   */
+  BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+    if(head == null)
+      return null;
+    int dnIndex = this.findDatanode(dn);
+    if(dnIndex < 0) // this block is not on the data-node list
+      return head;
+
+    BlockInfo next = this.getNext(dnIndex);
+    BlockInfo prev = this.getPrevious(dnIndex);
+    this.setNext(dnIndex, null);
+    this.setPrevious(dnIndex, null);
+    if(prev != null)
+      prev.setNext(prev.findDatanode(dn), next);
+    if(next != null)
+      next.setPrevious(next.findDatanode(dn), prev);
+    if(this == head)  // removing the head
+      head = next;
+    return head;
+  }
+
+  int listCount(DatanodeDescriptor dn) {
+    int count = 0;
+    for(BlockInfo cur = this; cur != null;
+          cur = cur.getNext(cur.findDatanode(dn)))
+      count++;
+    return count;
+  }
+
+  boolean listIsConsistent(DatanodeDescriptor dn) {
+    // going forward
+    int count = 0;
+    BlockInfo next, nextPrev;
+    BlockInfo cur = this;
+    while(cur != null) {
+      next = cur.getNext(cur.findDatanode(dn));
+      if(next != null) {
+        nextPrev = next.getPrevious(next.findDatanode(dn));
+        if(cur != nextPrev) {
+          System.out.println("Inconsistent list: cur->next->prev != cur");
+          return false;
+        }
+      }
+      cur = next;
+      count++;
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    // Super implementation is sufficient
+    return super.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+}

+ 3 - 4
src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
 import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.security.AccessTokenHandler;
 import org.apache.hadoop.security.AccessTokenHandler;
@@ -1051,7 +1050,7 @@ public class BlockManager {
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     synchronized (namesystem) {
     synchronized (namesystem) {
       neededReplications.clear();
       neededReplications.clear();
-      for (BlocksMap.BlockInfo block : blocksMap.getBlocks()) {
+      for (BlockInfo block : blocksMap.getBlocks()) {
         INodeFile fileINode = block.getINode();
         INodeFile fileINode = block.getINode();
         if (fileINode == null) {
         if (fileINode == null) {
           // block does not belong to any file
           // block does not belong to any file
@@ -1415,13 +1414,13 @@ public class BlockManager {
     blocksMap.removeBlock(block);
     blocksMap.removeBlock(block);
   }
   }
   
   
-  public int getCapacity() {
+  int getCapacity() {
     synchronized(namesystem) {
     synchronized(namesystem) {
       return blocksMap.getCapacity();
       return blocksMap.getCapacity();
     }
     }
   }
   }
   
   
-  public float getLoadFactor() {
+  float getLoadFactor() {
     return blocksMap.getLoadFactor();
     return blocksMap.getLoadFactor();
   }
   }
 }
 }

+ 12 - 281
src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -30,259 +30,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
  * the datanodes that store the block.
  * the datanodes that store the block.
  */
  */
 class BlocksMap {
 class BlocksMap {
-        
-  /**
-   * Internal class for block metadata.
-   */
-  static class BlockInfo extends Block {
-    private INodeFile inode;
-
-    /**
-     * This array contains triplets of references.
-     * For each i-th datanode the block belongs to
-     * triplets[3*i] is the reference to the DatanodeDescriptor
-     * 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 data-node.
-     */
-    private Object[] triplets;
-
-    BlockInfo(Block blk, int replication) {
-      super(blk);
-      this.triplets = new Object[3*replication];
-      this.inode = null;
-    }
-
-    INodeFile getINode() {
-      return inode;
-    }
-
-    DatanodeDescriptor getDatanode(int index) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-      DatanodeDescriptor node = (DatanodeDescriptor)triplets[index*3];
-      assert node == null || 
-          DatanodeDescriptor.class.getName().equals(node.getClass().getName()) : 
-                "DatanodeDescriptor is expected at " + index*3;
-      return node;
-    }
-
-    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];
-      assert info == null || 
-          BlockInfo.class.getName().equals(info.getClass().getName()) : 
-                "BlockInfo is expected at " + index*3;
-      return info;
-    }
-
-    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";
-      BlockInfo info = (BlockInfo)triplets[index*3+2];
-      assert info == null || 
-          BlockInfo.class.getName().equals(info.getClass().getName()) : 
-                "BlockInfo is expected at " + index*3;
-      return info;
-    }
-
-    void setDatanode(int index, DatanodeDescriptor node) {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
-      triplets[index*3] = node;
-    }
-
-    void 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";
-      triplets[index*3+1] = to;
-    }
-
-    void 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";
-      triplets[index*3+2] = to;
-    }
-
-    private int getCapacity() {
-      assert this.triplets != null : "BlockInfo is not initialized";
-      assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-      return triplets.length / 3;
-    }
-
-    /**
-     * 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];
-      for(int i=0; i < last*3; i++) {
-        triplets[i] = old[i];
-      }
-      return last;
-    }
-
-    /**
-     * Count the number of data-nodes the block belongs to.
-     */
-    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;
-    }
-
-    /**
-     * Add data-node this block belongs to.
-     */
-    boolean addNode(DatanodeDescriptor node) {
-      if(findDatanode(node) >= 0) // the node is already there
-        return false;
-      // find the last null node
-      int lastNode = ensureCapacity(1);
-      setDatanode(lastNode, node);
-      setNext(lastNode, null);
-      setPrevious(lastNode, null);
-      return true;
-    }
-
-    /**
-     * Remove data-node from the block.
-     */
-    boolean removeNode(DatanodeDescriptor node) {
-      int dnIndex = findDatanode(node);
-      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 
-      setDatanode(dnIndex, getDatanode(lastNode));
-      setNext(dnIndex, getNext(lastNode)); 
-      setPrevious(dnIndex, getPrevious(lastNode)); 
-      // set the last triplet to null
-      setDatanode(lastNode, null);
-      setNext(lastNode, null); 
-      setPrevious(lastNode, null); 
-      return true;
-    }
-
-    /**
-     * Find specified DatanodeDescriptor.
-     * @param dn
-     * @return index or -1 if not found.
-     */
-    int findDatanode(DatanodeDescriptor dn) {
-      int len = getCapacity();
-      for(int idx = 0; idx < len; idx++) {
-        DatanodeDescriptor cur = getDatanode(idx);
-        if(cur == dn)
-          return idx;
-        if(cur == null)
-          break;
-      }
-      return -1;
-    }
-
-    /**
-     * Insert this block into the head of the list of blocks 
-     * related to the specified DatanodeDescriptor.
-     * If the head is null then form a new list.
-     * @return current block as the new head of the list.
-     */
-    BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
-      int dnIndex = this.findDatanode(dn);
-      assert dnIndex >= 0 : "Data node is not found: current";
-      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);
-      if(head != null)
-        head.setPrevious(head.findDatanode(dn), this);
-      return this;
-    }
-
-    /**
-     * Remove this block from the list of blocks 
-     * related to the specified DatanodeDescriptor.
-     * 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
-     * empty after deletion.
-     */
-    BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
-      if(head == null)
-        return null;
-      int dnIndex = this.findDatanode(dn);
-      if(dnIndex < 0) // this block is not on the data-node list
-        return head;
-
-      BlockInfo next = this.getNext(dnIndex);
-      BlockInfo prev = this.getPrevious(dnIndex);
-      this.setNext(dnIndex, null);
-      this.setPrevious(dnIndex, null);
-      if(prev != null)
-        prev.setNext(prev.findDatanode(dn), next);
-      if(next != null)
-        next.setPrevious(next.findDatanode(dn), prev);
-      if(this == head)  // removing the head
-        head = next;
-      return head;
-    }
-
-    int listCount(DatanodeDescriptor dn) {
-      int count = 0;
-      for(BlockInfo cur = this; cur != null;
-            cur = cur.getNext(cur.findDatanode(dn)))
-        count++;
-      return count;
-    }
-
-    boolean listIsConsistent(DatanodeDescriptor dn) {
-      // going forward
-      int count = 0;
-      BlockInfo next, nextPrev;
-      BlockInfo cur = this;
-      while(cur != null) {
-        next = cur.getNext(cur.findDatanode(dn));
-        if(next != null) {
-          nextPrev = next.getPrevious(next.findDatanode(dn));
-          if(cur != nextPrev) {
-            System.out.println("Inconsistent list: cur->next->prev != cur");
-            return false;
-          }
-        }
-        cur = next;
-        count++;
-      }
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      // Super implementation is sufficient
-      return super.hashCode();
-    }
-    
-    @Override
-    public boolean equals(Object obj) {
-      // Sufficient to rely on super's implementation
-      return (this == obj) || super.equals(obj);
-    }
-  }
-
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
     private BlockInfo blockInfo;
     private BlockInfo blockInfo;
     private int nextIdx = 0;
     private int nextIdx = 0;
@@ -320,29 +67,22 @@ class BlocksMap {
     this.map = new HashMap<BlockInfo, BlockInfo>(initialCapacity, loadFactor);
     this.map = new HashMap<BlockInfo, BlockInfo>(initialCapacity, loadFactor);
   }
   }
 
 
-  /**
-   * Add BlockInfo if mapping does not exist.
-   */
-  private BlockInfo checkBlockInfo(Block b, int replication) {
-    BlockInfo info = map.get(b);
-    if (info == null) {
-      info = new BlockInfo(b, replication);
-      map.put(info, info);
-    }
-    return info;
-  }
-
   INodeFile getINode(Block b) {
   INodeFile getINode(Block b) {
     BlockInfo info = map.get(b);
     BlockInfo info = map.get(b);
-    return (info != null) ? info.inode : null;
+    return (info != null) ? info.getINode() : null;
   }
   }
 
 
   /**
   /**
    * Add block b belonging to the specified file inode to the map.
    * Add block b belonging to the specified file inode to the map.
    */
    */
   BlockInfo addINode(Block b, INodeFile iNode) {
   BlockInfo addINode(Block b, INodeFile iNode) {
-    BlockInfo info = checkBlockInfo(b, iNode.getReplication());
-    info.inode = iNode;
+    int replication = iNode.getReplication();
+    BlockInfo info = map.get(b);
+    if (info == null) {
+      info = new BlockInfo(b, replication);
+      map.put(info, info);
+    }
+    info.setINode(iNode);
     return info;
     return info;
   }
   }
 
 
@@ -356,7 +96,7 @@ class BlocksMap {
     if (blockInfo == null)
     if (blockInfo == null)
       return;
       return;
 
 
-    blockInfo.inode = null;
+    blockInfo.setINode(null);
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
@@ -379,15 +119,6 @@ class BlocksMap {
     return info == null ? 0 : info.numNodes();
     return info == null ? 0 : info.numNodes();
   }
   }
 
 
-  /** returns true if the node does not already exists and is added.
-   * false if the node already exists.*/
-  boolean addNode(Block b, DatanodeDescriptor node, int replication) {
-    // insert into the map if not there yet
-    BlockInfo info = checkBlockInfo(b, replication);
-    // add block to the data-node list and the node to the block info
-    return node.addBlock(info);
-  }
-
   /**
   /**
    * Remove data-node reference from the block.
    * Remove data-node reference from the block.
    * Remove the block from the block map
    * Remove the block from the block map
@@ -402,7 +133,7 @@ class BlocksMap {
     boolean removed = node.removeBlock(info);
     boolean removed = node.removeBlock(info);
 
 
     if (info.getDatanode(0) == null     // no datanodes left
     if (info.getDatanode(0) == null     // no datanodes left
-              && info.inode == null) {  // does not belong to a file
+              && info.getINode() == null) {  // does not belong to a file
       map.remove(b);  // remove block from the map
       map.remove(b);  // remove block from the map
     }
     }
     return removed;
     return removed;
@@ -437,7 +168,7 @@ class BlocksMap {
   }
   }
   
   
   /** Get the capacity of the HashMap that stores blocks */
   /** Get the capacity of the HashMap that stores blocks */
-  public int getCapacity() {
+  int getCapacity() {
     // Capacity doubles every time the map size reaches the threshold
     // Capacity doubles every time the map size reaches the threshold
     while (map.size() > (int)(capacity * loadFactor)) {
     while (map.size() > (int)(capacity * loadFactor)) {
       capacity <<= 1;
       capacity <<= 1;
@@ -446,7 +177,7 @@ class BlocksMap {
   }
   }
   
   
   /** Get the load factor of the map */
   /** Get the load factor of the map */
-  public float getLoadFactor() {
+  float getLoadFactor() {
     return loadFactor;
     return loadFactor;
   }
   }
 }
 }

+ 0 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;

+ 0 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 /*************************************************
 /*************************************************
  * FSDirectory stores the filesystem directory state.
  * FSDirectory stores the filesystem directory state.

+ 0 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -58,7 +58,6 @@ import org.apache.hadoop.hdfs.server.common.UpgradeManager;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
 import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;

+ 0 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMetrics;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;

+ 0 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 class INodeFile extends INode {
 class INodeFile extends INode {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);

+ 0 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -21,7 +21,6 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 
 
 class INodeFileUnderConstruction extends INodeFile {
 class INodeFileUnderConstruction extends INodeFile {

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

@@ -19,15 +19,12 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
 
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.BlocksMap.BlockInfo;
 
 
 /**
 /**
  * 
  *