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

(cherry picked from commit 972782d956)
(cherry picked from commit f8c9c0ff0e)

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
This commit is contained in:
Kihwal Lee 2017-06-20 12:10:43 -05:00
parent 1f9255672d
commit e87bf17129
10 changed files with 62 additions and 70 deletions

View File

@ -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.
*/ */

View File

@ -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) { if (committed) {
addExpectedReplicasToPending(lastBlock, bc); 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) { public void addExpectedReplicasToPending(BlockInfo blk) {
addExpectedReplicasToPending(blk);
}
private void addExpectedReplicasToPending(BlockInfo lastBlock) {
DatanodeStorageInfo[] expectedStorages = DatanodeStorageInfo[] expectedStorages =
lastBlock.getUnderConstructionFeature().getExpectedStorageLocations(); blk.getUnderConstructionFeature().getExpectedStorageLocations();
if (expectedStorages.length - lastBlock.numNodes() > 0) { 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 // skip abandoned block or block reopened for append
BlockCollection bc = getBlockCollection(block); if (block.isDeleted() || !block.isCompleteOrCommitted()) {
// abandoned block or block reopened for append
if (bc == null
|| (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
// 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 // skip abandoned block or block reopened for append
BlockCollection bc = getBlockCollection(block); if (block.isDeleted() || !block.isCompleteOrCommitted()) {
// abandoned block or block reopened for append
if (bc == null
|| (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
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 block is still under construction, then done for now
if (bc.isUnderConstruction()) { 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 (!storedBlock.isDeleted()) {
if (bc != null) {
namesystem.decrementSafeBlockCount(storedBlock); namesystem.decrementSafeBlockCount(storedBlock);
updateNeededReplications(storedBlock, -1, 0); updateNeededReplications(storedBlock, -1, 0);
} }

View File

@ -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) {

View File

@ -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 = Set<BlockInfo> toRetain = file.getSnapshotBlocksToRetain(latestSnapshot);
file.collectBlocksBeyondMax(newLength, collectedBlocks); long remainingLength = file.collectBlocksBeyondMax(newLength,
file.excludeSnapshotBlocks(latestSnapshot, collectedBlocks); 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;

View File

@ -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 void checkSuperuserPrivilege() throws AccessControlException {
public void checkSuperuserPrivilege()
throws AccessControlException {
if (isPermissionEnabled) { if (isPermissionEnabled) {
FSPermissionChecker pc = getPermissionChecker(); FSPermissionChecker pc = getPermissionChecker();
pc.checkSuperuserPrivilege(); pc.checkSuperuserPrivilege();

View File

@ -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));

View File

@ -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, Set<BlockInfo> getSnapshotBlocksToRetain(int snapshotId) {
BlocksMapUpdateInfo collectedBlocks) {
if(collectedBlocks == null || collectedBlocks.getToDeleteList().isEmpty())
return;
FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
if(sf == null) if(sf == null) {
return; return null;
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);
} }
BlockInfo[] snapshotBlocks = getDiffs().findEarlierSnapshotBlocks(snapshotId);
if(snapshotBlocks == null) {
return null;
}
Set<BlockInfo> toRetain = new HashSet<>(snapshotBlocks.length);
Collections.addAll(toRetain, snapshotBlocks);
return toRetain;
} }
/** /**

View File

@ -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();
} }

View File

@ -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());

View File

@ -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();
} }
} }
} }