Browse Source

svn merge -c 1335304 from trunk for HDFS-3363. Define BlockCollection and MutableBlockCollection interfaces so that INodeFile and INodeFileUnderConstruction do not have to be used in block management.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1335305 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 years ago
parent
commit
d8156648fe
16 changed files with 172 additions and 93 deletions
  1. 4 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 63 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  3. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
  5. 28 31
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  6. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  7. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  8. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  9. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java
  10. 0 38
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSInodeInfo.java
  11. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  12. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  13. 10 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  14. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java

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

@@ -289,6 +289,10 @@ Release 2.0.0 - UNRELEASED
     HDFS-3375. Put client name in DataXceiver thread name for readBlock
     and keepalive (todd)
 
+    HDFS-3363. Define BlockCollection and MutableBlockCollection interfaces
+    so that INodeFile and INodeFileUnderConstruction do not have to be used in
+    block management.  (John George via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-2477. Optimize computing the diff between a block report and the

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java

@@ -0,0 +1,63 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.fs.ContentSummary;
+
+/** 
+ * This interface is used by the block manager to expose a
+ * few characteristics of a collection of Block/BlockUnderConstruction.
+ */
+public interface BlockCollection {
+  /**
+   * Get the last block of the collection.
+   * Make sure it has the right type.
+   */
+  public <T extends BlockInfo> T getLastBlock() throws IOException;
+
+  /** 
+   * Get content summary.
+   */
+  public ContentSummary computeContentSummary();
+
+  /** @return the number of blocks */ 
+  public int numBlocks();
+
+  public BlockInfo[] getBlocks();
+  /**
+   * Get preferred block size for the collection 
+   * @return preferred block size in bytes
+   */
+  public long getPreferredBlockSize();
+
+  /**
+   * Get block replication for the collection 
+   * @return block replication value
+   */
+  public short getReplication();
+
+  /**
+   *  Get name of collection.
+   */
+  public String getName();
+}

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -19,14 +19,16 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.util.LightWeightGSet;
 
 /**
  * Internal class for block metadata.
+ * BlockInfo class maintains for a given block
+ * the {@link BlockCollection} it is part of and datanodes where the replicas of 
+ * the block are stored.
  */
 public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
-  private INodeFile inode;
+  private BlockCollection inode;
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   private LightWeightGSet.LinkedElement nextLinkedElement;
@@ -66,11 +68,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
     this.inode = from.inode;
   }
 
