瀏覽代碼

HDFS-9754. Avoid unnecessary getBlockCollection calls in BlockManager. Contributed by Jing Zhao.

(cherry picked from commit 972782d9568e0849484c027f27c1638ba50ec56e)
(cherry picked from commit f8c9c0ff0e2d977fc0f69bde4cdbb03371c0bac4)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java
Kihwal Lee 8 年之前
父節點
當前提交
e87bf17129

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

@@ -114,6 +114,10 @@ public abstract class BlockInfo extends Block
     this.bcId = id;
     this.bcId = id;
   }
   }
 
 
+  public void delete() {
+    setBlockCollectionId(INVALID_INODE_ID);
+  }
+
   public boolean isDeleted() {
   public boolean isDeleted() {
     return bcId == INVALID_INODE_ID;
     return bcId == INVALID_INODE_ID;
   }
   }
@@ -353,6 +357,12 @@ public abstract class BlockInfo extends Block
     return getBlockUCState().equals(BlockUCState.COMPLETE);
     return getBlockUCState().equals(BlockUCState.COMPLETE);
   }
   }
 
 
+  public final boolean isCompleteOrCommitted() {
+    final BlockUCState state = getBlockUCState();
+    return state.equals(BlockUCState.COMPLETE) ||
+        state.equals(BlockUCState.COMMITTED);
+  }
+
   /**
   /**
    * Add/Update the under construction feature.
    * Add/Update the under construction feature.
    */
    */

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

@@ -737,19 +737,19 @@ public class BlockManager implements BlockStatsMXBean {
     if(lastBlock.isComplete())
     if(lastBlock.isComplete())
       return false; // already completed (e.g. by syncBlock)
       return false; // already completed (e.g. by syncBlock)
     
     
-    final boolean b = commitBlock(lastBlock, commitBlock);
+    final boolean committed = commitBlock(lastBlock, commitBlock);
 
 
     // Count replicas on decommissioning nodes, as these will not be
     // Count replicas on decommissioning nodes, as these will not be
     // decommissioned unless recovery/completing last block has finished
     // decommissioned unless recovery/completing last block has finished
     NumberReplicas numReplicas = countNodes(lastBlock);
     NumberReplicas numReplicas = countNodes(lastBlock);
     if (numReplicas.liveReplicas() + numReplicas.decommissioning() >=
     if (numReplicas.liveReplicas() + numReplicas.decommissioning() >=
         minReplication) {
         minReplication) {
-      if (b) {
-        addExpectedReplicasToPending(lastBlock, bc);
+      if (committed) {
+        addExpectedReplicasToPending(lastBlock);
       }
       }
       completeBlock(lastBlock, iip, false);
       completeBlock(lastBlock, iip, false);
     }
     }
-    return b;
+    return committed;
   }
   }
 
 
   /**
   /**
@@ -757,24 +757,20 @@ public class BlockManager implements BlockStatsMXBean {
    * pendingReplications in order to keep ReplicationMonitor from scheduling
    * pendingReplications in order to keep ReplicationMonitor from scheduling
    * the block.
    * the block.
    */
    */
