|
@@ -541,7 +541,7 @@ public class BlockManager {
|
|
|
// source node returned is not used
|
|
|
chooseSourceDatanodes(getStoredBlock(block), containingNodes,
|
|
|
containingLiveReplicasNodes, numReplicas,
|
|
|
- new LinkedList<Short>(), 1, UnderReplicatedBlocks.LEVEL);
|
|
|
+ new LinkedList<Short>(), UnderReplicatedBlocks.LEVEL);
|
|
|
|
|
|
// containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are
|
|
|
// not included in the numReplicas.liveReplicas() count
|
|
@@ -1389,7 +1389,7 @@ public class BlockManager {
|
|
|
int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
|
|
|
int requiredReplication, numEffectiveReplicas;
|
|
|
List<DatanodeDescriptor> containingNodes;
|
|
|
- BlockCollection bc = null;
|
|
|
+ BlockCollection bc;
|
|
|
int additionalReplRequired;
|
|
|
|
|
|
int scheduledWork = 0;
|
|
@@ -1417,13 +1417,10 @@ public class BlockManager {
|
|
|
containingNodes = new ArrayList<>();
|
|
|
List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
|
|
|
NumberReplicas numReplicas = new NumberReplicas();
|
|
|
- List<Short> missingBlockIndices = new LinkedList<>();
|
|
|
- DatanodeDescriptor[] srcNodes;
|
|
|
- int numSourceNodes = bc.isStriped() ?
|
|
|
- HdfsConstants.NUM_DATA_BLOCKS : 1;
|
|
|
- srcNodes = chooseSourceDatanodes(
|
|
|
- block, containingNodes, liveReplicaNodes, numReplicas,
|
|
|
- missingBlockIndices, numSourceNodes, priority);
|
|
|
+ List<Short> liveBlockIndices = new ArrayList<>();
|
|
|
+ final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
|
|
|
+ containingNodes, liveReplicaNodes, numReplicas,
|
|
|
+ liveBlockIndices, priority);
|
|
|
if(srcNodes == null || srcNodes.length == 0) {
|
|
|
// block can not be replicated from any node
|
|
|
LOG.debug("Block " + block + " cannot be recovered " +
|
|
@@ -1455,15 +1452,14 @@ public class BlockManager {
|
|
|
} else {
|
|
|
additionalReplRequired = 1; // Needed on a new rack
|
|
|
}
|
|
|
- if (bc.isStriped()) {
|
|
|
+ if (block.isStriped()) {
|
|
|
+ short[] indices = new short[liveBlockIndices.size()];
|
|
|
+ for (int i = 0 ; i < liveBlockIndices.size(); i++) {
|
|
|
+ indices[i] = liveBlockIndices.get(i);
|
|
|
+ }
|
|
|
ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes,
|
|
|
containingNodes, liveReplicaNodes, additionalReplRequired,
|
|
|
- priority);
|
|
|
- short[] missingBlockArray = new short[missingBlockIndices.size()];
|
|
|
- for (int i = 0 ; i < missingBlockIndices.size(); i++) {
|
|
|
- missingBlockArray[i] = missingBlockIndices.get(i);
|
|
|
- }
|
|
|
- ecw.setMissingBlockIndices(missingBlockArray);
|
|
|
+ priority, indices);
|
|
|
recovWork.add(ecw);
|
|
|
} else {
|
|
|
recovWork.add(new ReplicationWork(block, bc, srcNodes,
|
|
@@ -1543,15 +1539,14 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
// Add block to the to be replicated list
|
|
|
- if (bc.isStriped()) {
|
|
|
+ if (block.isStriped()) {
|
|
|
assert rw instanceof ErasureCodingWork;
|
|
|
assert rw.targets.length > 0;
|
|
|
rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
|
|
|
new ExtendedBlock(namesystem.getBlockPoolId(), block),
|
|
|
rw.srcNodes, rw.targets,
|
|
|
- ((ErasureCodingWork)rw).getMissingBlockIndicies());
|
|
|
- }
|
|
|
- else {
|
|
|
+ ((ErasureCodingWork) rw).liveBlockIndicies);
|
|
|
+ } else {
|
|
|
rw.srcNodes[0].addBlockToBeReplicated(block, targets);
|
|
|
}
|
|
|
scheduledWork++;
|
|
@@ -1581,9 +1576,9 @@ public class BlockManager {
|
|
|
DatanodeStorageInfo[] targets = rw.targets;
|
|
|
if (targets != null && targets.length != 0) {
|
|
|
StringBuilder targetList = new StringBuilder("datanode(s)");
|
|
|
- for (int k = 0; k < targets.length; k++) {
|
|
|
+ for (DatanodeStorageInfo target : targets) {
|
|
|
targetList.append(' ');
|
|
|
- targetList.append(targets[k].getDatanodeDescriptor());
|
|
|
+ targetList.append(target.getDatanodeDescriptor());
|
|
|
}
|
|
|
blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
|
|
|
rw.block, targetList);
|
|
@@ -1694,11 +1689,8 @@ public class BlockManager {
|
|
|
* @param numReplicas NumberReplicas instance to be initialized with the
|
|
|
* counts of live, corrupt, excess, and decommissioned
|
|
|
* replicas of the given block.
|
|
|
- * @param missingBlockIndices List to be populated with indices of missing
|
|
|
- * blocks in a striped block group or missing
|
|
|
- * replicas of a replicated block
|
|
|
- * @param numSourceNodes integer specifying the number of source nodes to
|
|
|
- * choose
|
|
|
+ * @param liveBlockIndices List to be populated with indices of healthy
|
|
|
+ * blocks in a striped block group
|
|
|
* @param priority integer representing replication priority of the given
|
|
|
* block
|
|
|
* @return the array of DatanodeDescriptor of the chosen nodes from which to
|
|
@@ -1709,24 +1701,20 @@ public class BlockManager {
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
|
List<DatanodeStorageInfo> nodesContainingLiveReplicas,
|
|
|
NumberReplicas numReplicas,
|
|
|
- List<Short> missingBlockIndices, int numSourceNodes, int priority) {
|
|
|
+ List<Short> liveBlockIndices, int priority) {
|
|
|
containingNodes.clear();
|
|
|
nodesContainingLiveReplicas.clear();
|
|
|
- LinkedList<DatanodeDescriptor> srcNodes = new LinkedList<>();
|
|
|
+ List<DatanodeDescriptor> srcNodes = new ArrayList<>();
|
|
|
int live = 0;
|
|
|
int decommissioned = 0;
|
|
|
int decommissioning = 0;
|
|
|
int corrupt = 0;
|
|
|
int excess = 0;
|
|
|
- missingBlockIndices.clear();
|
|
|
- Set<Short> healthyIndices = new HashSet<>();
|
|
|
+ liveBlockIndices.clear();
|
|
|
+ final boolean isStriped = block.isStriped();
|
|
|
|
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
|
|
|
- for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
- if (block.isStriped()) {
|
|
|
- healthyIndices.add((short) ((BlockInfoStriped) block).
|
|
|
- getStorageBlockIndex(storage));
|
|
|
- }
|
|
|
+ for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
|
LightWeightLinkedSet<BlockInfo> excessBlocks =
|
|
|
excessReplicateMap.get(node.getDatanodeUuid());
|
|
@@ -1765,27 +1753,19 @@ public class BlockManager {
|
|
|
if(node.isDecommissioned())
|
|
|
continue;
|
|
|
|
|
|
- // We got this far, current node is a reasonable choice
|
|
|
- if(srcNodes.size() < numSourceNodes) {
|
|
|
+ if(isStriped || srcNodes.isEmpty()) {
|
|
|
srcNodes.add(node);
|
|
|
+ if (isStriped) {
|
|
|
+ liveBlockIndices.add((short) ((BlockInfoStriped) block).
|
|
|
+ getStorageBlockIndex(storage));
|
|
|
+ }
|
|
|
continue;
|
|
|
}
|
|
|
- // switch to a different node randomly
|
|
|
+ // for replicated block, switch to a different node randomly
|
|
|
// this to prevent from deterministically selecting the same node even
|
|
|
// if the node failed to replicate the block on previous iterations
|
|
|
- if(ThreadLocalRandom.current().nextBoolean()) {
|
|
|
- int pos = ThreadLocalRandom.current().nextInt(numSourceNodes);
|
|
|
- if(!srcNodes.get(pos).isDecommissionInProgress()) {
|
|
|
- srcNodes.set(pos, node);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- if (block.isStriped()) {
|
|
|
- for (short i = 0; i < HdfsConstants.NUM_DATA_BLOCKS +
|
|
|
- HdfsConstants.NUM_PARITY_BLOCKS; i++) {
|
|
|
- if (!healthyIndices.contains(i)) {
|
|
|
- missingBlockIndices.add(i);
|
|
|
- }
|
|
|
+ if (!isStriped && ThreadLocalRandom.current().nextBoolean()) {
|
|
|
+ srcNodes.set(0, node);
|
|
|
}
|
|
|
}
|
|
|
if(numReplicas != null)
|
|
@@ -3889,25 +3869,25 @@ public class BlockManager {
|
|
|
* to represent a task to recover a block through replication or erasure
|
|
|
* coding. Recovery is done by transferring data from srcNodes to targets
|
|
|
*/
|
|
|
- private static class BlockRecoveryWork {
|
|
|
- protected final BlockInfo block;
|
|
|
- protected final BlockCollection bc;
|
|
|
+ private abstract static class BlockRecoveryWork {
|
|
|
+ final BlockInfo block;
|
|
|
+ final BlockCollection bc;
|
|
|
|
|
|
/**
|
|
|
* An erasure coding recovery task has multiple source nodes.
|
|
|
* A replication task only has 1 source node, stored on top of the array
|
|
|
*/
|
|
|
- protected final DatanodeDescriptor[] srcNodes;
|
|
|
+ final DatanodeDescriptor[] srcNodes;
|
|
|
/** Nodes containing the block; avoid them in choosing new targets */
|
|
|
- protected final List<DatanodeDescriptor> containingNodes;
|
|
|
+ final List<DatanodeDescriptor> containingNodes;
|
|
|
/** Required by {@link BlockPlacementPolicy#chooseTarget} */
|
|
|
- protected final List<DatanodeStorageInfo> liveReplicaStorages;
|
|
|
- protected final int additionalReplRequired;
|
|
|
+ final List<DatanodeStorageInfo> liveReplicaStorages;
|
|
|
+ final int additionalReplRequired;
|
|
|
|
|
|
- protected DatanodeStorageInfo[] targets;
|
|
|
- protected final int priority;
|
|
|
+ DatanodeStorageInfo[] targets;
|
|
|
+ final int priority;
|
|
|
|
|
|
- public BlockRecoveryWork(BlockInfo block,
|
|
|
+ BlockRecoveryWork(BlockInfo block,
|
|
|
BlockCollection bc,
|
|
|
DatanodeDescriptor[] srcNodes,
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
@@ -3924,15 +3904,13 @@ public class BlockManager {
|
|
|
this.targets = null;
|
|
|
}
|
|
|
|
|
|
- protected void chooseTargets(BlockPlacementPolicy blockplacement,
|
|
|
+ abstract void chooseTargets(BlockPlacementPolicy blockplacement,
|
|
|
BlockStoragePolicySuite storagePolicySuite,
|
|
|
- Set<Node> excludedNodes) {
|
|
|
- }
|
|
|
+ Set<Node> excludedNodes);
|
|
|
}
|
|
|
|
|
|
private static class ReplicationWork extends BlockRecoveryWork {
|
|
|
-
|
|
|
- public ReplicationWork(BlockInfo block,
|
|
|
+ ReplicationWork(BlockInfo block,
|
|
|
BlockCollection bc,
|
|
|
DatanodeDescriptor[] srcNodes,
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
@@ -3944,7 +3922,8 @@ public class BlockManager {
|
|
|
LOG.debug("Creating a ReplicationWork to recover " + block);
|
|
|
}
|
|
|
|
|
|
- protected void chooseTargets(BlockPlacementPolicy blockplacement,
|
|
|
+ @Override
|
|
|
+ void chooseTargets(BlockPlacementPolicy blockplacement,
|
|
|
BlockStoragePolicySuite storagePolicySuite,
|
|
|
Set<Node> excludedNodes) {
|
|
|
assert srcNodes.length > 0
|
|
@@ -3961,30 +3940,23 @@ public class BlockManager {
|
|
|
}
|
|
|
|
|
|
private static class ErasureCodingWork extends BlockRecoveryWork {
|
|
|
+ final short[] liveBlockIndicies;
|
|
|
|
|
|
- private short[] missingBlockIndicies = null;
|
|
|
-
|
|
|
- public ErasureCodingWork(BlockInfo block,
|
|
|
+ ErasureCodingWork(BlockInfo block,
|
|
|
BlockCollection bc,
|
|
|
DatanodeDescriptor[] srcNodes,
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
|
List<DatanodeStorageInfo> liveReplicaStorages,
|
|
|
int additionalReplRequired,
|
|
|
- int priority) {
|
|
|
+ int priority, short[] liveBlockIndicies) {
|
|
|
super(block, bc, srcNodes, containingNodes,
|
|
|
liveReplicaStorages, additionalReplRequired, priority);
|
|
|
+ this.liveBlockIndicies = liveBlockIndicies;
|
|
|
LOG.debug("Creating an ErasureCodingWork to recover " + block);
|
|
|
}
|
|
|
|
|
|
- public short[] getMissingBlockIndicies() {
|
|
|
- return missingBlockIndicies;
|
|
|
- }
|
|
|
-
|
|
|
- public void setMissingBlockIndices(short[] missingBlockIndicies) {
|
|
|
- this.missingBlockIndicies = missingBlockIndicies;
|
|
|
- }
|
|
|
-
|
|
|
- protected void chooseTargets(BlockPlacementPolicy blockplacement,
|
|
|
+ @Override
|
|
|
+ void chooseTargets(BlockPlacementPolicy blockplacement,
|
|
|
BlockStoragePolicySuite storagePolicySuite,
|
|
|
Set<Node> excludedNodes) {
|
|
|
try {
|