Browse Source

HDFS-8248. Store INodeId instead of the INodeFile object in BlockInfoContiguous. Contributed by Haohui Mai.

Haohui Mai 10 years ago
parent
commit
4cbbfa2220
18 changed files with 154 additions and 94 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  3. 15 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  4. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  5. 3 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
  6. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
  7. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java
  8. 17 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  9. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  10. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeId.java
  11. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
  12. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
  13. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
  14. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
  15. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
  16. 39 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  17. 6 24
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
  18. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

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

@@ -844,6 +844,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8896. DataNode object isn't GCed when shutdown, because it has GC
     HDFS-8896. DataNode object isn't GCed when shutdown, because it has GC
     root in ShutdownHookManager. (Walter Su via jing9)
     root in ShutdownHookManager. (Walter Su via jing9)
 
 
+    HDFS-8248. Store INodeId instead of the INodeFile object in
+    BlockInfoContiguous. (wheat9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

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

@@ -79,4 +79,9 @@ public interface BlockCollection {
    * @return whether the block collection is under construction.
    * @return whether the block collection is under construction.
    */
    */
   public boolean isUnderConstruction();
   public boolean isUnderConstruction();
+
+  /**
+   * @return the id for the block collection
+   */
+  long getId();
 }
 }

+ 15 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -28,6 +28,8 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
 
 
+import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
+
 /**
 /**
  * BlockInfo class maintains for a given block
  * BlockInfo class maintains for a given block
  * the {@link BlockCollection} it is part of and datanodes where the replicas of
  * the {@link BlockCollection} it is part of and datanodes where the replicas of
@@ -40,11 +42,14 @@ public abstract class  BlockInfo extends Block
   public static final BlockInfo[] EMPTY_ARRAY = {};
   public static final BlockInfo[] EMPTY_ARRAY = {};
 
 
   /**
   /**
-   * Replication factor
+   * Replication factor.
    */
    */
   private short replication;
   private short replication;
 
 
-  private BlockCollection bc;
+  /**
+   * Block collection ID.
+   */
+  private long bcId;
 
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface. */
   /** For implementing {@link LightWeightGSet.LinkedElement} interface. */
   private LightWeightGSet.LinkedElement nextLinkedElement;
   private LightWeightGSet.LinkedElement nextLinkedElement;
@@ -71,14 +76,14 @@ public abstract class  BlockInfo extends Block
    */
    */
   public BlockInfo(short replication) {
   public BlockInfo(short replication) {
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
-    this.bc = null;
+    this.bcId = INVALID_INODE_ID;
     this.replication = replication;
     this.replication = replication;
   }
   }
 
 
   public BlockInfo(Block blk, short replication) {
   public BlockInfo(Block blk, short replication) {
     super(blk);
     super(blk);
     this.triplets = new Object[3*replication];
     this.triplets = new Object[3*replication];
-    this.bc = null;
+    this.bcId = INVALID_INODE_ID;
     this.replication = replication;
     this.replication = replication;
   }
   }
 
 
@@ -88,7 +93,7 @@ public abstract class  BlockInfo extends Block
    */
    */
   protected BlockInfo(BlockInfo from) {
   protected BlockInfo(BlockInfo from) {
     this(from, from.getReplication());
     this(from, from.getReplication());
-    this.bc = from.bc;
+    this.bcId = from.bcId;
   }
   }
 
 
   public short getReplication() {
   public short getReplication() {
@@ -99,16 +104,16 @@ public abstract class  BlockInfo extends Block
     this.replication = repl;
     this.replication = repl;
   }
   }
 
 
-  public BlockCollection getBlockCollection() {
-    return bc;
+  public long getBlockCollectionId() {
+    return bcId;
   }
   }
 
 
