|
@@ -743,7 +743,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
if (hasMinStorage(lastBlock)) {
|
|
|
if (committed) {
|
|
|
- addExpectedReplicasToPending(lastBlock, bc);
|
|
|
+ addExpectedReplicasToPending(lastBlock);
|
|
|
}
|
|
|
completeBlock(lastBlock, false);
|
|
|
}
|
|
@@ -755,26 +755,21 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* pendingReplications in order to keep ReplicationMonitor from scheduling
|
|
|
* the block.
|
|
|
*/
|
|
|
- public void addExpectedReplicasToPending(BlockInfo blk, BlockCollection bc) {
|
|
|
- if (!bc.isStriped()) {
|
|
|
- addExpectedReplicasToPending(blk);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void addExpectedReplicasToPending(BlockInfo lastBlock) {
|
|
|
- DatanodeStorageInfo[] expectedStorages =
|
|
|
- lastBlock.getUnderConstructionFeature().getExpectedStorageLocations();
|
|
|
- if (expectedStorages.length - lastBlock.numNodes() > 0) {
|
|
|
- ArrayList<DatanodeDescriptor> pendingNodes =
|
|
|
- new ArrayList<DatanodeDescriptor>();
|
|
|
- for (DatanodeStorageInfo storage : expectedStorages) {
|
|
|
- DatanodeDescriptor dnd = storage.getDatanodeDescriptor();
|
|
|
- if (lastBlock.findStorageInfo(dnd) == null) {
|
|
|
- pendingNodes.add(dnd);
|
|
|
+ public void addExpectedReplicasToPending(BlockInfo blk) {
|
|
|
+ if (!blk.isStriped()) {
|
|
|
+ DatanodeStorageInfo[] expectedStorages =
|
|
|
+ blk.getUnderConstructionFeature().getExpectedStorageLocations();
|
|
|
+ if (expectedStorages.length - blk.numNodes() > 0) {
|
|
|
+ ArrayList<DatanodeDescriptor> pendingNodes = new ArrayList<>();
|
|
|
+ for (DatanodeStorageInfo storage : expectedStorages) {
|
|
|
+ DatanodeDescriptor dnd = storage.getDatanodeDescriptor();
|
|
|
+ if (blk.findStorageInfo(dnd) == null) {
|
|
|
+ pendingNodes.add(dnd);
|
|
|
+ }
|
|
|
}
|
|
|
+ pendingReplications.increment(blk,
|
|
|
+ pendingNodes.toArray(new DatanodeDescriptor[pendingNodes.size()]));
|
|
|
}
|
|
|
- pendingReplications.increment(lastBlock,
|
|
|
- pendingNodes.toArray(new DatanodeDescriptor[pendingNodes.size()]));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -962,13 +957,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature();
|
|
|
if (blk.isStriped()) {
|
|
|
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
|
|
- final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
|
|
|
+ final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
|
|
|
blk);
|
|
|
return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
|
|
|
false);
|
|
|
} else {
|
|
|
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
|
|
- final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
|
|
|
+ final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
|
|
|
blk);
|
|
|
return newLocatedBlock(eb, storages, pos, false);
|
|
|
}
|
|
@@ -1011,7 +1006,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
" numNodes: " + numNodes +
|
|
|
" numCorrupt: " + numCorruptNodes +
|
|
|
" numCorruptRepls: " + numCorruptReplicas;
|
|
|
- final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
|
|
|
+ final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(), blk);
|
|
|
return blockIndices == null ?
|
|
|
newLocatedBlock(eb, machines, pos, isCorrupt) :
|
|
|
newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
|
|
@@ -1578,11 +1573,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
private BlockReconstructionWork scheduleReconstruction(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
|
|
|
neededReplications.remove(block, priority);
|
|
|
return null;
|
|
@@ -1626,6 +1618,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
additionalReplRequired = 1; // Needed on a new rack
|
|
|
}
|
|
|
|
|
|
+ final BlockCollection bc = getBlockCollection(block);
|
|
|
if (block.isStriped()) {
|
|
|
if (pendingNum > 0) {
|
|
|
// Wait the previous reconstruction to finish.
|
|
@@ -1649,11 +1642,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
BlockInfo block = rw.getBlock();
|
|
|
int priority = rw.getPriority();
|
|
|
// 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);
|
|
|
rw.resetTargets();
|
|
|
return false;
|
|
@@ -1688,23 +1678,12 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
assert rw.getTargets().length > 0;
|
|
|
assert pendingNum == 0 : "Should wait the previous reconstruction"
|
|
|
+ " to finish";
|
|
|
- String src = getBlockCollection(block).getName();
|
|
|
- ErasureCodingPolicy ecPolicy = null;
|
|
|
- try {
|
|
|
- ecPolicy = namesystem.getErasureCodingPolicyForPath(src);
|
|
|
- } catch (IOException e) {
|
|
|
- blockLog
|
|
|
- .warn("Failed to get EC policy for the file {} ", src);
|
|
|
- }
|
|
|
- if (ecPolicy == null) {
|
|
|
- blockLog.warn("No erasure coding policy found for the file {}. "
|
|
|
- + "So cannot proceed for reconstruction", src);
|
|
|
- // TODO: we may have to revisit later for what we can do better to
|
|
|
- // handle this case.
|
|
|
- return false;
|
|
|
- }
|
|
|
+ final ErasureCodingPolicy ecPolicy =
|
|
|
+ ((BlockInfoStriped) block).getErasureCodingPolicy();
|
|
|
+ assert ecPolicy != null;
|
|
|
+
|
|
|
rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
|
|
|
- new ExtendedBlock(namesystem.getBlockPoolId(), block),
|
|
|
+ new ExtendedBlock(getBlockPoolId(), block),
|
|
|
rw.getSrcNodes(), rw.getTargets(),
|
|
|
((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy);
|
|
|
} else {
|
|
@@ -2870,8 +2849,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// it will happen in next block report otherwise.
|
|
|
return block;
|
|
|
}
|
|
|
- BlockCollection bc = getBlockCollection(storedBlock);
|
|
|
- assert bc != null : "Block must belong to a file";
|
|
|
|
|
|
// add block to the datanode
|
|
|
AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
|
|
@@ -2907,7 +2884,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
|
|
|
hasMinStorage(storedBlock, numLiveReplicas)) {
|
|
|
- addExpectedReplicasToPending(storedBlock, bc);
|
|
|
+ addExpectedReplicasToPending(storedBlock);
|
|
|
completeBlock(storedBlock, false);
|
|
|
} else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
|
|
|
// check whether safe replication is reached for the block
|
|
@@ -2918,8 +2895,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
bmSafeMode.incrementSafeBlockCount(numCurrentReplica, storedBlock);
|
|
|
}
|
|
|
|
|
|
- // 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;
|
|
|
}
|
|
|
|
|
@@ -3444,8 +3421,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// necessary. In that case, put block on a possibly-will-
|
|
|
// be-replicated list.
|
|
|
//
|
|
|
- BlockCollection bc = getBlockCollection(storedBlock);
|
|
|
- if (bc != null) {
|
|
|
+ if (!storedBlock.isDeleted()) {
|
|
|
bmSafeMode.decrementSafeBlockCount(storedBlock);
|
|
|
updateNeededReplications(storedBlock, -1, 0);
|
|
|
}
|