-  public void addExpectedReplicasToPending(BlockInfo blk, BlockCollection bc) {
-    addExpectedReplicasToPending(blk);
-  }
-
-  private void addExpectedReplicasToPending(BlockInfo lastBlock) {
+  public void addExpectedReplicasToPending(BlockInfo blk) {
     DatanodeStorageInfo[] expectedStorages =
     DatanodeStorageInfo[] expectedStorages =
-        lastBlock.getUnderConstructionFeature().getExpectedStorageLocations();
-    if (expectedStorages.length - lastBlock.numNodes() > 0) {
+        blk.getUnderConstructionFeature().getExpectedStorageLocations();
+    if (expectedStorages.length - blk.numNodes() > 0) {
       ArrayList<DatanodeDescriptor> pendingNodes =
       ArrayList<DatanodeDescriptor> pendingNodes =
           new ArrayList<DatanodeDescriptor>();
           new ArrayList<DatanodeDescriptor>();
       for (DatanodeStorageInfo storage : expectedStorages) {
       for (DatanodeStorageInfo storage : expectedStorages) {
         DatanodeDescriptor dnd = storage.getDatanodeDescriptor();
         DatanodeDescriptor dnd = storage.getDatanodeDescriptor();
-        if (lastBlock.findStorageInfo(dnd) == null) {
+        if (blk.findStorageInfo(dnd) == null) {
           pendingNodes.add(dnd);
           pendingNodes.add(dnd);
         }
         }
+        pendingReplications.increment(blk,
+            pendingNodes.toArray(new DatanodeDescriptor[pendingNodes.size()]));
       }
       }
-      pendingReplications.increment(lastBlock,
-          pendingNodes.toArray(new DatanodeDescriptor[pendingNodes.size()]));
     }
     }
   }
   }
 
 