-  public void setBlockCollection(BlockCollection bc) {
-    this.bc = bc;
+  public void setBlockCollectionId(long id) {
+    this.bcId = id;
   }
   }
 
 
   public boolean isDeleted() {
   public boolean isDeleted() {
-    return (bc == null);
+    return bcId == INVALID_INODE_ID;
   }
   }
 
 
   public DatanodeDescriptor getDatanode(int index) {
   public DatanodeDescriptor getDatanode(int index) {

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -562,7 +562,7 @@ public class BlockManager implements BlockStatsMXBean {
                          numReplicas.decommissionedAndDecommissioning();
                          numReplicas.decommissionedAndDecommissioning();
     
     
     if (block instanceof BlockInfo) {
     if (block instanceof BlockInfo) {
-      BlockCollection bc = ((BlockInfo) block).getBlockCollection();
+      BlockCollection bc = getBlockCollection((BlockInfo)block);
       String fileName = (bc == null) ? "[orphaned]" : bc.getName();
       String fileName = (bc == null) ? "[orphaned]" : bc.getName();
       out.print(fileName + ": ");
       out.print(fileName + ": ");
     }
     }
@@ -1353,7 +1353,7 @@ public class BlockManager implements BlockStatsMXBean {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
           for (BlockInfo block : blocksToReplicate.get(priority)) {
           for (BlockInfo block : blocksToReplicate.get(priority)) {
             // block should belong to a file
             // block should belong to a file
-            bc = blocksMap.getBlockCollection(block);
+            bc = getBlockCollection(block);
             // abandoned block or block reopened for append
             // abandoned block or block reopened for append
             if (bc == null
             if (bc == null
                 || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
                 || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
@@ -1439,7 +1439,7 @@ public class BlockManager implements BlockStatsMXBean {
           int priority = rw.priority;
           int priority = rw.priority;
           // Recheck since global lock was released
           // Recheck since global lock was released
           // block should belong to a file
           // block should belong to a file
-          bc = blocksMap.getBlockCollection(block);
+          bc = getBlockCollection(block);
           // abandoned block or block reopened for append
           // abandoned block or block reopened for append
           if(bc == null || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
           if(bc == null || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
             neededReplications.remove(block, priority); // remove from neededReplications
             neededReplications.remove(block, priority); // remove from neededReplications
@@ -2542,7 +2542,7 @@ public class BlockManager implements BlockStatsMXBean {
     int numCurrentReplica = countLiveNodes(storedBlock);
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
         && numCurrentReplica >= minReplication) {
         && numCurrentReplica >= minReplication) {
-      completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
+      completeBlock(getBlockCollection(storedBlock), storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
       // check whether safe replication is reached for the block
       // only complete blocks are counted towards that.
       // only complete blocks are counted towards that.
@@ -2580,7 +2580,7 @@ public class BlockManager implements BlockStatsMXBean {
       // it will happen in next block report otherwise.
       // it will happen in next block report otherwise.
       return block;
       return block;
     }
     }
-    BlockCollection bc = storedBlock.getBlockCollection();
+    BlockCollection bc = getBlockCollection(storedBlock);
     assert bc != null : "Block must belong to a file";
     assert bc != null : "Block must belong to a file";
 
 
     // add block to the datanode
     // add block to the datanode
@@ -2975,7 +2975,8 @@ public class BlockManager implements BlockStatsMXBean {
                               BlockPlacementPolicy replicator) {
                               BlockPlacementPolicy replicator) {
     assert namesystem.hasWriteLock();
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
     // first form a rack to datanodes map and
-    BlockCollection bc = getBlockCollection(b);
+    BlockInfo bi = getStoredBlock(b);
+    BlockCollection bc = getBlockCollection(bi);
     final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
     final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
     final List<StorageType> excessTypes = storagePolicy.chooseExcess(
     final List<StorageType> excessTypes = storagePolicy.chooseExcess(
         replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
         replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
@@ -3112,7 +3113,7 @@ public class BlockManager implements BlockStatsMXBean {
       // necessary. In that case, put block on a possibly-will-
       // necessary. In that case, put block on a possibly-will-
       // be-replicated list.
       // be-replicated list.
       //
       //
-      BlockCollection bc = blocksMap.getBlockCollection(block);
+      BlockCollection bc = getBlockCollection(storedBlock);
       if (bc != null) {
       if (bc != null) {
         namesystem.decrementSafeBlockCount(storedBlock);
         namesystem.decrementSafeBlockCount(storedBlock);
         updateNeededReplications(storedBlock, -1, 0);
         updateNeededReplications(storedBlock, -1, 0);
@@ -3628,8 +3629,8 @@ public class BlockManager implements BlockStatsMXBean {
     return blocksMap.addBlockCollection(block, bc);
     return blocksMap.addBlockCollection(block, bc);
   }
   }
 
 
-  public BlockCollection getBlockCollection(Block b) {
-    return blocksMap.getBlockCollection(b);
+  public BlockCollection getBlockCollection(BlockInfo b) {
+    return namesystem.getBlockCollection(b.getBlockCollectionId());
   }
   }
 
 
   /** @return an iterator of the datanodes. */
   /** @return an iterator of the datanodes. */

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.Iterator;
 import java.util.Iterator;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -94,11 +95,6 @@ class BlocksMap {
     }
     }
   }
   }
 
 
-  BlockCollection getBlockCollection(Block b) {
-    BlockInfo info = blocks.get(b);
-    return (info != null) ? info.getBlockCollection() : null;
-  }
-
   /**
   /**
    * Add block b belonging to the specified block collection to the map.
    * Add block b belonging to the specified block collection to the map.
    */
    */
@@ -108,7 +104,7 @@ class BlocksMap {
       info = b;
       info = b;
       blocks.put(info);
       blocks.put(info);
     }
     }
-    info.setBlockCollection(bc);
+    info.setBlockCollectionId(bc.getId());
     return info;
     return info;
   }
   }
 
 
@@ -122,7 +118,7 @@ class BlocksMap {
     if (blockInfo == null)
     if (blockInfo == null)
       return;
       return;
 
 
-    blockInfo.setBlockCollection(null);
+    blockInfo.setBlockCollectionId(INodeId.INVALID_INODE_ID);
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
     for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
       dn.removeBlock(blockInfo); // remove from the list and wipe the location
       dn.removeBlock(blockInfo); // remove from the list and wipe the location

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.ChunkedArrayList;
@@ -528,12 +529,14 @@ public class DecommissionManager {
           it.remove();
           it.remove();
           continue;
           continue;
         }
         }
-        BlockCollection bc = blockManager.blocksMap.getBlockCollection(block);
-        if (bc == null) {
+
+        long bcId = block.getBlockCollectionId();
+        if (bcId == INodeId.INVALID_INODE_ID) {
           // Orphan block, will be invalidated eventually. Skip.
           // Orphan block, will be invalidated eventually. Skip.
           continue;
           continue;
         }
         }
 
 
+        BlockCollection bc = namesystem.getBlockCollection(bcId);
         final NumberReplicas num = blockManager.countNodes(block);
         final NumberReplicas num = blockManager.countNodes(block);
         final int liveReplicas = num.liveReplicas();
         final int liveReplicas = num.liveReplicas();
         final int curReplicas = liveReplicas;
         final int curReplicas = liveReplicas;

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.util.SequentialNumber;
 import org.apache.hadoop.util.SequentialNumber;
 
 
 /**
 /**
@@ -61,6 +62,8 @@ public class SequentialBlockIdGenerator extends SequentialNumber {
    * Returns whether the given block is one pointed-to by a file.
    * Returns whether the given block is one pointed-to by a file.
    */
    */
   private boolean isValidBlock(Block b) {
   private boolean isValidBlock(Block b) {
-    return (blockManager.getBlockCollection(b) != null);
+    BlockInfo bi = blockManager.getStoredBlock(b);
+    return bi != null && bi.getBlockCollectionId() !=
+        INodeId.INVALID_INODE_ID;
   }
   }
 }
 }

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

@@ -3209,7 +3209,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Override
   @Override
   public boolean isInSnapshot(BlockInfo blockUC) {
   public boolean isInSnapshot(BlockInfo blockUC) {
     assert hasReadLock();
     assert hasReadLock();
-    final BlockCollection bc = blockUC.getBlockCollection();
+    final BlockCollection bc = blockManager.getBlockCollection(blockUC);
     if (bc == null || !(bc instanceof INodeFile)
     if (bc == null || !(bc instanceof INodeFile)
         || !bc.isUnderConstruction()) {
         || !bc.isUnderConstruction()) {
       return false;
       return false;
@@ -3238,6 +3238,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return true;
     return true;
   }
   }
 
 
+  @Override
+  public BlockCollection getBlockCollection(long id) {
+    INode inode = getFSDirectory().getInode(id);
+    return inode == null ? null : inode.asFile();
+  }
+
   void commitBlockSynchronization(ExtendedBlock oldBlock,
   void commitBlockSynchronization(ExtendedBlock oldBlock,
       long newgenerationstamp, long newlength,
       long newgenerationstamp, long newlength,
       boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
       boolean closeFile, boolean deleteblock, DatanodeID[] newtargets,
@@ -3295,7 +3301,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             + " is null, likely because the file owning this block was"
             + " is null, likely because the file owning this block was"
             + " deleted and the block removal is delayed");
             + " deleted and the block removal is delayed");
       }
       }
-      INodeFile iFile = ((INode)storedBlock.getBlockCollection()).asFile();
+      long bcId = storedBlock.getBlockCollectionId();
+      INodeFile iFile = ((INode)getBlockCollection(bcId)).asFile();
       if (isFileDeleted(iFile)) {
       if (isFileDeleted(iFile)) {
         throw new FileNotFoundException("File not found: "
         throw new FileNotFoundException("File not found: "
             + iFile.getFullPathName() + ", likely due to delayed block"
             + iFile.getFullPathName() + ", likely due to delayed block"
@@ -3717,9 +3724,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         while (it.hasNext()) {
         while (it.hasNext()) {
           Block b = it.next();
           Block b = it.next();
           BlockInfo blockInfo = blockManager.getStoredBlock(b);
           BlockInfo blockInfo = blockManager.getStoredBlock(b);
-          if (blockInfo.getBlockCollection().getStoragePolicyID()
-              == lpPolicy.getId()) {
-            filesToDelete.add(blockInfo.getBlockCollection());
+          BlockCollection bc = getBlockCollection(
+              blockInfo.getBlockCollectionId());
+          if (bc.getStoragePolicyID() == lpPolicy.getId()) {
+            filesToDelete.add(bc);
           }
           }
         }
         }
 
 
@@ -5234,7 +5242,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     }
     
     
     // check file inode
     // check file inode
-    final INodeFile file = ((INode)storedBlock.getBlockCollection()).asFile();
+    long bcId = storedBlock.getBlockCollectionId();
+    final INodeFile file = ((INode)getBlockCollection(bcId)).asFile();
     if (file == null || !file.isUnderConstruction() || isFileDeleted(file)) {
     if (file == null || !file.isUnderConstruction() || isFileDeleted(file)) {
       throw new IOException("The file " + storedBlock + 
       throw new IOException("The file " + storedBlock + 
           " belonged to does not exist or it is not under construction.");
           " belonged to does not exist or it is not under construction.");
@@ -5497,7 +5506,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
 
       while (blkIterator.hasNext()) {
       while (blkIterator.hasNext()) {
         BlockInfo blk = blkIterator.next();
         BlockInfo blk = blkIterator.next();
-        final INode inode = (INode)blockManager.getBlockCollection(blk);
+        BlockCollection bc = getBlockCollection(blk.getBlockCollectionId());
+        final INode inode = (INode)bc;
         skip++;
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {
           String src = FSDirectory.getFullPathName(inode);
           String src = FSDirectory.getFullPathName(inode);

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

@@ -89,7 +89,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
     private final LongBitFormat BITS;
     private final LongBitFormat BITS;
 
 
-    private HeaderFormat(LongBitFormat previous, int length, long min) {
+    HeaderFormat(LongBitFormat previous, int length, long min) {
       BITS = new LongBitFormat(name(), previous, length, min);
       BITS = new LongBitFormat(name(), previous, length, min);
     }
     }
 
 
@@ -244,7 +244,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
   }
 
 
   void setLastBlock(BlockInfo blk) {
   void setLastBlock(BlockInfo blk) {
-    blk.setBlockCollection(this);
+    blk.setBlockCollectionId(this.getId());
     setBlock(numBlocks() - 1, blk);
     setBlock(numBlocks() - 1, blk);
   }
   }
 
 
@@ -460,7 +460,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
     setBlocks(newlist);
     setBlocks(newlist);
     for(BlockInfo b : blocks) {
     for(BlockInfo b : blocks) {
-      b.setBlockCollection(this);
+      b.setBlockCollectionId(getId());
       short oldRepl = b.getReplication();
       short oldRepl = b.getReplication();
       short repl = getPreferredBlockReplication();
       short repl = getPreferredBlockReplication();
       if (oldRepl != repl) {
       if (oldRepl != repl) {
@@ -544,7 +544,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (blocks != null && reclaimContext.collectedBlocks != null) {
     if (blocks != null && reclaimContext.collectedBlocks != null) {
       for (BlockInfo blk : blocks) {
       for (BlockInfo blk : blocks) {
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
-        blk.setBlockCollection(null);
+        blk.setBlockCollectionId(INodeId.INVALID_INODE_ID);
       }
       }
     }
     }
     clearBlocks();
     clearBlocks();

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

@@ -37,6 +37,7 @@ public class INodeId extends SequentialNumber {
    */
    */
   public static final long LAST_RESERVED_ID = 2 << 14 - 1;
   public static final long LAST_RESERVED_ID = 2 << 14 - 1;
   public static final long ROOT_INODE_ID = LAST_RESERVED_ID + 1;
   public static final long ROOT_INODE_ID = LAST_RESERVED_ID + 1;
+  public static final long INVALID_INODE_ID = -1;
 
 
   /**
   /**
    * To check if the request id is the same as saved id. Don't check fileId
    * To check if the request id is the same as saved id. Don't check fileId

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.ipc.StandbyException;
@@ -41,6 +42,8 @@ public interface Namesystem extends RwLock, SafeMode {
 
 
   boolean isGenStampInFuture(Block block);
   boolean isGenStampInFuture(Block block);
 
 
+  BlockCollection getBlockCollection(long id);
+
   void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
   void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
 
 
   void checkOperation(OperationCategory read) throws StandbyException;
   void checkOperation(OperationCategory read) throws StandbyException;

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
+import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
 import static org.hamcrest.core.Is.is;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
@@ -51,9 +52,9 @@ public class TestBlockInfo {
   public void testIsDeleted() {
   public void testIsDeleted() {
     BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
     BlockInfo blockInfo = new BlockInfoContiguous((short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
-    blockInfo.setBlockCollection(bc);
+    blockInfo.setBlockCollectionId(1000);
     Assert.assertFalse(blockInfo.isDeleted());
     Assert.assertFalse(blockInfo.isDeleted());
-    blockInfo.setBlockCollection(null);
+    blockInfo.setBlockCollectionId(INVALID_INODE_ID);
     Assert.assertTrue(blockInfo.isDeleted());
     Assert.assertTrue(blockInfo.isDeleted());
   }
   }
 
 
@@ -71,10 +72,10 @@ public class TestBlockInfo {
 
 
   @Test
   @Test
   public void testCopyConstructor() {
   public void testCopyConstructor() {
-    BlockInfo old = new BlockInfoContiguous((short) 3);
+    BlockInfoContiguous old = new BlockInfoContiguous((short) 3);
     try {
     try {
-      BlockInfo copy = new BlockInfoContiguous((BlockInfoContiguous)old);
-      assertEquals(old.getBlockCollection(), copy.getBlockCollection());
+      BlockInfoContiguous copy = new BlockInfoContiguous(old);
+      assertEquals(old.getBlockCollectionId(), copy.getBlockCollectionId());
       assertEquals(old.getCapacity(), copy.getCapacity());
       assertEquals(old.getCapacity(), copy.getCapacity());
     } catch (Exception e) {
     } catch (Exception e) {
       Assert.fail("Copy constructor throws exception: " + e);
       Assert.fail("Copy constructor throws exception: " + e);

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -89,6 +90,7 @@ public class TestBlockManager {
 
 
   private FSNamesystem fsn;
   private FSNamesystem fsn;
   private BlockManager bm;
   private BlockManager bm;
+  private long mockINodeId;
 
 
   @Before
   @Before
   public void setupMockCluster() throws IOException {
   public void setupMockCluster() throws IOException {
@@ -97,6 +99,7 @@ public class TestBlockManager {
              "need to set a dummy value here so it assumes a multi-rack cluster");
              "need to set a dummy value here so it assumes a multi-rack cluster");
     fsn = Mockito.mock(FSNamesystem.class);
     fsn = Mockito.mock(FSNamesystem.class);
     Mockito.doReturn(true).when(fsn).hasWriteLock();
     Mockito.doReturn(true).when(fsn).hasWriteLock();
+    Mockito.doReturn(true).when(fsn).hasReadLock();
     bm = new BlockManager(fsn, conf);
     bm = new BlockManager(fsn, conf);
     final String[] racks = {
     final String[] racks = {
         "/rackA",
         "/rackA",
@@ -109,6 +112,7 @@ public class TestBlockManager {
     nodes = Arrays.asList(DFSTestUtil.toDatanodeDescriptor(storages));
     nodes = Arrays.asList(DFSTestUtil.toDatanodeDescriptor(storages));
     rackA = nodes.subList(0, 3);
     rackA = nodes.subList(0, 3);
     rackB = nodes.subList(3, 6);
     rackB = nodes.subList(3, 6);
+    mockINodeId = INodeId.ROOT_INODE_ID + 1;
   }
   }
 
 
   private void addNodes(Iterable<DatanodeDescriptor> nodesToAdd) {
   private void addNodes(Iterable<DatanodeDescriptor> nodesToAdd) {
@@ -433,9 +437,14 @@ public class TestBlockManager {
   }
   }
   
   
   private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
   private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
+    long inodeId = ++mockINodeId;
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
+    Mockito.doReturn(inodeId).when(bc).getId();
+    Mockito.doReturn(bc).when(fsn).getBlockCollection(inodeId);
     BlockInfo blockInfo = blockOnNodes(blockId, nodes);
     BlockInfo blockInfo = blockOnNodes(blockId, nodes);
 
 
+    blockInfo.setReplication((short) 3);
+    blockInfo.setBlockCollectionId(inodeId);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
     return blockInfo;
   }
   }
@@ -740,7 +749,10 @@ public class TestBlockManager {
     BlockInfo blockInfo =
     BlockInfo blockInfo =
         new BlockInfoContiguous(block, (short) 3);
         new BlockInfoContiguous(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
+    long inodeId = ++mockINodeId;
+    doReturn(inodeId).when(bc).getId();
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
+    doReturn(bc).when(fsn).getBlockCollection(inodeId);
     return blockInfo;
     return blockInfo;
   }
   }
 
 
@@ -749,7 +761,10 @@ public class TestBlockManager {
     BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
     BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 3);
     blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null);
     blockInfo.convertToBlockUnderConstruction(UNDER_CONSTRUCTION, null);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
+    long inodeId = ++mockINodeId;
+    doReturn(inodeId).when(bc).getId();
     bm.blocksMap.addBlockCollection(blockInfo, bc);
     bm.blocksMap.addBlockCollection(blockInfo, bc);
+    doReturn(bc).when(fsn).getBlockCollection(inodeId);
     return blockInfo;
     return blockInfo;
   }
   }
   
   

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java

@@ -1199,6 +1199,7 @@ public class TestReplicationPolicy {
     Namesystem mockNS = mock(Namesystem.class);
     Namesystem mockNS = mock(Namesystem.class);
     when(mockNS.isPopulatingReplQueues()).thenReturn(true);
     when(mockNS.isPopulatingReplQueues()).thenReturn(true);
     when(mockNS.hasWriteLock()).thenReturn(true);
     when(mockNS.hasWriteLock()).thenReturn(true);
+    when(mockNS.hasReadLock()).thenReturn(true);
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
 
@@ -1225,6 +1226,8 @@ public class TestReplicationPolicy {
     BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1);
     info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
     info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, null);
     BlockCollection bc = mock(BlockCollection.class);
     BlockCollection bc = mock(BlockCollection.class);
+    when(bc.getId()).thenReturn(1000L);
+    when(mockNS.getBlockCollection(1000L)).thenReturn(bc);
     bm.addBlockCollection(info, bc);
     bm.addBlockCollection(info, bc);
 
 
     // Adding this block will increase its current replication, and that will
     // Adding this block will increase its current replication, and that will
@@ -1245,6 +1248,8 @@ public class TestReplicationPolicy {
           throws IOException {
           throws IOException {
     Namesystem mockNS = mock(Namesystem.class);
     Namesystem mockNS = mock(Namesystem.class);
     when(mockNS.isPopulatingReplQueues()).thenReturn(true);
     when(mockNS.isPopulatingReplQueues()).thenReturn(true);
+    when(mockNS.hasReadLock()).thenReturn(true);
+
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
 
@@ -1266,13 +1271,14 @@ public class TestReplicationPolicy {
 
 
     final BlockInfo info = new BlockInfoContiguous(block1, (short) 1);
     final BlockInfo info = new BlockInfoContiguous(block1, (short) 1);
     final BlockCollection mbc = mock(BlockCollection.class);
     final BlockCollection mbc = mock(BlockCollection.class);
+    when(mbc.getId()).thenReturn(1000L);
     when(mbc.getLastBlock()).thenReturn(info);
     when(mbc.getLastBlock()).thenReturn(info);
     when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
     when(mbc.getPreferredBlockSize()).thenReturn(block1.getNumBytes() + 1);
     when(mbc.isUnderConstruction()).thenReturn(true);
     when(mbc.isUnderConstruction()).thenReturn(true);
     ContentSummary cs = mock(ContentSummary.class);
     ContentSummary cs = mock(ContentSummary.class);
     when(cs.getLength()).thenReturn((long)1);
     when(cs.getLength()).thenReturn((long)1);
     when(mbc.computeContentSummary(bm.getStoragePolicySuite())).thenReturn(cs);
     when(mbc.computeContentSummary(bm.getStoragePolicySuite())).thenReturn(cs);
-    info.setBlockCollection(mbc);
+    info.setBlockCollectionId(1000);
     bm.addBlockCollection(info, mbc);
     bm.addBlockCollection(info, mbc);
 
 
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
@@ -1305,6 +1311,8 @@ public class TestReplicationPolicy {
       throws IOException {
       throws IOException {
     Namesystem mockNS = mock(Namesystem.class);
     Namesystem mockNS = mock(Namesystem.class);
     when(mockNS.isPopulatingReplQueues()).thenReturn(true);
     when(mockNS.isPopulatingReplQueues()).thenReturn(true);
+    when(mockNS.hasReadLock()).thenReturn(true);
+
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
 

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

@@ -70,7 +70,7 @@ public class TestCommitBlockSynchronization {
     BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 1);
     BlockInfo blockInfo = new BlockInfoContiguous(block, (short) 1);
     blockInfo.convertToBlockUnderConstruction(
     blockInfo.convertToBlockUnderConstruction(
         HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
         HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
-    blockInfo.setBlockCollection(file);
+    blockInfo.setBlockCollectionId(file.getId());
     blockInfo.setGenerationStamp(genStamp);
     blockInfo.setGenerationStamp(genStamp);
     blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo,
     blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo,
         genStamp);
         genStamp);
@@ -88,8 +88,7 @@ public class TestCommitBlockSynchronization {
   }
   }
 
 
   private INodeFile mockFileUnderConstruction() {
   private INodeFile mockFileUnderConstruction() {
-    INodeFile file = mock(INodeFile.class);
-    return file;
+    return mock(INodeFile.class);
   }
   }
 
 
   @Test
   @Test
@@ -110,7 +109,7 @@ public class TestCommitBlockSynchronization {
 
 
     // Simulate 'completing' the block.
     // Simulate 'completing' the block.
     BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
-    completedBlockInfo.setBlockCollection(file);
+    completedBlockInfo.setBlockCollectionId(file.getId());
     completedBlockInfo.setGenerationStamp(genStamp);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)
     doReturn(completedBlockInfo).when(namesystemSpy)
         .getStoredBlock(any(Block.class));
         .getStoredBlock(any(Block.class));
@@ -182,7 +181,7 @@ public class TestCommitBlockSynchronization {
         lastBlock, genStamp, length, true, false, newTargets, null);
         lastBlock, genStamp, length, true, false, newTargets, null);
 
 
     BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
     BlockInfo completedBlockInfo = new BlockInfoContiguous(block, (short) 1);
-    completedBlockInfo.setBlockCollection(file);
+    completedBlockInfo.setBlockCollectionId(file.getId());
     completedBlockInfo.setGenerationStamp(genStamp);
     completedBlockInfo.setGenerationStamp(genStamp);
     doReturn(completedBlockInfo).when(namesystemSpy)
     doReturn(completedBlockInfo).when(namesystemSpy)
         .getStoredBlock(any(Block.class));
         .getStoredBlock(any(Block.class));

+ 39 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -80,6 +80,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -828,11 +829,19 @@ public class TestFsck {
 
 
       // decommission datanode
       // decommission datanode
       ExtendedBlock eb = DFSTestUtil.getFirstBlock(dfs, path);
       ExtendedBlock eb = DFSTestUtil.getFirstBlock(dfs, path);
-      DatanodeDescriptor dn =
-          cluster.getNameNode().getNamesystem().getBlockManager()
-              .getBlockCollection(eb.getLocalBlock()).getBlocks()[0].getDatanode(0);
-      cluster.getNameNode().getNamesystem().getBlockManager().getDatanodeManager()
-          .getDecomManager().startDecommission(dn);
+      FSNamesystem fsn = cluster.getNameNode().getNamesystem();
+      BlockManager bm = fsn.getBlockManager();
+      BlockCollection bc = null;
+      try {
+        fsn.writeLock();
+        BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
+        bc = bm.getBlockCollection(bi);
+      } finally {
+        fsn.writeUnlock();
+      }
+      DatanodeDescriptor dn = bc.getBlocks()[0]
+          .getDatanode(0);
+      bm.getDatanodeManager().getDecomManager().startDecommission(dn);
       String dnName = dn.getXferAddr();
       String dnName = dn.getXferAddr();
 
 
       // check the replica status while decommissioning
       // check the replica status while decommissioning
@@ -1387,12 +1396,19 @@ public class TestFsck {
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
 
 
       //decommission datanode
       //decommission datanode
+      FSNamesystem fsn = cluster.getNameNode().getNamesystem();
+      BlockManager bm = fsn.getBlockManager();
       ExtendedBlock eb = util.getFirstBlock(dfs, path);
       ExtendedBlock eb = util.getFirstBlock(dfs, path);
-      DatanodeDescriptor dn = cluster.getNameNode().getNamesystem()
-          .getBlockManager().getBlockCollection(eb.getLocalBlock())
-          .getBlocks()[0].getDatanode(0);
-      cluster.getNameNode().getNamesystem().getBlockManager()
-          .getDatanodeManager().getDecomManager().startDecommission(dn);
+      BlockCollection bc = null;
+      try {
+        fsn.writeLock();
+        BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
+        bc = bm.getBlockCollection(bi);
+      } finally {
+        fsn.writeUnlock();
+      }
+      DatanodeDescriptor dn = bc.getBlocks()[0].getDatanode(0);
+      bm.getDatanodeManager().getDecomManager().startDecommission(dn);
       String dnName = dn.getXferAddr();
       String dnName = dn.getXferAddr();
 
 
       //wait for decommission start
       //wait for decommission start
@@ -1595,12 +1611,20 @@ public class TestFsck {
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
 
 
       // decommission datanode
       // decommission datanode
+      FSNamesystem fsn = cluster.getNameNode().getNamesystem();
+      BlockManager bm = fsn.getBlockManager();
       ExtendedBlock eb = util.getFirstBlock(dfs, path);
       ExtendedBlock eb = util.getFirstBlock(dfs, path);
-      DatanodeDescriptor dn = cluster.getNameNode().getNamesystem()
-          .getBlockManager().getBlockCollection(eb.getLocalBlock())
-          .getBlocks()[0].getDatanode(0);
-      cluster.getNameNode().getNamesystem().getBlockManager()
-          .getDatanodeManager().getDecomManager().startDecommission(dn);
+      BlockCollection bc = null;
+      try {
+        fsn.writeLock();
+        BlockInfo bi = bm.getStoredBlock(eb.getLocalBlock());
+        bc = bm.getBlockCollection(bi);
+      } finally {
+        fsn.writeUnlock();
+      }
+      DatanodeDescriptor dn = bc.getBlocks()[0]
+          .getDatanode(0);
+      bm.getDatanodeManager().getDecomManager().startDecommission(dn);
       String dnName = dn.getXferAddr();
       String dnName = dn.getXferAddr();
 
 
       // wait for decommission start
       // wait for decommission start

+ 6 - 24
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java

@@ -17,11 +17,11 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
+import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
@@ -87,23 +87,6 @@ public class TestSnapshotBlocksMap {
     }
     }
   }
   }
 
 
-  void assertAllNull(INodeFile inode, Path path, String[] snapshots) throws Exception { 
-    Assert.assertNull(inode.getBlocks());
-    assertINodeNull(path.toString());
-    assertINodeNullInSnapshots(path, snapshots);
-  }
-
-  void assertINodeNull(String path) throws Exception {
-    Assert.assertNull(fsdir.getINode(path));
-  }
-
-  void assertINodeNullInSnapshots(Path path, String... snapshots) throws Exception {
-    for(String s : snapshots) {
-      assertINodeNull(SnapshotTestHelper.getSnapshotPath(
-          path.getParent(), s, path.getName()).toString());
-    }
-  }
-
   static INodeFile assertBlockCollection(String path, int numBlocks,
   static INodeFile assertBlockCollection(String path, int numBlocks,
      final FSDirectory dir, final BlockManager blkManager) throws Exception {
      final FSDirectory dir, final BlockManager blkManager) throws Exception {
     final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
     final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
@@ -117,8 +100,7 @@ public class TestSnapshotBlocksMap {
   static void assertBlockCollection(final BlockManager blkManager,
   static void assertBlockCollection(final BlockManager blkManager,
       final INodeFile file, final BlockInfo b) {
       final INodeFile file, final BlockInfo b) {
     Assert.assertSame(b, blkManager.getStoredBlock(b));
     Assert.assertSame(b, blkManager.getStoredBlock(b));
-    Assert.assertSame(file, blkManager.getBlockCollection(b));
-    Assert.assertSame(file, b.getBlockCollection());
+    Assert.assertEquals(file.getId(), b.getBlockCollectionId());
   }
   }
 
 
   /**
   /**
@@ -150,7 +132,7 @@ public class TestSnapshotBlocksMap {
       hdfs.delete(sub2, true);
       hdfs.delete(sub2, true);
       // The INode should have been removed from the blocksMap
       // The INode should have been removed from the blocksMap
       for(BlockInfo b : blocks) {
       for(BlockInfo b : blocks) {
-        assertNull(blockmanager.getBlockCollection(b));
+        assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
       }
       }
     }
     }
     
     
@@ -188,7 +170,7 @@ public class TestSnapshotBlocksMap {
     hdfs.delete(file0, true);
     hdfs.delete(file0, true);
     // Make sure the blocks of file0 is still in blocksMap
     // Make sure the blocks of file0 is still in blocksMap
     for(BlockInfo b : blocks0) {
     for(BlockInfo b : blocks0) {
-      assertNotNull(blockmanager.getBlockCollection(b));
+      assertNotEquals(INVALID_INODE_ID, b.getBlockCollectionId());
     }
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
     
     
@@ -202,7 +184,7 @@ public class TestSnapshotBlocksMap {
 
 
     // Make sure the first block of file0 is still in blocksMap
     // Make sure the first block of file0 is still in blocksMap
     for(BlockInfo b : blocks0) {
     for(BlockInfo b : blocks0) {
-      assertNotNull(blockmanager.getBlockCollection(b));
+      assertNotEquals(INVALID_INODE_ID, b.getBlockCollectionId());
     }
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
 
 

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
+import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
@@ -268,7 +269,7 @@ public class TestSnapshotDeletion {
     checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     // check blocks of tempFile
     // check blocks of tempFile
     for (BlockInfo b : blocks) {
     for (BlockInfo b : blocks) {
-      assertNull(blockmanager.getBlockCollection(b));
+      assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
     }
     }
     
     
     // make a change: create a new file under subsub
     // make a change: create a new file under subsub
@@ -345,7 +346,7 @@ public class TestSnapshotDeletion {
     // newFile
     // newFile
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
     for (BlockInfo b : blocks) {
     for (BlockInfo b : blocks) {
-      assertNull(blockmanager.getBlockCollection(b));
+      assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
     }
     }
     
     
     // make sure the whole subtree of sub is stored correctly in snapshot
     // make sure the whole subtree of sub is stored correctly in snapshot
@@ -508,7 +509,7 @@ public class TestSnapshotDeletion {
     // metaChangeFile's replication factor decreases
     // metaChangeFile's replication factor decreases
     checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
     checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
     for (BlockInfo b : blocks) {
     for (BlockInfo b : blocks) {
-      assertNull(blockmanager.getBlockCollection(b));
+      assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
     }
     }
     
     
     // check 1. there is no snapshot s0
     // check 1. there is no snapshot s0
@@ -839,7 +840,7 @@ public class TestSnapshotDeletion {
     assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file15_s1));
     assertFalse(hdfs.exists(file15_s1));
     for (BlockInfo b : blocks_14) {
     for (BlockInfo b : blocks_14) {
-      assertNull(blockmanager.getBlockCollection(b));
+      assertEquals(INVALID_INODE_ID, b.getBlockCollectionId());
     }
     }
 
 
     INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());
     INodeFile nodeFile13 = (INodeFile) fsdir.getINode(file13.toString());