Browse Source

Revert 1140913 and 1140909 for HDFS-2107.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1140920 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 14 years ago
parent
commit
97b6ca4dd7
53 changed files with 350 additions and 489 deletions
  1. 0 3
      hdfs/CHANGES.txt
  2. 2 2
      hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  3. 0 57
      hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java
  4. 3 2
      hdfs/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  5. 13 14
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java
  6. 10 17
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java
  7. 73 103
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
  8. 4 6
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicy.java
  9. 2 5
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicyDefault.java
  10. 5 6
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java
  11. 1 2
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java
  12. 34 38
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
  13. 0 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/DecommissionManager.java
  14. 11 18
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  15. 2 24
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  16. 0 2
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  17. 0 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  18. 1 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  19. 68 34
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  20. 1 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java
  21. 4 6
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  22. 5 9
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  23. 5 10
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  24. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  25. 1 1
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  26. 3 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  27. 4 4
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  28. 1 3
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java
  29. 5 6
      hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnderReplicatedBlocks.java
  30. 3 8
      hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
  31. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
  32. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
  33. 11 12
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
  34. 0 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  35. 1 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
  36. 0 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  37. 2 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java
  38. 6 8
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCorruptReplicaInfo.java
  39. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDatanodeDescriptor.java
  40. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
  41. 9 7
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
  42. 1 2
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java
  43. 2 3
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java
  44. 6 6
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
  45. 3 4
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java
  46. 4 5
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
  47. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestPendingReplication.java
  48. 9 9
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java
  49. 3 4
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java
  50. 8 9
      hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  51. 1 1
      hdfs/src/test/hdfs/org/apache/hadoop/net/TestNetworkTopology.java
  52. 7 13
      hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java
  53. 10 0
      hdfs/src/webapps/hdfs/block_info_xml.jsp

+ 0 - 3
hdfs/CHANGES.txt

@@ -534,9 +534,6 @@ Trunk (unreleased changes)
 
     HDFS-2110. StreamFile and ByteRangeInputStream cleanup. (eli)
 
-    HDFS-2107. Move block management code from o.a.h.h.s.namenode to a new
-    package o.a.h.h.s.blockmanagement.  (szetszwo)
-
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

+ 2 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -61,10 +61,10 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
+import org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicy;
+import org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.io.IOUtils;

+ 0 - 57
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/NumberReplicas.java

@@ -1,57 +0,0 @@
-/**
- * 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;
-
-/**
- * A immutable object that stores the number of live replicas and
- * the number of decommissined Replicas.
- */
-public class NumberReplicas {
-  private int liveReplicas;
-  private int decommissionedReplicas;
-  private int corruptReplicas;
-  private int excessReplicas;
-
-  NumberReplicas() {
-    initialize(0, 0, 0, 0);
-  }
-
-  NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
-    initialize(live, decommissioned, corrupt, excess);
-  }
-
-  void initialize(int live, int decommissioned, int corrupt, int excess) {
-    liveReplicas = live;
-    decommissionedReplicas = decommissioned;
-    corruptReplicas = corrupt;
-    excessReplicas = excess;
-  }
-
-  public int liveReplicas() {
-    return liveReplicas;
-  }
-  public int decommissionedReplicas() {
-    return decommissionedReplicas;
-  }
-  public int corruptReplicas() {
-    return corruptReplicas;
-  }
-  public int excessReplicas() {
-    return excessReplicas;
-  }
-} 

+ 3 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -26,8 +26,8 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.URL;
 import java.net.URLEncoder;
-import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -45,13 +45,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HtmlQuoting;
 import org.apache.hadoop.io.Text;

+ 13 - 14
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfo.java

@@ -15,17 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 /**
  * Internal class for block metadata.
  */
-public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
+class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   private INodeFile inode;
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
@@ -45,12 +44,12 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * Construct an entry for blocksmap
    * @param replication the block's replication factor
    */
