|
@@ -140,7 +140,7 @@ public class BlockManager {
|
|
|
private final long replicationRecheckInterval;
|
|
|
|
|
|
/**
|
|
|
- * Mapping: Block -> { INode, datanodes, self ref }
|
|
|
+ * Mapping: Block -> { BlockCollection, datanodes, self ref }
|
|
|
* Updated only in response to client-sent information.
|
|
|
*/
|
|
|
final BlocksMap blocksMap;
|
|
@@ -190,7 +190,7 @@ public class BlockManager {
|
|
|
public final short minReplication;
|
|
|
/** Default number of replicas */
|
|
|
public final int defaultReplication;
|
|
|
- /** The maximum number of entries returned by getCorruptInodes() */
|
|
|
+ /** value returned by MAX_CORRUPT_FILES_RETURNED */
|
|
|
final int maxCorruptFilesReturned;
|
|
|
|
|
|
/** variable to enable check for enough racks */
|
|
@@ -382,7 +382,7 @@ public class BlockManager {
|
|
|
numReplicas.decommissionedReplicas();
|
|
|
|
|
|
if (block instanceof BlockInfo) {
|
|
|
- String fileName = ((BlockInfo)block).getINode().getName();
|
|
|
+ String fileName = ((BlockInfo)block).getBlockCollection().getName();
|
|
|
out.print(fileName + ": ");
|
|
|
}
|
|
|
// l: == live:, d: == decommissioned c: == corrupt e: == excess
|
|
@@ -452,17 +452,17 @@ public class BlockManager {
|
|
|
* Commit the last block of the file and mark it as complete if it has
|
|
|
* meets the minimum replication requirement
|
|
|
*
|
|
|
- * @param fileINode file inode
|
|
|
+ * @param bc block collection
|
|
|
* @param commitBlock - contains client reported block length and generation
|
|
|
* @return true if the last block is changed to committed state.
|
|
|
* @throws IOException if the block does not have at least a minimal number
|
|
|
* of replicas reported from data-nodes.
|
|
|
*/
|
|
|
- public boolean commitOrCompleteLastBlock(MutableBlockCollection fileINode,
|
|
|
+ public boolean commitOrCompleteLastBlock(MutableBlockCollection bc,
|
|
|
Block commitBlock) throws IOException {
|
|
|
if(commitBlock == null)
|
|
|
return false; // not committing, this is a block allocation retry
|
|
|
- BlockInfo lastBlock = fileINode.getLastBlock();
|
|
|
+ BlockInfo lastBlock = bc.getLastBlock();
|
|
|
if(lastBlock == null)
|
|
|
return false; // no blocks in file yet
|
|
|
if(lastBlock.isComplete())
|
|
@@ -470,22 +470,22 @@ public class BlockManager {
|
|
|
|
|
|
final boolean b = commitBlock((BlockInfoUnderConstruction)lastBlock, commitBlock);
|
|
|
if(countNodes(lastBlock).liveReplicas() >= minReplication)
|
|
|
- completeBlock(fileINode, fileINode.numBlocks()-1, false);
|
|
|
+ completeBlock(bc, bc.numBlocks()-1, false);
|
|
|
return b;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Convert a specified block of the file to a complete block.
|
|
|
- * @param fileINode file
|
|
|
+ * @param bc file
|
|
|
* @param blkIndex block index in the file
|
|
|
* @throws IOException if the block does not have at least a minimal number
|
|
|
* of replicas reported from data-nodes.
|
|
|
*/
|
|
|
- private BlockInfo completeBlock(final MutableBlockCollection fileINode,
|
|
|
+ private BlockInfo completeBlock(final MutableBlockCollection bc,
|
|
|
final int blkIndex, boolean force) throws IOException {
|
|
|
if(blkIndex < 0)
|
|
|
return null;
|
|
|
- BlockInfo curBlock = fileINode.getBlocks()[blkIndex];
|
|
|
+ BlockInfo curBlock = bc.getBlocks()[blkIndex];
|
|
|
if(curBlock.isComplete())
|
|
|
return curBlock;
|
|
|
BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)curBlock;
|
|
@@ -498,7 +498,7 @@ public class BlockManager {
|
|
|
"Cannot complete block: block has not been COMMITTED by the client");
|
|
|
BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
|
|
|
// replace penultimate block in file
|
|
|
- fileINode.setBlock(blkIndex, completeBlock);
|
|
|
+ bc.setBlock(blkIndex, completeBlock);
|
|
|
|
|
|
// Since safe-mode only counts complete blocks, and we now have
|
|
|
// one more complete block, we need to adjust the total up, and
|
|
@@ -514,12 +514,12 @@ public class BlockManager {
|
|
|
return blocksMap.replaceBlock(completeBlock);
|
|
|
}
|
|
|
|
|
|
- private BlockInfo completeBlock(final MutableBlockCollection fileINode,
|
|
|
+ private BlockInfo completeBlock(final MutableBlockCollection bc,
|
|
|
final BlockInfo block, boolean force) throws IOException {
|
|
|
- BlockInfo[] fileBlocks = fileINode.getBlocks();
|
|
|
+ BlockInfo[] fileBlocks = bc.getBlocks();
|
|
|
for(int idx = 0; idx < fileBlocks.length; idx++)
|
|
|
if(fileBlocks[idx] == block) {
|
|
|
- return completeBlock(fileINode, idx, force);
|
|
|
+ return completeBlock(bc, idx, force);
|
|
|
}
|
|
|
return block;
|
|
|
}
|
|
@@ -529,10 +529,10 @@ public class BlockManager {
|
|
|
* regardless of whether enough replicas are present. This is necessary
|
|
|
* when tailing edit logs as a Standby.
|
|
|
*/
|
|
|
- public BlockInfo forceCompleteBlock(final MutableBlockCollection fileINode,
|
|
|
+ public BlockInfo forceCompleteBlock(final MutableBlockCollection bc,
|
|
|
final BlockInfoUnderConstruction block) throws IOException {
|
|
|
block.commitBlock(block);
|
|
|
- return completeBlock(fileINode, block, true);
|
|
|
+ return completeBlock(bc, block, true);
|
|
|
}
|
|
|
|
|
|
|
|
@@ -546,14 +546,14 @@ public class BlockManager {
|
|
|
* The methods returns null if there is no partial block at the end.
|
|
|
* The client is supposed to allocate a new block with the next call.
|
|
|
*
|
|
|
- * @param fileINode file
|
|
|
+ * @param bc file
|
|
|
* @return the last block locations if the block is partial or null otherwise
|
|
|
*/
|
|
|
public LocatedBlock convertLastBlockToUnderConstruction(
|
|
|
- MutableBlockCollection fileINode) throws IOException {
|
|
|
- BlockInfo oldBlock = fileINode.getLastBlock();
|
|
|
+ MutableBlockCollection bc) throws IOException {
|
|
|
+ BlockInfo oldBlock = bc.getLastBlock();
|
|
|
if(oldBlock == null ||
|
|
|
- fileINode.getPreferredBlockSize() == oldBlock.getNumBytes())
|
|
|
+ bc.getPreferredBlockSize() == oldBlock.getNumBytes())
|
|
|
return null;
|
|
|
assert oldBlock == getStoredBlock(oldBlock) :
|
|
|
"last block of the file is not in blocksMap";
|
|
@@ -561,7 +561,7 @@ public class BlockManager {
|
|
|
DatanodeDescriptor[] targets = getNodes(oldBlock);
|
|
|
|
|
|
BlockInfoUnderConstruction ucBlock =
|
|
|
- fileINode.setLastBlock(oldBlock, targets);
|
|
|
+ bc.setLastBlock(oldBlock, targets);
|
|
|
blocksMap.replaceBlock(ucBlock);
|
|
|
|
|
|
// Remove block from replication queue.
|
|
@@ -581,7 +581,7 @@ public class BlockManager {
|
|
|
// always decrement total blocks
|
|
|
-1);
|
|
|
|
|
|
- final long fileLength = fileINode.computeContentSummary().getLength();
|
|
|
+ final long fileLength = bc.computeContentSummary().getLength();
|
|
|
final long pos = fileLength - ucBlock.getNumBytes();
|
|
|
return createLocatedBlock(ucBlock, pos, AccessMode.WRITE);
|
|
|
}
|
|
@@ -921,8 +921,8 @@ public class BlockManager {
|
|
|
" does not exist. ");
|
|
|
}
|
|
|
|
|
|
- BlockCollection inode = storedBlock.getINode();
|
|
|
- if (inode == null) {
|
|
|
+ BlockCollection bc = storedBlock.getBlockCollection();
|
|
|
+ if (bc == null) {
|
|
|
NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
|
|
|
"block " + storedBlock +
|
|
|
" could not be marked as corrupt as it" +
|
|
@@ -936,7 +936,7 @@ public class BlockManager {
|
|
|
|
|
|
// Add this replica to corruptReplicas Map
|
|
|
corruptReplicas.addToCorruptReplicasMap(storedBlock, node, reason);
|
|
|
- if (countNodes(storedBlock).liveReplicas() >= inode.getReplication()) {
|
|
|
+ if (countNodes(storedBlock).liveReplicas() >= bc.getReplication()) {
|
|
|
// the block is over-replicated so invalidate the replicas immediately
|
|
|
invalidateBlock(storedBlock, node);
|
|
|
} else if (namesystem.isPopulatingReplQueues()) {
|
|
@@ -1049,7 +1049,7 @@ public class BlockManager {
|
|
|
int requiredReplication, numEffectiveReplicas;
|
|
|
List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
|
|
|
DatanodeDescriptor srcNode;
|
|
|
- BlockCollection fileINode = null;
|
|
|
+ BlockCollection bc = null;
|
|
|
int additionalReplRequired;
|
|
|
|
|
|
int scheduledWork = 0;
|
|
@@ -1061,15 +1061,15 @@ public class BlockManager {
|
|
|
for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
|
|
|
for (Block block : blocksToReplicate.get(priority)) {
|
|
|
// block should belong to a file
|
|
|
- fileINode = blocksMap.getINode(block);
|
|
|
+ bc = blocksMap.getBlockCollection(block);
|
|
|
// abandoned block or block reopened for append
|
|
|
- if(fileINode == null || fileINode instanceof MutableBlockCollection) {
|
|
|
+ if(bc == null || bc instanceof MutableBlockCollection) {
|
|
|
neededReplications.remove(block, priority); // remove from neededReplications
|
|
|
neededReplications.decrementReplicationIndex(priority);
|
|
|
continue;
|
|
|
}
|
|
|
|
|
|
- requiredReplication = fileINode.getReplication();
|
|
|
+ requiredReplication = bc.getReplication();
|
|
|
|
|
|
// get a source data-node
|
|
|
containingNodes = new ArrayList<DatanodeDescriptor>();
|
|
@@ -1105,7 +1105,7 @@ public class BlockManager {
|
|
|
} else {
|
|
|
additionalReplRequired = 1; // Needed on a new rack
|
|
|
}
|
|
|
- work.add(new ReplicationWork(block, fileINode, srcNode,
|
|
|
+ work.add(new ReplicationWork(block, bc, srcNode,
|
|
|
containingNodes, liveReplicaNodes, additionalReplRequired,
|
|
|
priority));
|
|
|
}
|
|
@@ -1127,8 +1127,8 @@ public class BlockManager {
|
|
|
|
|
|
// choose replication targets: NOT HOLDING THE GLOBAL LOCK
|
|
|
// It is costly to extract the filename for which chooseTargets is called,
|
|
|
- // so for now we pass in the Inode itself.
|
|
|
- rw.targets = blockplacement.chooseTarget(rw.fileINode,
|
|
|
+ // so for now we pass in the block collection itself.
|
|
|
+ rw.targets = blockplacement.chooseTarget(rw.bc,
|
|
|
rw.additionalReplRequired, rw.srcNode, rw.liveReplicaNodes,
|
|
|
excludedNodes, rw.block.getNumBytes());
|
|
|
}
|
|
@@ -1147,15 +1147,15 @@ public class BlockManager {
|
|
|
int priority = rw.priority;
|
|
|
// Recheck since global lock was released
|
|
|
// block should belong to a file
|
|
|
- fileINode = blocksMap.getINode(block);
|
|
|
+ bc = blocksMap.getBlockCollection(block);
|
|
|
// abandoned block or block reopened for append
|
|
|
- if(fileINode == null || fileINode instanceof MutableBlockCollection) {
|
|
|
+ if(bc == null || bc instanceof MutableBlockCollection) {
|
|
|
neededReplications.remove(block, priority); // remove from neededReplications
|
|
|
rw.targets = null;
|
|
|
neededReplications.decrementReplicationIndex(priority);
|
|
|
continue;
|
|
|
}
|
|
|
- requiredReplication = fileINode.getReplication();
|
|
|
+ requiredReplication = bc.getReplication();
|
|
|
|
|
|
// do not schedule more if enough replicas is already pending
|
|
|
NumberReplicas numReplicas = countNodes(block);
|
|
@@ -1802,8 +1802,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
case COMPLETE:
|
|
|
case COMMITTED:
|
|
|
if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
|
|
|
- return new BlockToMarkCorrupt(new BlockInfo(iblk, storedBlock
|
|
|
- .getINode().getReplication()),
|
|
|
+ return new BlockToMarkCorrupt(storedBlock,
|
|
|
"block is " + ucState + " and reported genstamp " +
|
|
|
iblk.getGenerationStamp() + " does not match " +
|
|
|
"genstamp in block map " + storedBlock.getGenerationStamp());
|
|
@@ -1823,8 +1822,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
if (!storedBlock.isComplete()) {
|
|
|
return null; // not corrupt
|
|
|
} else if (storedBlock.getGenerationStamp() != iblk.getGenerationStamp()) {
|
|
|
- return new BlockToMarkCorrupt(new BlockInfo(iblk, storedBlock
|
|
|
- .getINode().getReplication()),
|
|
|
+ return new BlockToMarkCorrupt(storedBlock,
|
|
|
"reported " + reportedState + " replica with genstamp " +
|
|
|
iblk.getGenerationStamp() + " does not match COMPLETE block's " +
|
|
|
"genstamp in block map " + storedBlock.getGenerationStamp());
|
|
@@ -1916,7 +1914,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
int numCurrentReplica = countLiveNodes(storedBlock);
|
|
|
if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
|
|
|
&& numCurrentReplica >= minReplication) {
|
|
|
- completeBlock((MutableBlockCollection)storedBlock.getINode(), storedBlock, false);
|
|
|
+ completeBlock((MutableBlockCollection)storedBlock.getBlockCollection(), storedBlock, false);
|
|
|
} else if (storedBlock.isComplete()) {
|
|
|
// check whether safe replication is reached for the block
|
|
|
// only complete blocks are counted towards that.
|
|
@@ -1944,7 +1942,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
} else {
|
|
|
storedBlock = block;
|
|
|
}
|
|
|
- if (storedBlock == null || storedBlock.getINode() == null) {
|
|
|
+ if (storedBlock == null || storedBlock.getBlockCollection() == null) {
|
|
|
// If this block does not belong to anyfile, then we are done.
|
|
|
NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
|
|
|
+ node + " size " + block.getNumBytes()
|
|
@@ -1954,8 +1952,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return block;
|
|
|
}
|
|
|
assert storedBlock != null : "Block must be stored by now";
|
|
|
- BlockCollection fileINode = storedBlock.getINode();
|
|
|
- assert fileINode != null : "Block must belong to a file";
|
|
|
+ BlockCollection bc = storedBlock.getBlockCollection();
|
|
|
+ assert bc != null : "Block must belong to a file";
|
|
|
|
|
|
// add block to the datanode
|
|
|
boolean added = node.addBlock(storedBlock);
|
|
@@ -1981,7 +1979,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
|
|
|
if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
|
|
|
numLiveReplicas >= minReplication) {
|
|
|
- storedBlock = completeBlock((MutableBlockCollection)fileINode, storedBlock, false);
|
|
|
+ storedBlock = completeBlock((MutableBlockCollection)bc, storedBlock, false);
|
|
|
} else if (storedBlock.isComplete()) {
|
|
|
// check whether safe replication is reached for the block
|
|
|
// only complete blocks are counted towards that
|
|
@@ -1992,7 +1990,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
}
|
|
|
|
|
|
// if file is under construction, then done for now
|
|
|
- if (fileINode instanceof MutableBlockCollection) {
|
|
|
+ if (bc instanceof MutableBlockCollection) {
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
@@ -2002,7 +2000,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
}
|
|
|
|
|
|
// handle underReplication/overReplication
|
|
|
- short fileReplication = fileINode.getReplication();
|
|
|
+ short fileReplication = bc.getReplication();
|
|
|
if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
|
|
|
neededReplications.remove(storedBlock, numCurrentReplica,
|
|
|
num.decommissionedReplicas(), fileReplication);
|
|
@@ -2129,8 +2127,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
* what happened with it.
|
|
|
*/
|
|
|
private MisReplicationResult processMisReplicatedBlock(BlockInfo block) {
|
|
|
- BlockCollection fileINode = block.getINode();
|
|
|
- if (fileINode == null) {
|
|
|
+ BlockCollection bc = block.getBlockCollection();
|
|
|
+ if (bc == null) {
|
|
|
// block does not belong to any file
|
|
|
addToInvalidates(block);
|
|
|
return MisReplicationResult.INVALID;
|
|
@@ -2141,7 +2139,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return MisReplicationResult.UNDER_CONSTRUCTION;
|
|
|
}
|
|
|
// calculate current replication
|
|
|
- short expectedReplication = fileINode.getReplication();
|
|
|
+ short expectedReplication = bc.getReplication();
|
|
|
NumberReplicas num = countNodes(block);
|
|
|
int numCurrentReplica = num.liveReplicas();
|
|
|
// add to under-replicated queue if need to be
|
|
@@ -2258,7 +2256,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
BlockPlacementPolicy replicator) {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
// first form a rack to datanodes map and
|
|
|
- BlockCollection inode = getINode(b);
|
|
|
+ BlockCollection bc = getBlockCollection(b);
|
|
|
final Map<String, List<DatanodeDescriptor>> rackMap
|
|
|
= new HashMap<String, List<DatanodeDescriptor>>();
|
|
|
for(final Iterator<DatanodeDescriptor> iter = nonExcess.iterator();
|
|
@@ -2298,7 +2296,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
|| (addedNode != null && !priSet.contains(addedNode))) ) {
|
|
|
cur = delNodeHint;
|
|
|
} else { // regular excessive replica removal
|
|
|
- cur = replicator.chooseReplicaToDelete(inode, b, replication,
|
|
|
+ cur = replicator.chooseReplicaToDelete(bc, b, replication,
|
|
|
priSet, remains);
|
|
|
}
|
|
|
firstOne = false;
|
|
@@ -2379,8 +2377,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
// necessary. In that case, put block on a possibly-will-
|
|
|
// be-replicated list.
|
|
|
//
|
|
|
- BlockCollection fileINode = blocksMap.getINode(block);
|
|
|
- if (fileINode != null) {
|
|
|
+ BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
|
+ if (bc != null) {
|
|
|
namesystem.decrementSafeBlockCount(block);
|
|
|
updateNeededReplications(block, -1, 0);
|
|
|
}
|
|
@@ -2611,7 +2609,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
NumberReplicas num) {
|
|
|
int curReplicas = num.liveReplicas();
|
|
|
int curExpectedReplicas = getReplication(block);
|
|
|
- BlockCollection fileINode = blocksMap.getINode(block);
|
|
|
+ BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
|
Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
|
|
|
StringBuilder nodeList = new StringBuilder();
|
|
|
while (nodeIter.hasNext()) {
|
|
@@ -2624,7 +2622,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
+ ", corrupt replicas: " + num.corruptReplicas()
|
|
|
+ ", decommissioned replicas: " + num.decommissionedReplicas()
|
|
|
+ ", excess replicas: " + num.excessReplicas()
|
|
|
- + ", Is Open File: " + (fileINode instanceof MutableBlockCollection)
|
|
|
+ + ", Is Open File: " + (bc instanceof MutableBlockCollection)
|
|
|
+ ", Datanodes having this block: " + nodeList + ", Current Datanode: "
|
|
|
+ srcNode + ", Is current datanode decommissioning: "
|
|
|
+ srcNode.isDecommissionInProgress());
|
|
@@ -2639,8 +2637,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
final Iterator<? extends Block> it = srcNode.getBlockIterator();
|
|
|
while(it.hasNext()) {
|
|
|
final Block block = it.next();
|
|
|
- BlockCollection fileINode = blocksMap.getINode(block);
|
|
|
- short expectedReplication = fileINode.getReplication();
|
|
|
+ BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
|
+ short expectedReplication = bc.getReplication();
|
|
|
NumberReplicas num = countNodes(block);
|
|
|
int numCurrentReplica = num.liveReplicas();
|
|
|
if (numCurrentReplica > expectedReplication) {
|
|
@@ -2662,9 +2660,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
final Iterator<? extends Block> it = srcNode.getBlockIterator();
|
|
|
while(it.hasNext()) {
|
|
|
final Block block = it.next();
|
|
|
- BlockCollection fileINode = blocksMap.getINode(block);
|
|
|
+ BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
|
|
|
|
- if (fileINode != null) {
|
|
|
+ if (bc != null) {
|
|
|
NumberReplicas num = countNodes(block);
|
|
|
int curReplicas = num.liveReplicas();
|
|
|
int curExpectedReplicas = getReplication(block);
|
|
@@ -2679,7 +2677,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
|
|
|
decommissionOnlyReplicas++;
|
|
|
}
|
|
|
- if (fileINode instanceof MutableBlockCollection) {
|
|
|
+ if (bc instanceof MutableBlockCollection) {
|
|
|
underReplicatedInOpenFiles++;
|
|
|
}
|
|
|
}
|
|
@@ -2782,11 +2780,11 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
|
|
|
/* get replication factor of a block */
|
|
|
private int getReplication(Block block) {
|
|
|
- BlockCollection fileINode = blocksMap.getINode(block);
|
|
|
- if (fileINode == null) { // block does not belong to any file
|
|
|
+ BlockCollection bc = blocksMap.getBlockCollection(block);
|
|
|
+ if (bc == null) { // block does not belong to any file
|
|
|
return 0;
|
|
|
}
|
|
|
- return fileINode.getReplication();
|
|
|
+ return bc.getReplication();
|
|
|
}
|
|
|
|
|
|
|
|
@@ -2858,12 +2856,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
return this.neededReplications.getCorruptBlockSize();
|
|
|
}
|
|
|
|
|
|
- public BlockInfo addINode(BlockInfo block, BlockCollection iNode) {
|
|
|
- return blocksMap.addINode(block, iNode);
|
|
|
+ public BlockInfo addBlockCollection(BlockInfo block, BlockCollection bc) {
|
|
|
+ return blocksMap.addBlockCollection(block, bc);
|
|
|
}
|
|
|
|
|
|
- public BlockCollection getINode(Block b) {
|
|
|
- return blocksMap.getINode(b);
|
|
|
+ public BlockCollection getBlockCollection(Block b) {
|
|
|
+ return blocksMap.getBlockCollection(b);
|
|
|
}
|
|
|
|
|
|
/** @return an iterator of the datanodes. */
|
|
@@ -3002,7 +3000,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
private static class ReplicationWork {
|
|
|
|
|
|
private Block block;
|
|
|
- private BlockCollection fileINode;
|
|
|
+ private BlockCollection bc;
|
|
|
|
|
|
private DatanodeDescriptor srcNode;
|
|
|
private List<DatanodeDescriptor> containingNodes;
|
|
@@ -3013,14 +3011,14 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|
|
private int priority;
|
|
|
|
|
|
public ReplicationWork(Block block,
|
|
|
- BlockCollection fileINode,
|
|
|
+ BlockCollection bc,
|
|
|
DatanodeDescriptor srcNode,
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
|
List<DatanodeDescriptor> liveReplicaNodes,
|
|
|
int additionalReplRequired,
|
|
|
int priority) {
|
|
|
this.block = block;
|
|
|
- this.fileINode = fileINode;
|
|
|
+ this.bc = bc;
|
|
|
this.srcNode = srcNode;
|
|
|
this.containingNodes = containingNodes;
|
|
|
this.liveReplicaNodes = liveReplicaNodes;
|