@@ -1586,11 +1582,8 @@ public class BlockManager implements BlockStatsMXBean {
   }
   }
 
 
   private ReplicationWork scheduleReplication(BlockInfo block, int priority) {
   private ReplicationWork scheduleReplication(BlockInfo block, int priority) {
-    // block should belong to a file
-    BlockCollection bc = getBlockCollection(block);
-    // abandoned block or block reopened for append
-    if (bc == null
-        || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
+    // skip abandoned block or block reopened for append
+    if (block.isDeleted() || !block.isCompleteOrCommitted()) {
       // remove from neededReplications
       // remove from neededReplications
       neededReplications.remove(block, priority);
       neededReplications.remove(block, priority);
       return null;
       return null;
@@ -1629,6 +1622,7 @@ public class BlockManager implements BlockStatsMXBean {
     } else {
     } else {
       additionalReplRequired = 1; // Needed on a new rack
       additionalReplRequired = 1; // Needed on a new rack
     }
     }
+    final BlockCollection bc = getBlockCollection(block);
     return new ReplicationWork(block, bc, srcNode, containingNodes,
     return new ReplicationWork(block, bc, srcNode, containingNodes,
         liveReplicaNodes, additionalReplRequired, priority);
         liveReplicaNodes, additionalReplRequired, priority);
   }
   }
@@ -1637,11 +1631,8 @@ public class BlockManager implements BlockStatsMXBean {
     BlockInfo block = rw.getBlock();
     BlockInfo block = rw.getBlock();
     int priority = rw.getPriority();
     int priority = rw.getPriority();
     // Recheck since global lock was released
     // Recheck since global lock was released
-    // block should belong to a file
-    BlockCollection bc = getBlockCollection(block);
-    // abandoned block or block reopened for append
-    if (bc == null
-        || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
+    // skip abandoned block or block reopened for append
+    if (block.isDeleted() || !block.isCompleteOrCommitted()) {
       neededReplications.remove(block, priority);
       neededReplications.remove(block, priority);
       rw.resetTargets();
       rw.resetTargets();
       return false;
       return false;
@@ -2685,8 +2676,6 @@ 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 = getBlockCollection(storedBlock);
-    assert bc != null : "Block must belong to a file";
 
 
     // add block to the datanode
     // add block to the datanode
     AddBlockResult result = storageInfo.addBlock(storedBlock);
     AddBlockResult result = storageInfo.addBlock(storedBlock);
@@ -2722,7 +2711,7 @@ public class BlockManager implements BlockStatsMXBean {
 
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
         numLiveReplicas >= minReplication) {
         numLiveReplicas >= minReplication) {
-      addExpectedReplicasToPending(storedBlock, bc);
+      addExpectedReplicasToPending(storedBlock);
       completeBlock(storedBlock, null, false);
       completeBlock(storedBlock, null, 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
@@ -2733,8 +2722,8 @@ public class BlockManager implements BlockStatsMXBean {
       namesystem.incrementSafeBlockCount(numCurrentReplica);
       namesystem.incrementSafeBlockCount(numCurrentReplica);
     }
     }
     
     
-    // if file is under construction, then done for now
-    if (bc.isUnderConstruction()) {
+    // if block is still under construction, then done for now
+    if (!storedBlock.isCompleteOrCommitted()) {
       return storedBlock;
       return storedBlock;
     }
     }
 
 
@@ -3148,8 +3137,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 = getBlockCollection(storedBlock);
-      if (bc != null) {
+      if (!storedBlock.isDeleted()) {
         namesystem.decrementSafeBlockCount(storedBlock);
         namesystem.decrementSafeBlockCount(storedBlock);
         updateNeededReplications(storedBlock, -1, 0);
         updateNeededReplications(storedBlock, -1, 0);
       }
       }

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

@@ -114,7 +114,7 @@ class BlocksMap {
     if (blockInfo == null)
     if (blockInfo == null)
       return;
       return;
 
 
-    blockInfo.setBlockCollectionId(INodeId.INVALID_INODE_ID);
+    assert blockInfo.getBlockCollectionId() == 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);
       if (dn != null) {
       if (dn != null) {

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Set;
 
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
@@ -178,6 +179,7 @@ final class FSDirTruncateOp {
           "Should be the same block.";
           "Should be the same block.";
       if (oldBlock.getBlockId() != tBlk.getBlockId()
       if (oldBlock.getBlockId() != tBlk.getBlockId()
           && !file.isBlockInLatestSnapshot(oldBlock)) {
           && !file.isBlockInLatestSnapshot(oldBlock)) {
+        oldBlock.delete();
         fsd.getBlockManager().removeBlockFromMap(oldBlock);
         fsd.getBlockManager().removeBlockFromMap(oldBlock);
       }
       }
     }
     }
@@ -288,9 +290,9 @@ final class FSDirTruncateOp {
 
 
     verifyQuotaForTruncate(fsn, iip, file, newLength, delta);
     verifyQuotaForTruncate(fsn, iip, file, newLength, delta);
 
 
-    long remainingLength =
-        file.collectBlocksBeyondMax(newLength, collectedBlocks);
-    file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks);
+    Set<BlockInfo> toRetain = file.getSnapshotBlocksToRetain(latestSnapshot);
+    long remainingLength = file.collectBlocksBeyondMax(newLength,
+        collectedBlocks, toRetain);
     file.setModificationTime(mtime);
     file.setModificationTime(mtime);
     // return whether on a block boundary
     // return whether on a block boundary
     return (remainingLength - newLength) == 0;
     return (remainingLength - newLength) == 0;

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

@@ -3308,7 +3308,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final BlockInfo b = blocks[i];
       final BlockInfo b = blocks[i];
       if (b != null && b.getBlockUCState() == BlockUCState.COMMITTED) {
       if (b != null && b.getBlockUCState() == BlockUCState.COMMITTED) {
         // b is COMMITTED but not yet COMPLETE, add it to pending replication.
         // b is COMMITTED but not yet COMPLETE, add it to pending replication.
-        blockManager.addExpectedReplicasToPending(b, pendingFile);
+        blockManager.addExpectedReplicasToPending(b);
       }
       }
     }
     }
   }
   }
@@ -5129,9 +5129,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
   }
 
 
-  @Override
-  public void checkSuperuserPrivilege()
-      throws AccessControlException {
+  void checkSuperuserPrivilege() throws AccessControlException {
     if (isPermissionEnabled) {
     if (isPermissionEnabled) {
       FSPermissionChecker pc = getPermissionChecker();
       FSPermissionChecker pc = getPermissionChecker();
       pc.checkSuperuserPrivilege();
       pc.checkSuperuserPrivilege();

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

@@ -1040,14 +1040,10 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
      */
      */
     public void addDeleteBlock(BlockInfo toDelete) {
     public void addDeleteBlock(BlockInfo toDelete) {
       assert toDelete != null : "toDelete is null";
       assert toDelete != null : "toDelete is null";
+      toDelete.delete();
       toDeleteList.add(toDelete);
       toDeleteList.add(toDelete);
     }
     }
 
 
-    public void removeDeleteBlock(BlockInfo block) {
-      assert block != null : "block is null";
-      toDeleteList.remove(block);
-    }
-
     public void addUpdateReplicationFactor(BlockInfo block, short targetRepl) {
     public void addUpdateReplicationFactor(BlockInfo block, short targetRepl) {
       toUpdateReplicationInfo.add(
       toUpdateReplicationInfo.add(
           new UpdatedReplicationInfo(targetRepl, block));
           new UpdatedReplicationInfo(targetRepl, block));

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

@@ -25,6 +25,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Set;
 import java.util.Set;
@@ -290,12 +291,13 @@ public class INodeFile extends INodeWithAdditionalFields
       return null;
       return null;
     }
     }
 
 
-    BlockInfo ucBlock = blocks[size_1];
+    BlockInfo lastBlock = blocks[size_1];
     //copy to a new list
     //copy to a new list
     BlockInfo[] newlist = new BlockInfo[size_1];
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     System.arraycopy(blocks, 0, newlist, 0, size_1);
     setBlocks(newlist);
     setBlocks(newlist);
-    return ucBlock;
+    lastBlock.delete();
+    return lastBlock;
   }
   }
 
 
   /* End of Under-Construction Feature */
   /* End of Under-Construction Feature */
@@ -571,7 +573,6 @@ 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.setBlockCollectionId(INodeId.INVALID_INODE_ID);
       }
       }
     }
     }
     clearBlocks();
     clearBlocks();