-  public BlockInfo(int replication) {
+  protected BlockInfo(int replication) {
     this.triplets = new Object[3*replication];
     this.inode = null;
   }
   
-  public BlockInfo(Block blk, int replication) {
+  protected BlockInfo(Block blk, int replication) {
     super(blk);
     this.triplets = new Object[3*replication];
     this.inode = null;
@@ -66,11 +65,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.inode = from.inode;
   }
 
-  public INodeFile getINode() {
+  INodeFile getINode() {
     return inode;
   }
 
-  public void setINode(INodeFile inode) {
+  void setINode(INodeFile inode) {
     this.inode = inode;
   }
 
@@ -163,7 +162,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   /**
    * Add data-node this block belongs to.
    */
-  public boolean addNode(DatanodeDescriptor node) {
+  boolean addNode(DatanodeDescriptor node) {
     if(findDatanode(node) >= 0) // the node is already there
       return false;
     // find the last null node
@@ -177,7 +176,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   /**
    * Remove data-node from the block.
    */
-  public boolean removeNode(DatanodeDescriptor node) {
+  boolean removeNode(DatanodeDescriptor node) {
     int dnIndex = findDatanode(node);
     if(dnIndex < 0) // the node is not found
       return false;
@@ -219,7 +218,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * If the head is null then form a new list.
    * @return current block as the new head of the list.
    */
-  public BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
+  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 : 
@@ -239,7 +238,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * @return the new head of the list or null if the list becomes
    * empty after deletion.
    */
-  public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
+  BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
     if(head == null)
       return null;
     int dnIndex = this.findDatanode(dn);
@@ -285,7 +284,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * to {@link BlockInfoUnderConstruction}.
    * @return {@link BlockUCState#COMPLETE}
    */
-  public BlockUCState getBlockUCState() {
+  BlockUCState getBlockUCState() {
     return BlockUCState.COMPLETE;
   }
 
@@ -294,7 +293,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * 
    * @return true if the state of the block is {@link BlockUCState#COMPLETE}
    */
-  public boolean isComplete() {
+  boolean isComplete() {
     return getBlockUCState().equals(BlockUCState.COMPLETE);
   }
 
@@ -303,7 +302,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
    * 
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
-  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
+  BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeDescriptor[] targets) {
     if(isComplete()) {
       return new BlockInfoUnderConstruction(

+ 10 - 17
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockInfoUnderConstruction.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -24,13 +24,12 @@ import java.util.List;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
-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 BlockInfoUnderConstruction extends BlockInfo {
+class BlockInfoUnderConstruction extends BlockInfo {
   /** Block state. See {@link BlockUCState} */
   private BlockUCState blockUCState;
 
@@ -129,14 +128,11 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * Create block and set its state to
    * {@link BlockUCState#UNDER_CONSTRUCTION}.
    */
-  public BlockInfoUnderConstruction(Block blk, int replication) {
+  BlockInfoUnderConstruction(Block blk, int replication) {
     this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
   }
 
-  /**
-   * Create a block that is currently being constructed.
-   */
-  public BlockInfoUnderConstruction(Block blk, int replication,
+  BlockInfoUnderConstruction(Block blk, int replication,
                              BlockUCState state,
                              DatanodeDescriptor[] targets) {
     super(blk, replication);
@@ -164,8 +160,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     return new BlockInfo(this);
   }
 
-  /** Set expected locations */
-  public void setExpectedLocations(DatanodeDescriptor[] targets) {
+  void setExpectedLocations(DatanodeDescriptor[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
     for(int i = 0; i < numLocations; i++)
@@ -177,7 +172,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    */
-  public DatanodeDescriptor[] getExpectedLocations() {
+  DatanodeDescriptor[] getExpectedLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
     for(int i = 0; i < numLocations; i++)
@@ -185,8 +180,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     return locations;
   }
 
-  /** Get the number of expected locations */
-  public int getNumExpectedLocations() {
+  int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.size();
   }
 
@@ -195,7 +189,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * @see BlockUCState
    */
   @Override // BlockInfo
-  public BlockUCState getBlockUCState() {
+  BlockUCState getBlockUCState() {
     return blockUCState;
   }
 
@@ -203,8 +197,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     blockUCState = s;
   }
 
-  /** Get block recovery ID */
-  public long getBlockRecoveryId() {
+  long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
@@ -227,7 +220,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
    * Find the first alive data-node starting from the previous primary and
    * make it primary.
    */
-  public void initializeBlockRecovery(long recoveryId) {
+  void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {

+ 73 - 103
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -39,14 +39,10 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
-import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
+import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks;
 
 /**
  * Keeps information related to the blocks stored in the Hadoop cluster.
@@ -61,43 +57,18 @@ public class BlockManager {
 
   private final FSNamesystem namesystem;
 
-  private volatile long pendingReplicationBlocksCount = 0L;
-  private volatile long corruptReplicaBlocksCount = 0L;
-  private volatile long underReplicatedBlocksCount = 0L;
-  public volatile long scheduledReplicationBlocksCount = 0L;
-  private volatile long excessBlocksCount = 0L;
-  private volatile long pendingDeletionBlocksCount = 0L;
+  volatile long pendingReplicationBlocksCount = 0L;
+  volatile long corruptReplicaBlocksCount = 0L;
+  volatile long underReplicatedBlocksCount = 0L;
+  volatile long scheduledReplicationBlocksCount = 0L;
+  volatile long excessBlocksCount = 0L;
+  volatile long pendingDeletionBlocksCount = 0L;
 
-  /** Used by metrics */
-  public long getPendingReplicationBlocksCount() {
-    return pendingReplicationBlocksCount;
-  }
-  /** Used by metrics */
-  public long getUnderReplicatedBlocksCount() {
-    return underReplicatedBlocksCount;
-  }
-  /** Used by metrics */
-  public long getCorruptReplicaBlocksCount() {
-    return corruptReplicaBlocksCount;
-  }
-  /** Used by metrics */
-  public long getScheduledReplicationBlocksCount() {
-    return scheduledReplicationBlocksCount;
-  }
-  /** Used by metrics */
-  public long getPendingDeletionBlocksCount() {
-    return pendingDeletionBlocksCount;
-  }
-  /** Used by metrics */
-  public long getExcessBlocksCount() {
-    return excessBlocksCount;
-  }
-
-  /**
-   * Mapping: Block -> { INode, datanodes, self ref }
-   * Updated only in response to client-sent information.
-   */
-  public final BlocksMap blocksMap;
+  //
+  // Mapping: Block -> { INode, datanodes, self ref }
+  // Updated only in response to client-sent information.
+  //
+  final BlocksMap blocksMap;
 
   //
   // Store blocks-->datanodedescriptor(s) map of corrupt replicas
@@ -119,24 +90,24 @@ public class BlockManager {
   // eventually remove these extras.
   // Mapping: StorageID -> TreeSet<Block>
   //
-  public final Map<String, Collection<Block>> excessReplicateMap =
+  Map<String, Collection<Block>> excessReplicateMap =
     new TreeMap<String, Collection<Block>>();
 
   //
   // Store set of Blocks that need to be replicated 1 or more times.
   // We also store pending replication-orders.
   //
-  public UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
+  UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
   private PendingReplicationBlocks pendingReplications;
 
   //  The maximum number of replicas allowed for a block
-  public int maxReplication;
+  int maxReplication;
   //  How many outgoing replication streams a given node should have at one time
-  public int maxReplicationStreams;
+  int maxReplicationStreams;
   // Minimum copies needed or else write is disallowed
-  public int minReplication;
+  int minReplication;
   // Default number of replicas
-  public int defaultReplication;
+  int defaultReplication;
   // How many entries are returned by getCorruptInodes()
   int maxCorruptFilesReturned;
   
@@ -150,9 +121,9 @@ public class BlockManager {
   Random r = new Random();
 
   // for block replicas placement
-  public BlockPlacementPolicy replicator;
+  BlockPlacementPolicy replicator;
 
-  public BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
+  BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
     this(fsn, conf, DEFAULT_INITIAL_MAP_CAPACITY);
   }
   
@@ -207,16 +178,16 @@ public class BlockManager {
     FSNamesystem.LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
   }
 
-  public void activate() {
+  void activate() {
     pendingReplications.start();
   }
 
-  public void close() {
+  void close() {
     if (pendingReplications != null) pendingReplications.stop();
     blocksMap.close();
   }
 
-  public void metaSave(PrintWriter out) {
+  void metaSave(PrintWriter out) {
     //
     // Dump contents of neededReplication
     //
@@ -278,7 +249,7 @@ public class BlockManager {
    * @param block
    * @return true if the block has minimum replicas
    */
-  public boolean checkMinReplication(Block block) {
+  boolean checkMinReplication(Block block) {
     return (countNodes(block).liveReplicas() >= minReplication);
   }
 
@@ -326,7 +297,7 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  public void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
+  void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
       Block commitBlock) throws IOException {
     
     if(commitBlock == null)
@@ -391,7 +362,7 @@ public class BlockManager {
    * @param fileINode file
    * @return the last block locations if the block is partial or null otherwise
    */
-  public LocatedBlock convertLastBlockToUnderConstruction(
+  LocatedBlock convertLastBlockToUnderConstruction(
       INodeFileUnderConstruction fileINode) throws IOException {
     BlockInfo oldBlock = fileINode.getLastBlock();
     if(oldBlock == null ||
@@ -422,7 +393,7 @@ public class BlockManager {
   /**
    * Get all valid locations of the block
    */
-  public ArrayList<String> getValidLocations(Block block) {
+  ArrayList<String> getValidLocations(Block block) {
     ArrayList<String> machineSet =
       new ArrayList<String>(blocksMap.numNodes(block));
     for(Iterator<DatanodeDescriptor> it =
@@ -436,7 +407,7 @@ public class BlockManager {
     return machineSet;
   }
 
-  public List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
+  List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
       long length, int nrBlocksToReturn) throws IOException {
     int curBlk = 0;
     long curPos = 0, blkSize = 0;
@@ -465,15 +436,11 @@ public class BlockManager {
     return results;
   }
 
-  /** @return a LocatedBlock for the given block */
-  public LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
+  /** @param needBlockToken 
+   * @return a LocatedBlock for the given block */
+  LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
       ) throws IOException {
-    if (blk instanceof BlockInfoUnderConstruction) {
-      if (blk.isComplete()) {
-        throw new IOException(
-            "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
-            + ", blk=" + blk);
-      }
+    if (!blk.isComplete()) {
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
       final DatanodeDescriptor[] locations = uc.getExpectedLocations();
       return namesystem.createLocatedBlock(uc, locations, pos, false);
@@ -509,7 +476,7 @@ public class BlockManager {
    * Check whether the replication parameter is within the range
    * determined by system configuration.
    */
-   public void verifyReplication(String src,
+   void verifyReplication(String src,
                           short replication,
                           String clientName) throws IOException {
 
@@ -577,7 +544,7 @@ public class BlockManager {
    * @param b block
    * @param dn datanode
    */
-  public void addToInvalidates(Block b, DatanodeInfo dn) {
+  void addToInvalidates(Block b, DatanodeInfo dn) {
     addToInvalidates(b, dn, true);
   }
 
@@ -618,7 +585,7 @@ public class BlockManager {
     }
   }
 
-  public void findAndMarkBlockAsCorrupt(Block blk,
+  void findAndMarkBlockAsCorrupt(Block blk,
                                  DatanodeInfo dn) throws IOException {
     BlockInfo storedBlock = getStoredBlock(blk);
     if (storedBlock == null) {
@@ -701,14 +668,14 @@ public class BlockManager {
     }
   }
 
-  public void updateState() {
+  void updateState() {
     pendingReplicationBlocksCount = pendingReplications.size();
     underReplicatedBlocksCount = neededReplications.size();
     corruptReplicaBlocksCount = corruptReplicas.size();
   }
 
   /** Return number of under-replicated but not missing blocks */
-  public int getUnderReplicatedNotMissingBlocks() {
+  int getUnderReplicatedNotMissingBlocks() {
     return neededReplications.getUnderReplicatedBlockCount();
   }
   
@@ -717,7 +684,7 @@ public class BlockManager {
    * @param nodesToProcess number of datanodes to schedule deletion work
    * @return total number of block for deletion
    */
-  public int computeInvalidateWork(int nodesToProcess) {
+  int computeInvalidateWork(int nodesToProcess) {
     int numOfNodes = recentInvalidateSets.size();
     nodesToProcess = Math.min(numOfNodes, nodesToProcess);
 
@@ -757,7 +724,7 @@ public class BlockManager {
    *
    * @return number of blocks scheduled for replication during this iteration.
    */
-  public int computeReplicationWork(int blocksToProcess) throws IOException {
+  int computeReplicationWork(int blocksToProcess) throws IOException {
     // Choose the blocks to be replicated
     List<List<Block>> blocksToReplicate =
       chooseUnderReplicatedBlocks(blocksToProcess);
@@ -1064,7 +1031,7 @@ public class BlockManager {
    * If there were any replication requests that timed out, reap them
    * and put them back into the neededReplication queue
    */
-  public void processPendingReplications() {
+  void processPendingReplications() {
     Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();
@@ -1497,7 +1464,7 @@ public class BlockManager {
     short fileReplication = fileINode.getReplication();
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
-          num.decommissionedReplicas(), fileReplication);
+          num.decommissionedReplicas, fileReplication);
     } else {
       updateNeededReplications(storedBlock, curReplicaDelta, 0);
     }
@@ -1558,7 +1525,7 @@ public class BlockManager {
    * For each block in the name-node verify whether it belongs to any file,
    * over or under replicated. Place it into the respective queue.
    */
-  public void processMisReplicatedBlocks() {
+  void processMisReplicatedBlocks() {
     long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
     namesystem.writeLock();
     try {
@@ -1603,7 +1570,7 @@ public class BlockManager {
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  public void processOverReplicatedBlock(Block block, short replication,
+  void processOverReplicatedBlock(Block block, short replication,
       DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
@@ -1630,7 +1597,7 @@ public class BlockManager {
         addedNode, delNodeHint, replicator);
   }
 
-  public void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  void addToExcessReplicate(DatanodeInfo dn, Block block) {
     assert namesystem.hasWriteLock();
     Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
     if (excessBlocks == null) {
@@ -1651,7 +1618,7 @@ public class BlockManager {
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    */
-  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
+  void removeStoredBlock(Block block, DatanodeDescriptor node) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
           + block + " from " + node.getName());
@@ -1706,7 +1673,7 @@ public class BlockManager {
   /**
    * The given node is reporting that it received a certain block.
    */
-  public void addBlock(DatanodeDescriptor node, Block block, String delHint)
+  void addBlock(DatanodeDescriptor node, Block block, String delHint)
       throws IOException {
     // decrement number of blocks scheduled to this datanode.
     node.decBlocksScheduled();
@@ -1759,7 +1726,7 @@ public class BlockManager {
   /**
    * Return the number of nodes that are live and decommissioned.
    */
-  public NumberReplicas countNodes(Block b) {
+  NumberReplicas countNodes(Block b) {
     int count = 0;
     int live = 0;
     int corrupt = 0;
@@ -1838,7 +1805,7 @@ public class BlockManager {
    * Return true if there are any blocks on this node that have not
    * yet reached their replication factor. Otherwise returns false.
    */
-  public boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
+  boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
     boolean status = false;
     int underReplicatedBlocks = 0;
     int decommissionOnlyReplicas = 0;
@@ -1888,11 +1855,11 @@ public class BlockManager {
     return status;
   }
 
-  public int getActiveBlockCount() {
+  int getActiveBlockCount() {
     return blocksMap.size() - (int)pendingDeletionBlocksCount;
   }
 
-  public DatanodeDescriptor[] getNodes(BlockInfo block) {
+  DatanodeDescriptor[] getNodes(BlockInfo block) {
     DatanodeDescriptor[] nodes =
       new DatanodeDescriptor[block.numNodes()];
     Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
@@ -1902,22 +1869,22 @@ public class BlockManager {
     return nodes;
   }
 
-  public int getTotalBlocks() {
+  int getTotalBlocks() {
     return blocksMap.size();
   }
 
-  public void removeBlock(Block block) {
+  void removeBlock(Block block) {
     addToInvalidates(block);
     corruptReplicas.removeFromCorruptReplicasMap(block);
     blocksMap.removeBlock(block);
   }
 
-  public BlockInfo getStoredBlock(Block block) {
+  BlockInfo getStoredBlock(Block block) {
     return blocksMap.getStoredBlock(block);
   }
 
   /* updates a block in under replication queue */
-  public void updateNeededReplications(Block block, int curReplicasDelta,
+  void updateNeededReplications(Block block, int curReplicasDelta,
       int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -1938,13 +1905,13 @@ public class BlockManager {
     }
   }
 
-  public void checkReplication(Block block, int numExpectedReplicas) {
+  void checkReplication(Block block, int numExpectedReplicas) {
     // filter out containingNodes that are marked for decommission.
     NumberReplicas number = countNodes(block);
     if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) { 
       neededReplications.add(block,
                              number.liveReplicas(),
-                             number.decommissionedReplicas(),
+                             number.decommissionedReplicas,
                              numExpectedReplicas);
     }
   }
@@ -1959,8 +1926,11 @@ public class BlockManager {
     return fileINode.getReplication();
   }
 
-  /** Remove a datanode from the invalidatesSet */
-  public void removeFromInvalidates(String storageID) {
+  /**
+   * Remove a datanode from the invalidatesSet
+   * @param n datanode
+   */
+  void removeFromInvalidates(String storageID) {
     Collection<Block> blocks = recentInvalidateSets.remove(storageID);
     if (blocks != null) {
       pendingDeletionBlocksCount -= blocks.size();
@@ -2028,7 +1998,7 @@ public class BlockManager {
   //Returns the number of racks over which a given block is replicated
   //decommissioning/decommissioned nodes are not counted. corrupt replicas 
   //are also ignored
-  public int getNumberOfRacks(Block b) {
+  int getNumberOfRacks(Block b) {
     HashSet<String> rackSet = new HashSet<String>(0);
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(b);
@@ -2086,32 +2056,32 @@ public class BlockManager {
     }
   }
   
-  public long getMissingBlocksCount() {
+  long getMissingBlocksCount() {
     // not locking
     return this.neededReplications.getCorruptBlockSize();
   }
 
-  public BlockInfo addINode(BlockInfo block, INodeFile iNode) {
+  BlockInfo addINode(BlockInfo block, INodeFile iNode) {
     return blocksMap.addINode(block, iNode);
   }
 
-  public INodeFile getINode(Block b) {
+  INodeFile getINode(Block b) {
     return blocksMap.getINode(b);
   }
 
-  public void removeFromCorruptReplicasMap(Block block) {
+  void removeFromCorruptReplicasMap(Block block) {
     corruptReplicas.removeFromCorruptReplicasMap(block);
   }
 
-  public int numCorruptReplicas(Block block) {
+  int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
 
-  public void removeBlockFromMap(Block block) {
+  void removeBlockFromMap(Block block) {
     blocksMap.removeBlock(block);
   }
 
-  public int getCapacity() {
+  int getCapacity() {
     namesystem.readLock();
     try {
       return blocksMap.getCapacity();
@@ -2134,7 +2104,7 @@ public class BlockManager {
    * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
    *
    */
-  public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
+  long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
                                    Long startingBlockId) {
     return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
                                                      startingBlockId);
@@ -2143,7 +2113,7 @@ public class BlockManager {
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    */
-  public BlockIterator getCorruptReplicaBlockIterator() {
+  UnderReplicatedBlocks.BlockIterator getCorruptReplicaBlockIterator() {
     return neededReplications
         .iterator(UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }

+ 4 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicy.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -26,8 +26,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
-import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -96,7 +94,7 @@ public abstract class BlockPlacementPolicy {
    * @return array of DatanodeDescriptor instances chosen as target
    * and sorted as a pipeline.
    */
-  public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
+  abstract DatanodeDescriptor[] chooseTarget(String srcPath,
                                              int numOfReplicas,
                                              DatanodeDescriptor writer,
                                              List<DatanodeDescriptor> chosenNodes,
@@ -224,11 +222,11 @@ public abstract class BlockPlacementPolicy {
    * @param numOfReplicas number of replicas wanted.
    * @param writer the writer's machine, null if not in the cluster.
    * @param blocksize size of the data to be written.
-   * @param excludedNodes datanodes that should not be considered as targets.
+   * @param excludedNodes: datanodes that should not be considered as targets.
    * @return array of DatanodeDescriptor instances chosen as targets
    * and sorted as a pipeline.
    */
-  public DatanodeDescriptor[] chooseTarget(String srcPath,
+  DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     HashMap<Node, Node> excludedNodes,

+ 2 - 5
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlockPlacementPolicyDefault.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -32,9 +32,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
-import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
@@ -92,7 +89,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
   /** {@inheritDoc} */
   @Override
-  public DatanodeDescriptor[] chooseTarget(String srcPath,
+  DatanodeDescriptor[] chooseTarget(String srcPath,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,

+ 5 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BlocksMap.java

@@ -15,12 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Iterator;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.GSet;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
@@ -29,7 +28,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
  * block's metadata currently includes INode it belongs to and
  * the datanodes that store the block.
  */
-public class BlocksMap {
+class BlocksMap {
   private static class NodeIterator implements Iterator<DatanodeDescriptor> {
     private BlockInfo blockInfo;
     private int nextIdx = 0;
@@ -101,7 +100,7 @@ public class BlocksMap {
   /**
    * Add block b belonging to the specified file inode to the map.
    */
-  public BlockInfo addINode(BlockInfo b, INodeFile iNode) {
+  BlockInfo addINode(BlockInfo b, INodeFile iNode) {
     BlockInfo info = blocks.get(b);
     if (info != b) {
       info = b;
@@ -137,7 +136,7 @@ public class BlocksMap {
    * Searches for the block in the BlocksMap and 
    * returns Iterator that iterates through the nodes the block belongs to.
    */
-  public Iterator<DatanodeDescriptor> nodeIterator(Block b) {
+  Iterator<DatanodeDescriptor> nodeIterator(Block b) {
     return nodeIterator(blocks.get(b));
   }
 
@@ -186,7 +185,7 @@ public class BlocksMap {
   /**
    * Check if the block exists in map
    */
-  public boolean contains(Block block) {
+  boolean contains(Block block) {
     return blocks.contains(block);
   }
   

+ 1 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/CorruptReplicasMap.java

@@ -15,11 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.ipc.Server;
 
 import java.util.*;

+ 34 - 38
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.DataInput;
 import java.io.IOException;
@@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.io.WritableUtils;
@@ -45,7 +44,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   
   // Stores status of decommissioning.
   // If node is not decommissioning, do not use this object for anything.
-  public DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
+  DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
   
   /** Block and targets pair */
   @InterfaceAudience.Private
@@ -97,8 +96,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private int numBlocks = 0;
   // isAlive == heartbeats.contains(this)
   // This is an optimization, because contains takes O(n) time on Arraylist
-  public boolean isAlive = false;
-  public boolean needKeyUpdate = false;
+  protected boolean isAlive = false;
+  protected boolean needKeyUpdate = false;
 
   /** A queue of blocks to be replicated by this datanode */
   private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
@@ -205,7 +204,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Add datanode to the block.
    * Add block to the head of the list of blocks belonging to the data-node.
    */
-  public boolean addBlock(BlockInfo b) {
+  boolean addBlock(BlockInfo b) {
     if(!b.addNode(this))
       return false;
     // add to the head of the data-node list
@@ -218,7 +217,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Remove block from the list of blocks belonging to the data-node.
    * Remove datanode from the block.
    */
-  public boolean removeBlock(BlockInfo b) {
+  boolean removeBlock(BlockInfo b) {
     blockList = b.listRemove(blockList, this);
     if ( b.removeNode(this) ) {
       numBlocks--;
@@ -243,7 +242,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * @param newBlock - a replacement block
    * @return the new block
    */
-  public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
+  BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
     boolean done = removeBlock(oldBlock);
     assert done : "Old block should belong to the data-node when replacing";
     done = addBlock(newBlock);
@@ -251,7 +250,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return newBlock;
   }
 
-  public void resetBlocks() {
+  void resetBlocks() {
     this.capacity = 0;
     this.remaining = 0;
     this.blockPoolUsed = 0;
@@ -269,7 +268,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Updates stats from datanode heartbeat.
    */
-  public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
+  void updateHeartbeat(long capacity, long dfsUsed, long remaining,
       long blockPoolUsed, int xceiverCount, int volFailures) {
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
@@ -284,7 +283,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Iterates over the list of blocks belonging to the datanode.
    */
-  public static class BlockIterator implements Iterator<BlockInfo> {
+  static class BlockIterator implements Iterator<BlockInfo> {
     private BlockInfo current;
     private DatanodeDescriptor node;
       
@@ -308,7 +307,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
-  public Iterator<BlockInfo> getBlockIterator() {
+  Iterator<BlockInfo> getBlockIterator() {
     return new BlockIterator(this.blockList, this);
   }
   
@@ -362,11 +361,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
   
-  public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
+  List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
     return replicateBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
+  BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
@@ -376,7 +375,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Remove the specified number of blocks to be invalidated
    */
-  public Block[] getInvalidateBlocks(int maxblocks) {
+  Block[] getInvalidateBlocks(int maxblocks) {
     return getBlockArray(invalidateBlocks, maxblocks); 
   }
 
@@ -419,7 +418,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /** Serialization for FSEditLog */
-  public void readFieldsFromFSEditLog(DataInput in) throws IOException {
+  void readFieldsFromFSEditLog(DataInput in) throws IOException {
     this.name = DeprecatedUTF8.readString(in);
     this.storageID = DeprecatedUTF8.readString(in);
     this.infoPort = in.readShort() & 0x0000ffff;
@@ -446,7 +445,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Increments counter for number of blocks scheduled. 
    */
-  public void incBlocksScheduled() {
+  void incBlocksScheduled() {
     currApproxBlocksScheduled++;
   }
   
@@ -486,13 +485,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     // by DatanodeID
     return (this == obj) || super.equals(obj);
   }
-
-  /** Decommissioning status */
-  public class DecommissioningStatus {
-    private int underReplicatedBlocks;
-    private int decommissionOnlyReplicas;
-    private int underReplicatedInOpenFiles;
-    private long startTime;
+  
+  class DecommissioningStatus {
+    int underReplicatedBlocks;
+    int decommissionOnlyReplicas;
+    int underReplicatedInOpenFiles;
+    long startTime;
     
     synchronized void set(int underRep,
         int onlyRep, int underConstruction) {
@@ -503,34 +501,32 @@ public class DatanodeDescriptor extends DatanodeInfo {
       decommissionOnlyReplicas = onlyRep;
       underReplicatedInOpenFiles = underConstruction;
     }
-
-    /** @return the number of under-replicated blocks */
-    public synchronized int getUnderReplicatedBlocks() {
+    
+    synchronized int getUnderReplicatedBlocks() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
       return underReplicatedBlocks;
     }
-    /** @return the number of decommission-only replicas */
-    public synchronized int getDecommissionOnlyReplicas() {
+    synchronized int getDecommissionOnlyReplicas() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
       return decommissionOnlyReplicas;
     }
-    /** @return the number of under-replicated blocks in open files */
-    public synchronized int getUnderReplicatedInOpenFiles() {
+
+    synchronized int getUnderReplicatedInOpenFiles() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
       return underReplicatedInOpenFiles;
     }
-    /** Set start time */
-    public synchronized void setStartTime(long time) {
+
+    synchronized void setStartTime(long time) {
       startTime = time;
     }
-    /** @return start time */
-    public synchronized long getStartTime() {
+    
+    synchronized long getStartTime() {
       if (isDecommissionInProgress() == false) {
         return 0;
       }
@@ -542,11 +538,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Set the flag to indicate if this datanode is disallowed from communicating
    * with the namenode.
    */
-  public void setDisallowed(boolean flag) {
+  void setDisallowed(boolean flag) {
     disallowed = flag;
   }
-  /** Is the datanode disallowed from communicating with the namenode? */
-  public boolean isDisallowed() {
+  
+  boolean isDisallowed() {
     return disallowed;
   }
 

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

@@ -21,7 +21,6 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.util.CyclicIteration;
 
 /**

+ 11 - 18
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
-
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -26,40 +24,35 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
-import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
-import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
+import org.apache.hadoop.hdfs.protocol.FSLimitException.*;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 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.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.util.ByteArray;
+import static org.apache.hadoop.hdfs.server.common.Util.now;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /*************************************************
  * FSDirectory stores the filesystem directory state.
@@ -70,7 +63,7 @@ import org.apache.hadoop.hdfs.util.ByteArray;
  * and logged to disk.
  * 
  *************************************************/
-public class FSDirectory implements Closeable {
+class FSDirectory implements Closeable {
 
   INodeDirectoryWithQuota rootDir;
   FSImage fsImage;  
@@ -1339,7 +1332,7 @@ public class FSDirectory implements Closeable {
    * @throws QuotaExceededException if the new count violates any quota limit
    * @throws FileNotFound if path does not exist.
    */
-  public void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
+  void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
                                          throws QuotaExceededException,
                                                 FileNotFoundException,
                                                 UnresolvedLinkException {

+ 2 - 24
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
-
 import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.EOFException;
@@ -34,30 +32,10 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 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.BlockInfoUnderConstruction;
+import static org.apache.hadoop.hdfs.server.common.Util.now;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
 
 public class FSEditLogLoader {
   private final FSNamesystem fsNamesys;

+ 0 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -32,8 +32,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.io.BytesWritable;

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

@@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 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.common.GenerationStamp;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;

+ 1 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -22,6 +22,7 @@ import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -32,9 +33,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;

+ 68 - 34
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -97,20 +97,15 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
-import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 import org.apache.hadoop.hdfs.server.common.Util;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@@ -240,7 +235,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   // Stores the correct file name hierarchy
   //
   public FSDirectory dir;
-  public BlockManager blockManager;
+  BlockManager blockManager;
   
   // Block pool ID used by this namenode
   String blockPoolId;
@@ -275,10 +270,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Stores a set of DatanodeDescriptor objects.
    * This is a subset of {@link #datanodeMap}, containing nodes that are 
    * considered alive.
-   * The HeartbeatMonitor periodically checks for out-dated entries,
+   * The {@link HeartbeatMonitor} periodically checks for outdated entries,
    * and removes them from the list.
    */
-  public ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
+  ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
 
   public LeaseManager leaseManager = new LeaseManager(this); 
 
@@ -319,8 +314,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   private volatile SafeModeInfo safeMode;  // safe mode information
   private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
     
-  /** datanode network toplogy */
-  public NetworkTopology clusterMap = new NetworkTopology();
+  // datanode networktoplogy
+  NetworkTopology clusterMap = new NetworkTopology();
   private DNSToSwitchMapping dnsToSwitchMapping;
 
   private HostsFileReader hostsReader; 
@@ -334,7 +329,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   private final GenerationStamp generationStamp = new GenerationStamp();
 
   // Ask Datanode only up to this many blocks to delete.
-  public int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
+  int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
 
   // precision of access times.
   private long accessTimePrecision = 0;
@@ -477,23 +472,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
 
   // utility methods to acquire and release read lock and write lock
-  public void readLock() {
+  void readLock() {
     this.fsLock.readLock().lock();
   }
 
-  public void readUnlock() {
+  void readUnlock() {
     this.fsLock.readLock().unlock();
   }
 
-  public void writeLock() {
+  void writeLock() {
     this.fsLock.writeLock().lock();
   }
 
-  public void writeUnlock() {
+  void writeUnlock() {
     this.fsLock.writeLock().unlock();
   }
 
-  public boolean hasWriteLock() {
+  boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();
   }
 
@@ -1019,7 +1014,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
 
   /** Create a LocatedBlock. */
-  public LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
+  LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
       final long offset, final boolean corrupt) throws IOException {
     return new LocatedBlock(getExtendedBlock(b), locations, offset, corrupt);
   }
@@ -3018,7 +3013,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * @return an array of datanode commands 
    * @throws IOException
    */
-  public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
+  DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
       long capacity, long dfsUsed, long remaining, long blockPoolUsed,
       int xceiverCount, int xmitsInProgress, int failedVolumes) 
         throws IOException {
@@ -3526,7 +3521,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * If no such a node is available,
    * then pick a node with least free space
    */
-  public void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
+  void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess, 
                               Block b, short replication,
                               DatanodeDescriptor addedNode,
                               DatanodeDescriptor delNodeHint,
@@ -3984,6 +3979,45 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
     return replication;
   }
     
+  /**
+   * A immutable object that stores the number of live replicas and
+   * the number of decommissined Replicas.
+   */
+  static class NumberReplicas {
+    private int liveReplicas;
+    int decommissionedReplicas;
+    private int corruptReplicas;
+    private int excessReplicas;
+
+    NumberReplicas() {
+      initialize(0, 0, 0, 0);
+    }
+
+    NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
+      initialize(live, decommissioned, corrupt, excess);
+    }
+
+    void initialize(int live, int decommissioned, int corrupt, int excess) {
+      liveReplicas = live;
+      decommissionedReplicas = decommissioned;
+      corruptReplicas = corrupt;
+      excessReplicas = excess;
+    }
+
+    int liveReplicas() {
+      return liveReplicas;
+    }
+    int decommissionedReplicas() {
+      return decommissionedReplicas;
+    }
+    int corruptReplicas() {
+      return corruptReplicas;
+    }
+    int excessReplicas() {
+      return excessReplicas;
+    }
+  } 
+
   /**
    * Change, if appropriate, the admin state of a datanode to 
    * decommission completed. Return true if decommission is complete.
@@ -4641,7 +4675,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Check whether the name node is in safe mode.
    * @return true if safe mode is ON, false otherwise
    */
-  public boolean isInSafeMode() {
+  boolean isInSafeMode() {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4652,7 +4686,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * Check whether the name node is in startup mode.
    */
-  public boolean isInStartupSafeMode() {
+  boolean isInStartupSafeMode() {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4663,7 +4697,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * Check whether replication queues are populated.
    */
-  public boolean isPopulatingReplQueues() {
+  boolean isPopulatingReplQueues() {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4675,7 +4709,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
    * Increment number of blocks that reached minimal replication.
    * @param replication current replication 
    */
-  public void incrementSafeBlockCount(int replication) {
+  void incrementSafeBlockCount(int replication) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
@@ -4686,7 +4720,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * Decrement number of blocks that reached minimal replication.
    */
-  public void decrementSafeBlockCount(Block b) {
+  void decrementSafeBlockCount(Block b) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
@@ -5008,13 +5042,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   @Override // FSNamesystemMBean
   @Metric
   public long getPendingReplicationBlocks() {
-    return blockManager.getPendingReplicationBlocksCount();
+    return blockManager.pendingReplicationBlocksCount;
   }
 
   @Override // FSNamesystemMBean
   @Metric
   public long getUnderReplicatedBlocks() {
-    return blockManager.getUnderReplicatedBlocksCount();
+    return blockManager.underReplicatedBlocksCount;
   }
 
   /** Return number of under-replicated but not missing blocks */
@@ -5025,23 +5059,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /** Returns number of blocks with corrupt replicas */
   @Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
   public long getCorruptReplicaBlocks() {
-    return blockManager.getCorruptReplicaBlocksCount();
+    return blockManager.corruptReplicaBlocksCount;
   }
 
   @Override // FSNamesystemMBean
   @Metric
   public long getScheduledReplicationBlocks() {
-    return blockManager.getScheduledReplicationBlocksCount();
+    return blockManager.scheduledReplicationBlocksCount;
   }
 
   @Metric
   public long getPendingDeletionBlocks() {
-    return blockManager.getPendingDeletionBlocksCount();
+    return blockManager.pendingDeletionBlocksCount;
   }
 
   @Metric
   public long getExcessBlocks() {
-    return blockManager.getExcessBlocksCount();
+    return blockManager.excessBlocksCount;
   }
   
   @Metric
@@ -5410,7 +5444,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   }
 
   /** Get a datanode descriptor given corresponding storageID */
-  public DatanodeDescriptor getDatanode(String nodeID) {
+  DatanodeDescriptor getDatanode(String nodeID) {
     assert hasReadOrWriteLock();
     return datanodeMap.get(nodeID);
   }
@@ -5474,7 +5508,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
       if (startBlockAfter != null) {
         startBlockId = Block.filename2id(startBlockAfter);
       }
-      UnderReplicatedBlocks.BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
       while (blkIterator.hasNext()) {
         Block blk = blkIterator.next();
         INode inode = blockManager.getINode(blk);

+ 1 - 4
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/Host2NodesMap.java

@@ -17,13 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.HashMap;
-import java.util.Random;
+import java.util.*;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-
 class Host2NodesMap {
   private HashMap<String, DatanodeDescriptor[]> map
     = new HashMap<String, DatanodeDescriptor[]>();

+ 4 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -20,13 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -34,7 +32,7 @@ import org.apache.hadoop.util.StringUtils;
  * This is a base INode class containing common fields for file and 
  * directory inodes.
  */
-public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
+abstract class INode implements Comparable<byte[]>, FSInodeInfo {
   /*
    *  The inode name is in java UTF8 encoding; 
    *  The name in HdfsFileStatus should keep the same encoding as this.
@@ -326,7 +324,7 @@ public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
   /**
    * Is this inode being constructed?
    */
-  public boolean isUnderConstruction() {
+  boolean isUnderConstruction() {
     return false;
   }
 

+ 5 - 9
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -24,11 +24,8 @@ import org.apache.hadoop.fs.permission.FsAction;
 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.BlockInfoUnderConstruction;
 
-/** I-node for closed file. */
-public class INodeFile extends INode {
+class INodeFile extends INode {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 
   //Number of bits for Block size
@@ -109,7 +106,7 @@ public class INodeFile extends INode {
    * Get file blocks 
    * @return file blocks
    */
-  public BlockInfo[] getBlocks() {
+  BlockInfo[] getBlocks() {
     return this.blocks;
   }
 
@@ -152,7 +149,7 @@ public class INodeFile extends INode {
   /**
    * Set file block
    */
-  public void setBlock(int idx, BlockInfo blk) {
+  void setBlock(int idx, BlockInfo blk) {
     this.blocks[idx] = blk;
   }
 
@@ -240,7 +237,7 @@ public class INodeFile extends INode {
    * Get the last block of the file.
    * Make sure it has the right type.
    */
-  public <T extends BlockInfo> T getLastBlock() throws IOException {
+  <T extends BlockInfo> T getLastBlock() throws IOException {
     if (blocks == null || blocks.length == 0)
       return null;
     T returnBlock = null;
@@ -255,8 +252,7 @@ public class INodeFile extends INode {
     return returnBlock;
   }
 
-  /** @return the number of blocks */ 
-  public int numBlocks() {
+  int numBlocks() {
     return blocks == null ? 0 : blocks.length;
   }
 }

+ 5 - 10
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -21,15 +21,10 @@ import java.io.IOException;
 
 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.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 
-/**
- * I-node for file being written.
- */
-public class INodeFileUnderConstruction extends INodeFile {
+
+class INodeFileUnderConstruction extends INodeFile {
   private  String clientName;         // lease holder
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@@ -48,7 +43,7 @@ public class INodeFileUnderConstruction extends INodeFile {
     this.clientNode = clientNode;
   }
 
-  INodeFileUnderConstruction(byte[] name,
+  public INodeFileUnderConstruction(byte[] name,
                              short blockReplication,
                              long modificationTime,
                              long preferredBlockSize,
@@ -85,7 +80,7 @@ public class INodeFileUnderConstruction extends INodeFile {
    * Is this inode being constructed?
    */
   @Override
-  public boolean isUnderConstruction() {
+  boolean isUnderConstruction() {
     return true;
   }
 
@@ -127,7 +122,7 @@ public class INodeFileUnderConstruction extends INodeFile {
    * Convert the last block of the file to an under-construction block.
    * Set its locations.
    */
-  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
+  BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
                                           DatanodeDescriptor[] targets)
   throws IOException {
     if (blocks == null || blocks.length == 0) {

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

@@ -163,7 +163,7 @@ public class LeaseManager {
   /**
    * Finds the pathname for the specified pendingFile
    */
-  public synchronized String findPath(INodeFileUnderConstruction pendingFile)
+  synchronized String findPath(INodeFileUnderConstruction pendingFile)
       throws IOException {
     Lease lease = getLease(pendingFile.getClientName());
     if (lease != null) {

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

@@ -247,7 +247,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
   /** Return the {@link FSNamesystem} object.
    * @return {@link FSNamesystem} object.
    */
-  public FSNamesystem getNamesystem() {
+  FSNamesystem getNamesystem() {
     return namesystem;
   }
 

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

@@ -36,20 +36,20 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NodeBase;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 
 /**

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

@@ -36,24 +36,24 @@ import javax.servlet.http.HttpServletResponse;
 import javax.servlet.jsp.JspWriter;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
-import org.znerd.xmlenc.XMLOutputter;
+
+import org.znerd.xmlenc.*;
 
 class NamenodeJspHelper {
   static String getSafeModeText(FSNamesystem fsn) {

+ 1 - 3
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/PendingReplicationBlocks.java

@@ -15,11 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-
 import static org.apache.hadoop.hdfs.server.common.Util.now;
 import org.apache.hadoop.util.*;
 import java.io.*;

+ 5 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java → hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/UnderReplicatedBlocks.java

@@ -15,18 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.*;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 /* Class for keeping track of under replication blocks
  * Blocks have replication priority, with priority 0 indicating the highest
  * Blocks have only one replicas has the highest
  */
-public class UnderReplicatedBlocks implements Iterable<Block> {
+class UnderReplicatedBlocks implements Iterable<Block> {
   static final int LEVEL = 5;
   static public final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   private List<TreeSet<Block>> priorityQueues = new ArrayList<TreeSet<Block>>();
@@ -48,7 +47,7 @@ public class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /* Return the total number of under replication blocks */
-  public synchronized int size() {
+  synchronized int size() {
     int size = 0;
     for (int i=0; i<LEVEL; i++) {
       size += priorityQueues.get(i).size();
@@ -71,7 +70,7 @@ public class UnderReplicatedBlocks implements Iterable<Block> {
   }
   
   /* Check if a block is in the neededReplication queue */
-  public synchronized boolean contains(Block block) {
+  synchronized boolean contains(Block block) {
     for(TreeSet<Block> set:priorityQueues) {
       if(set.contains(block)) { return true; }
     }
@@ -219,7 +218,7 @@ public class UnderReplicatedBlocks implements Iterable<Block> {
     return new BlockIterator();
   }
   
-  public class BlockIterator implements Iterator<Block> {
+  class BlockIterator implements Iterator<Block> {
     private int level;
     private boolean isIteratorForLevel = false;
     private List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();

+ 3 - 8
hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java

@@ -17,20 +17,15 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
+import java.io.*;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
-import org.apache.hadoop.io.WritableFactory;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.io.*;
 
 
 /****************************************************

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -61,10 +61,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java

@@ -25,7 +25,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 
 /**
  * This class tests DatanodeDescriptor.getBlocksScheduled() at the

+ 11 - 12
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java

@@ -17,31 +17,30 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
-import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
-
 import java.io.File;
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import static org.apache.hadoop.test.MetricsAsserts.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.log4j.Level;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
 /**
  * Test reporting of DN volume failure counts and metrics.

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

@@ -23,7 +23,6 @@ import java.io.IOException;
 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.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 

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

@@ -19,11 +19,10 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 
+import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.ipc.Server;
 
 /**
  * This is a utility class to expose NameNode functionality for unit tests.

+ 0 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java

@@ -34,8 +34,6 @@ import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;

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

@@ -17,15 +17,14 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 
+import junit.framework.TestCase;
+
 /**
  * Test if FSNamesystem handles heartbeat right
  */

+ 6 - 8
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestCorruptReplicaInfo.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCorruptReplicaInfo.java

@@ -15,20 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
+import java.util.*;
 import junit.framework.TestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.CorruptReplicasMap;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 
 
 /**
@@ -82,6 +79,7 @@ public class TestCorruptReplicaInfo extends TestCase {
       
       DatanodeDescriptor dn1 = new DatanodeDescriptor();
       DatanodeDescriptor dn2 = new DatanodeDescriptor();
+      DatanodeDescriptor dn3 = new DatanodeDescriptor();
       
       crm.addToCorruptReplicasMap(getBlock(0), dn1);
       assertEquals("Number of corrupt blocks not returning correctly",

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeDescriptor.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDatanodeDescriptor.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.ArrayList;
 

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

@@ -29,9 +29,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;

+ 9 - 7
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java

@@ -17,15 +17,19 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Random;
 
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -36,10 +40,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 
 /**
  * This class tests the decommissioning of nodes.

+ 1 - 2
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;

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

@@ -18,10 +18,9 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+
+import junit.framework.TestCase;
 
 public class TestHost2NodesMap extends TestCase {
   static private Host2NodesMap map = new Host2NodesMap();

+ 6 - 6
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java

@@ -21,17 +21,17 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.util.ArrayList;
 
-import junit.framework.TestCase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 
 

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

@@ -21,8 +21,6 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.concurrent.TimeoutException;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,8 +30,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
+
+import junit.framework.TestCase;
 
 /**
  * Test if live nodes count per node is correct 

+ 4 - 5
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java

@@ -20,21 +20,20 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.IOException;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+
+import junit.framework.TestCase;
 
 public class TestOverReplicatedBlocks extends TestCase {
   /** Test processOverReplicatedBlock can handle corrupt replicas fine.

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

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import junit.framework.TestCase;
 import java.lang.System;

+ 9 - 9
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java

@@ -15,7 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
+
+package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -23,19 +24,18 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetworkTopology;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.net.Node;
+
+import junit.framework.TestCase;
 
 public class TestReplicationPolicy extends TestCase {
   private static final int BLOCK_SIZE = 1024;

+ 3 - 4
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlocks.java → hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestUnderReplicatedBlocks.java

@@ -15,9 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import junit.framework.TestCase;
+package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -27,7 +25,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+
+import junit.framework.TestCase;
 
 public class TestUnderReplicatedBlocks extends TestCase {
   public void testSetrepIncWithUnderReplicatedBlocks() throws Exception {

+ 8 - 9
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -17,35 +17,34 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.metrics;
 
-import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
-import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
-import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
-
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.Random;
 
 import junit.framework.TestCase;
 
-import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.log4j.Level;
 
+import org.apache.commons.logging.LogFactory;
+
+import static org.apache.hadoop.test.MetricsAsserts.*;
+
 /**
  * Test for metrics published by the Namenode
  */

+ 1 - 1
hdfs/src/test/hdfs/org/apache/hadoop/net/TestNetworkTopology.java

@@ -24,8 +24,8 @@ import java.util.Map;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 public class TestNetworkTopology extends TestCase {
   private final static NetworkTopology cluster = new NetworkTopology();

+ 7 - 13
hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestNNLeaseRecovery.java

@@ -19,16 +19,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static junit.framework.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-import java.io.File;
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -44,16 +34,20 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.After;
+
+import static org.junit.Assert.*;
 import org.junit.Before;
 import org.junit.Test;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.io.IOException;
 
 public class TestNNLeaseRecovery {
   private static final Log LOG = LogFactory.getLog(TestNNLeaseRecovery.class);

+ 10 - 0
hdfs/src/webapps/hdfs/block_info_xml.jsp

@@ -59,9 +59,19 @@
 %>
 <%@ page
   contentType="application/xml"
+  import="java.io.IOException"
+  import="java.util.Iterator"
+  import="org.apache.hadoop.conf.Configuration"
+  import="org.apache.hadoop.hdfs.protocol.Block"
+  import="org.apache.hadoop.hdfs.server.namenode.INode"
+  import="org.apache.hadoop.hdfs.server.namenode.BlocksMap"
+  import="org.apache.hadoop.hdfs.server.namenode.BlockInfo"
+  import="org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor"
   import="org.apache.hadoop.hdfs.server.namenode.NamenodeJspHelper.XMLBlockInfo"
   import="org.apache.hadoop.hdfs.server.common.JspHelper"
+  import="org.apache.hadoop.util.ServletUtil"
   import="org.znerd.xmlenc.*"
+  
 %>
 <%!
   //for java.io.Serializable