-  public INodeFile getINode() {
+  public BlockCollection getINode() {
     return inode;
   }
 
-  public void setINode(INodeFile inode) {
+  public void setINode(BlockCollection inode) {
     this.inode = inode;
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java

@@ -234,7 +234,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
       NameNode.stateChangeLog.warn("BLOCK*"
-        + " INodeFileUnderConstruction.initLeaseRecovery:"
+        + " BlockInfoUnderConstruction.initLeaseRecovery:"
         + " No blocks found, lease removed.");
     }
 

+ 28 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -57,8 +57,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
-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.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
@@ -386,7 +384,7 @@ public class BlockManager {
                          numReplicas.decommissionedReplicas();
     
     if (block instanceof BlockInfo) {
-      String fileName = ((BlockInfo)block).getINode().getFullPathName();
+      String fileName = ((BlockInfo)block).getINode().getName();
       out.print(fileName + ": ");
     }
     // l: == live:, d: == decommissioned c: == corrupt e: == excess
@@ -462,7 +460,7 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  public boolean commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode, 
+  public boolean commitOrCompleteLastBlock(MutableBlockCollection fileINode, 
       Block commitBlock) throws IOException {
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
@@ -474,7 +472,7 @@ public class BlockManager {
     
     final boolean b = commitBlock((BlockInfoUnderConstruction)lastBlock, commitBlock);
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
-      completeBlock(fileINode,fileINode.numBlocks()-1, false);
+      completeBlock(fileINode, fileINode.numBlocks()-1, false);
     return b;
   }
 
@@ -485,7 +483,7 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private BlockInfo completeBlock(final INodeFile fileINode,
+  private BlockInfo completeBlock(final MutableBlockCollection fileINode,
       final int blkIndex, boolean force) throws IOException {
     if(blkIndex < 0)
       return null;
@@ -518,7 +516,7 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
   }
 
-  private BlockInfo completeBlock(final INodeFile fileINode,
+  private BlockInfo completeBlock(final MutableBlockCollection fileINode,
       final BlockInfo block, boolean force) throws IOException {
     BlockInfo[] fileBlocks = fileINode.getBlocks();
     for(int idx = 0; idx < fileBlocks.length; idx++)
@@ -533,7 +531,7 @@ public class BlockManager {
    * regardless of whether enough replicas are present. This is necessary
    * when tailing edit logs as a Standby.
    */
-  public BlockInfo forceCompleteBlock(final INodeFile fileINode,
+  public BlockInfo forceCompleteBlock(final MutableBlockCollection fileINode,
       final BlockInfoUnderConstruction block) throws IOException {
     block.commitBlock(block);
     return completeBlock(fileINode, block, true);
@@ -554,7 +552,7 @@ public class BlockManager {
    * @return the last block locations if the block is partial or null otherwise
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
-      INodeFileUnderConstruction fileINode) throws IOException {
+      MutableBlockCollection fileINode) throws IOException {
     BlockInfo oldBlock = fileINode.getLastBlock();
     if(oldBlock == null ||
         fileINode.getPreferredBlockSize() == oldBlock.getNumBytes())
@@ -925,7 +923,7 @@ public class BlockManager {
                             " does not exist. ");
     }
 
-    INodeFile inode = storedBlock.getINode();
+    BlockCollection inode = storedBlock.getINode();
     if (inode == null) {
       NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
                                    "block " + storedBlock +
@@ -1053,7 +1051,7 @@ public class BlockManager {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
     DatanodeDescriptor srcNode;
-    INodeFile fileINode = null;
+    BlockCollection fileINode = null;
     int additionalReplRequired;
 
     int scheduledWork = 0;
@@ -1067,7 +1065,7 @@ public class BlockManager {
             // block should belong to a file
             fileINode = blocksMap.getINode(block);
             // abandoned block or block reopened for append
-            if(fileINode == null || fileINode.isUnderConstruction()) {
+            if(fileINode == null || fileINode instanceof MutableBlockCollection) {
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.decrementReplicationIndex(priority);
               continue;
@@ -1153,7 +1151,7 @@ public class BlockManager {
           // block should belong to a file
           fileINode = blocksMap.getINode(block);
           // abandoned block or block reopened for append
-          if(fileINode == null || fileINode.isUnderConstruction()) {
+          if(fileINode == null || fileINode instanceof MutableBlockCollection) {
             neededReplications.remove(block, priority); // remove from neededReplications
             rw.targets = null;
             neededReplications.decrementReplicationIndex(priority);
@@ -1918,7 +1916,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
         && numCurrentReplica >= minReplication) {
-      completeBlock(storedBlock.getINode(), storedBlock, false);
+      completeBlock((MutableBlockCollection)storedBlock.getINode(), storedBlock, false);
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that.
@@ -1956,7 +1954,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       return block;
     }
     assert storedBlock != null : "Block must be stored by now";
-    INodeFile fileINode = storedBlock.getINode();
+    BlockCollection fileINode = storedBlock.getINode();
     assert fileINode != null : "Block must belong to a file";
 
     // add block to the datanode
@@ -1983,7 +1981,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
         numLiveReplicas >= minReplication) {
-      storedBlock = completeBlock(fileINode, storedBlock, false);
+      storedBlock = completeBlock((MutableBlockCollection)fileINode, storedBlock, false);
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that
@@ -1994,7 +1992,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
     
     // if file is under construction, then done for now
-    if (fileINode.isUnderConstruction()) {
+    if (fileINode instanceof MutableBlockCollection) {
       return storedBlock;
     }
 
@@ -2131,7 +2129,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
    * what happened with it.
    */
   private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
-    INodeFile fileINode = block.getINode();
+    BlockCollection fileINode = block.getINode();
     if (fileINode == null) {
       // block does not belong to any file
       addToInvalidates(block);
@@ -2260,7 +2258,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
                               BlockPlacementPolicy replicator) {
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
-    INodeFile inode = getINode(b);
+    BlockCollection inode = getINode(b);
     final Map<String, List<DatanodeDescriptor>> rackMap
         = new HashMap<String, List<DatanodeDescriptor>>();
     for(final Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
@@ -2381,7 +2379,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       // necessary. In that case, put block on a possibly-will-
       // be-replicated list.
       //
-      INodeFile fileINode = blocksMap.getINode(block);
+      BlockCollection fileINode = blocksMap.getINode(block);
       if (fileINode != null) {
         namesystem.decrementSafeBlockCount(block);
         updateNeededReplications(block, -1, 0);
@@ -2613,7 +2611,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
       NumberReplicas num) {
     int curReplicas = num.liveReplicas();
     int curExpectedReplicas = getReplication(block);
-    INodeFile fileINode = blocksMap.getINode(block);
+    BlockCollection fileINode = blocksMap.getINode(block);
     Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
     StringBuilder nodeList = new StringBuilder();
     while (nodeIter.hasNext()) {
@@ -2626,7 +2624,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         + ", corrupt replicas: " + num.corruptReplicas()
         + ", decommissioned replicas: " + num.decommissionedReplicas()
         + ", excess replicas: " + num.excessReplicas()
-        + ", Is Open File: " + fileINode.isUnderConstruction()
+        + ", Is Open File: " + (fileINode instanceof MutableBlockCollection)
         + ", Datanodes having this block: " + nodeList + ", Current Datanode: "
         + srcNode + ", Is current datanode decommissioning: "
         + srcNode.isDecommissionInProgress());
@@ -2641,7 +2639,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     final Iterator<? extends Block> it = srcNode.getBlockIterator();
     while(it.hasNext()) {
       final Block block = it.next();
-      INodeFile fileINode = blocksMap.getINode(block);
+      BlockCollection fileINode = blocksMap.getINode(block);
       short expectedReplication = fileINode.getReplication();
       NumberReplicas num = countNodes(block);
       int numCurrentReplica = num.liveReplicas();
@@ -2664,7 +2662,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     final Iterator<? extends Block> it = srcNode.getBlockIterator();
     while(it.hasNext()) {
       final Block block = it.next();
-      INodeFile fileINode = blocksMap.getINode(block);
+      BlockCollection fileINode = blocksMap.getINode(block);
 
       if (fileINode != null) {
         NumberReplicas num = countNodes(block);
@@ -2681,7 +2679,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
             if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
               decommissionOnlyReplicas++;
             }
-            if (fileINode.isUnderConstruction()) {
+            if (fileINode instanceof MutableBlockCollection) {
               underReplicatedInOpenFiles++;
             }
           }
@@ -2784,11 +2782,10 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
 
   /* get replication factor of a block */
   private int getReplication(Block block) {
-    INodeFile fileINode = blocksMap.getINode(block);
+    BlockCollection fileINode = blocksMap.getINode(block);
     if (fileINode == null) { // block does not belong to any file
       return 0;
     }
-    assert !fileINode.isDirectory() : "Block cannot belong to a directory.";
     return fileINode.getReplication();
   }
 
@@ -2861,11 +2858,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     return this.neededReplications.getCorruptBlockSize();
   }
 
-  public BlockInfo addINode(BlockInfo block, INodeFile iNode) {
+  public BlockInfo addINode(BlockInfo block, BlockCollection iNode) {
     return blocksMap.addINode(block, iNode);
   }
 
-  public INodeFile getINode(Block b) {
+  public BlockCollection getINode(Block b) {
     return blocksMap.getINode(b);
   }
 
@@ -3005,7 +3002,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
   private static class ReplicationWork {
 
     private Block block;
-    private INodeFile fileINode;
+    private BlockCollection fileINode;
 
     private DatanodeDescriptor srcNode;
     private List<DatanodeDescriptor> containingNodes;
@@ -3016,7 +3013,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     private int priority;
 
     public ReplicationWork(Block block,
-        INodeFile fileINode,
+        BlockCollection fileINode,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeDescriptor> liveReplicaNodes,

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

@@ -29,7 +29,6 @@ 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;
@@ -123,13 +122,13 @@ public abstract class BlockPlacementPolicy {
    * @return array of DatanodeDescriptor instances chosen as target 
    * and sorted as a pipeline.
    */
-  DatanodeDescriptor[] chooseTarget(FSInodeInfo srcInode,
+  DatanodeDescriptor[] chooseTarget(BlockCollection srcInode,
                                     int numOfReplicas,
                                     DatanodeDescriptor writer,
                                     List<DatanodeDescriptor> chosenNodes,
                                     HashMap<Node, Node> excludedNodes,
                                     long blocksize) {
-    return chooseTarget(srcInode.getFullPathName(), numOfReplicas, writer,
+    return chooseTarget(srcInode.getName(), numOfReplicas, writer,
                         chosenNodes, excludedNodes, blocksize);
   }
 
@@ -159,7 +158,7 @@ public abstract class BlockPlacementPolicy {
                    listed in the previous parameter.
    * @return the replica that is the best candidate for deletion
    */
-  abstract public DatanodeDescriptor chooseReplicaToDelete(FSInodeInfo srcInode,
+  abstract public DatanodeDescriptor chooseReplicaToDelete(BlockCollection srcInode,
                                       Block block, 
                                       short replicationFactor,
                                       Collection<DatanodeDescriptor> existingReplicas,

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 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.net.NodeBase;
@@ -547,7 +546,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
   }
 
   @Override
-  public DatanodeDescriptor chooseReplicaToDelete(FSInodeInfo inode,
+  public DatanodeDescriptor chooseReplicaToDelete(BlockCollection inode,
                                                  Block block,
                                                  short replicationFactor,
                                                  Collection<DatanodeDescriptor> first, 

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

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 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;
 
@@ -93,7 +92,7 @@ class BlocksMap {
     blocks = null;
   }
 
-  INodeFile getINode(Block b) {
+  BlockCollection getINode(Block b) {
     BlockInfo info = blocks.get(b);
     return (info != null) ? info.getINode() : null;
   }
@@ -101,7 +100,7 @@ class BlocksMap {
   /**
    * Add block b belonging to the specified file inode to the map.
    */
-  BlockInfo addINode(BlockInfo b, INodeFile iNode) {
+  BlockInfo addINode(BlockInfo b, BlockCollection iNode) {
     BlockInfo info = blocks.get(b);
     if (info != b) {
       info = b;

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/MutableBlockCollection.java

@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.fs.ContentSummary;
+
+/** 
+ * This interface is used by the block manager to expose a
+ * few characteristics of a collection of Block/BlockUnderConstruction.
+ */
+public interface MutableBlockCollection extends BlockCollection {
+  /**
+   * Set block 
+   */
+  public void setBlock(int idx, BlockInfo blk);
+
+  /**
+   * Convert the last block of the collection to an under-construction block.
+   * Set its locations.
+   */
+  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
+                       DatanodeDescriptor[] targets) throws IOException;
+}

+ 0 - 38
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSInodeInfo.java

@@ -1,38 +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.namenode;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/** 
- * This interface is used used the pluggable block placement policy
- * to expose a few characteristics of an Inode.
- */
-@InterfaceAudience.Private
-public interface FSInodeInfo {
-
-  /**
-   * a string representation of an inode
-   * 
-   * @return the full pathname (from root) that this inode represents
-   */
-
-  public String getFullPathName() ;
-}
-    
-    

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2840,7 +2840,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (storedBlock == null) {
         throw new IOException("Block (=" + lastblock + ") not found");
       }
-      INodeFile iFile = storedBlock.getINode();
+      INodeFile iFile = (INodeFile) storedBlock.getINode();
       if (!iFile.isUnderConstruction() || storedBlock.isComplete()) {
         throw new IOException("Unexpected block (=" + lastblock
                               + ") since the file (=" + iFile.getLocalName()
@@ -4394,7 +4394,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     
     // check file inode
-    INodeFile file = storedBlock.getINode();
+    INodeFile file = (INodeFile) storedBlock.getINode();
     if (file==null || !file.isUnderConstruction()) {
       throw new IOException("The file " + storedBlock + 
           " belonged to does not exist or it is not under construction.");
@@ -4706,7 +4706,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       while (blkIterator.hasNext()) {
         Block blk = blkIterator.next();
-        INode inode = blockManager.getINode(blk);
+        INode inode = (INodeFile) blockManager.getINode(blk);
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
           String src = FSDirectory.getFullPathName(inode);

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -38,7 +38,7 @@ import com.google.common.primitives.SignedBytes;
  * directory inodes.
  */
 @InterfaceAudience.Private
-abstract class INode implements Comparable<byte[]>, FSInodeInfo {
+abstract class INode implements Comparable<byte[]> {
   /*
    *  The inode name is in java UTF8 encoding; 
    *  The name in HdfsFileStatus should keep the same encoding as this.
@@ -264,7 +264,6 @@ abstract class INode implements Comparable<byte[]>, FSInodeInfo {
     this.name = name;
   }
 
-  @Override
   public String getFullPathName() {
     // Get the full path name of this inode.
     return FSDirectory.getFullPathName(this);

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -20,15 +20,18 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 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;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 
 /** I-node for closed file. */
-public class INodeFile extends INode {
+@InterfaceAudience.Private
+public class INodeFile extends INode implements BlockCollection {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 
   //Number of bits for Block size
@@ -167,6 +170,12 @@ public class INodeFile extends INode {
     blocks = null;
     return 1;
   }
+  
+  public String getName() {
+    // Get the full path name of this inode.
+    return getFullPathName();
+  }
+
 
   @Override
   long[] computeContentSummary(long[] summary) {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -25,13 +25,15 @@ 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.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 
 import com.google.common.base.Joiner;
 
 /**
  * I-node for file being written.
  */
-public class INodeFileUnderConstruction extends INodeFile {
+public class INodeFileUnderConstruction extends INodeFile 
+                                        implements MutableBlockCollection {
   private  String clientName;         // lease holder
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -734,7 +734,7 @@ class NamenodeJspHelper {
         this.inode = null;
       } else {
         this.block = new Block(blockId);
-        this.inode = blockManager.getINode(block);
+        this.inode = (INodeFile) blockManager.getINode(block);
       }
     }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java

@@ -46,9 +46,9 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -585,7 +585,7 @@ public class TestDNFencing {
     }
 
     @Override
-    public DatanodeDescriptor chooseReplicaToDelete(FSInodeInfo inode,
+    public DatanodeDescriptor chooseReplicaToDelete(BlockCollection inode,
         Block block, short replicationFactor,
         Collection<DatanodeDescriptor> first,
         Collection<DatanodeDescriptor> second) {