@@ -801,7 +802,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return sum of sizes of the remained blocks
    * @return sum of sizes of the remained blocks
    */
    */
   public long collectBlocksBeyondMax(final long max,
   public long collectBlocksBeyondMax(final long max,
-      final BlocksMapUpdateInfo collectedBlocks) {
+      final BlocksMapUpdateInfo collectedBlocks, Set<BlockInfo> toRetain) {
     final BlockInfo[] oldBlocks = getBlocks();
     final BlockInfo[] oldBlocks = getBlocks();
     if (oldBlocks == null) {
     if (oldBlocks == null) {
       return 0;
       return 0;
@@ -823,7 +824,10 @@ public class INodeFile extends INodeWithAdditionalFields
     // collect the blocks beyond max
     // collect the blocks beyond max
     if (collectedBlocks != null) {
     if (collectedBlocks != null) {
       for(; n < oldBlocks.length; n++) {
       for(; n < oldBlocks.length; n++) {
-        collectedBlocks.addDeleteBlock(oldBlocks[n]);
+        final BlockInfo del = oldBlocks[n];
+        if (toRetain == null || !toRetain.contains(del)) {
+          collectedBlocks.addDeleteBlock(del);
+        }
       }
       }
     }
     }
     return size;
     return size;
@@ -914,22 +918,18 @@ public class INodeFile extends INodeWithAdditionalFields
   }
   }
 
 
   /** Exclude blocks collected for deletion that belong to a snapshot. */
   /** Exclude blocks collected for deletion that belong to a snapshot. */
-  void excludeSnapshotBlocks(int snapshotId,
-                             BlocksMapUpdateInfo collectedBlocks) {
-    if(collectedBlocks == null || collectedBlocks.getToDeleteList().isEmpty())
-      return;
+  Set<BlockInfo> getSnapshotBlocksToRetain(int snapshotId) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
-    if(sf == null)
-      return;
-    BlockInfo[] snapshotBlocks =
-        getDiffs().findEarlierSnapshotBlocks(snapshotId);
-    if(snapshotBlocks == null)
-      return;
-    List<BlockInfo> toDelete = collectedBlocks.getToDeleteList();
-    for(BlockInfo blk : snapshotBlocks) {
-      if(toDelete.contains(blk))
-        collectedBlocks.removeDeleteBlock(blk);
+    if(sf == null) {
+      return null;
+    }
+    BlockInfo[] snapshotBlocks = getDiffs().findEarlierSnapshotBlocks(snapshotId);
+    if(snapshotBlocks == null) {
+      return null;
     }
     }
+    Set<BlockInfo> toRetain = new HashSet<>(snapshotBlocks.length);
+    Collections.addAll(toRetain, snapshotBlocks);
+    return toRetain;
   }
   }
 
 
   /**
   /**

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

@@ -24,7 +24,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 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;
-import org.apache.hadoop.security.AccessControlException;
 
 
 /** Namesystem operations. */
 /** Namesystem operations. */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
@@ -32,9 +31,6 @@ public interface Namesystem extends RwLock, SafeMode {
   /** Is this name system running? */
   /** Is this name system running? */
   boolean isRunning();
   boolean isRunning();
 
 
-  /** Check if the user has superuser privilege. */
-  void checkSuperuserPrivilege() throws AccessControlException;
-
   /** @return the block pool ID */
   /** @return the block pool ID */
   String getBlockPoolId();
   String getBlockPoolId();
 
 
@@ -54,4 +50,4 @@ public interface Namesystem extends RwLock, SafeMode {
 
 
   CacheManager getCacheManager();
   CacheManager getCacheManager();
   HAContext getHAContext();
   HAContext getHAContext();
-}
+}

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

@@ -220,7 +220,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     FileDiff last = diffs.getLast();
     FileDiff last = diffs.getLast();
     BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
     BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
     if(snapshotBlocks == null)
     if(snapshotBlocks == null)
-      file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
+      file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks(), null);
     else
     else
       file.collectBlocksBeyondSnapshot(snapshotBlocks,
       file.collectBlocksBeyondSnapshot(snapshotBlocks,
                                        reclaimContext.collectedBlocks());
                                        reclaimContext.collectedBlocks());

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMetadataConsistency.java

@@ -98,6 +98,7 @@ public class TestNameNodeMetadataConsistency {
     BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager
     BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager
         ().getStoredBlock(block.getLocalBlock());
         ().getStoredBlock(block.getLocalBlock());
     cluster.getNameNode().getNamesystem().writeLock();
     cluster.getNameNode().getNamesystem().writeLock();
+    bInfo.delete();
     cluster.getNameNode().getNamesystem().getBlockManager()
     cluster.getNameNode().getNamesystem().getBlockManager()
         .removeBlock(bInfo);
         .removeBlock(bInfo);
     cluster.getNameNode().getNamesystem().writeUnlock();
     cluster.getNameNode().getNamesystem().writeUnlock();
@@ -162,6 +163,7 @@ public class TestNameNodeMetadataConsistency {
     BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager
     BlockInfo bInfo = cluster.getNameNode().getNamesystem().getBlockManager
         ().getStoredBlock(block.getLocalBlock());
         ().getStoredBlock(block.getLocalBlock());
     cluster.getNameNode().getNamesystem().writeLock();
     cluster.getNameNode().getNamesystem().writeLock();
+    bInfo.delete();
     cluster.getNameNode().getNamesystem().getBlockManager()
     cluster.getNameNode().getNamesystem().getBlockManager()
         .removeBlock(bInfo);
         .removeBlock(bInfo);
     cluster.getNameNode().getNamesystem().writeUnlock();
     cluster.getNameNode().getNamesystem().writeUnlock();
@@ -184,4 +186,4 @@ public class TestNameNodeMetadataConsistency {
       e.printStackTrace();
       e.printStackTrace();
     }
     }
   }
   }
-}
+}