|
@@ -89,7 +89,6 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
|
|
import org.apache.hadoop.hdfs.util.LightWeightHashSet;
|
|
|
-import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
|
|
@@ -219,7 +218,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* Maps a StorageID to the set of blocks that are "extra" for this
|
|
|
* DataNode. We'll eventually remove these extras.
|
|
|
*/
|
|
|
- public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
|
|
|
+ public final Map<String, LightWeightHashSet<BlockInfo>> excessReplicateMap =
|
|
|
new HashMap<>();
|
|
|
|
|
|
/**
|
|
@@ -1421,11 +1420,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
|
|
|
- int requiredReplication, numEffectiveReplicas;
|
|
|
- List<DatanodeDescriptor> containingNodes;
|
|
|
- BlockCollection bc;
|
|
|
- int additionalReplRequired;
|
|
|
-
|
|
|
int scheduledWork = 0;
|
|
|
List<BlockRecoveryWork> recovWork = new LinkedList<>();
|
|
|
|
|
@@ -1786,7 +1780,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
|
|
|
for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
|
- LightWeightLinkedSet<BlockInfo> excessBlocks =
|
|
|
+ LightWeightHashSet<BlockInfo> excessBlocks =
|
|
|
excessReplicateMap.get(node.getDatanodeUuid());
|
|
|
int countableReplica = storage.getState() == State.NORMAL ? 1 : 0;
|
|
|
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
|
|
@@ -3090,7 +3084,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
postponeBlock(block);
|
|
|
return;
|
|
|
}
|
|
|
- LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
|
+ LightWeightHashSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
|
cur.getDatanodeUuid());
|
|
|
if (excessBlocks == null || !excessBlocks.contains(block)) {
|
|
|
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
|
@@ -3297,10 +3291,10 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
- LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
|
+ LightWeightHashSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
|
dn.getDatanodeUuid());
|
|
|
if (excessBlocks == null) {
|
|
|
- excessBlocks = new LightWeightLinkedSet<>();
|
|
|
+ excessBlocks = new LightWeightHashSet<>();
|
|
|
excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks);
|
|
|
}
|
|
|
if (excessBlocks.add(storedBlock)) {
|
|
@@ -3364,7 +3358,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// We've removed a block from a node, so it's definitely no longer
|
|
|
// in "excess" there.
|
|
|
//
|
|
|
- LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
|
+ LightWeightHashSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
|
node.getDatanodeUuid());
|
|
|
if (excessBlocks != null) {
|
|
|
if (excessBlocks.remove(storedBlock)) {
|
|
@@ -3581,7 +3575,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
} else if (node.isDecommissioned()) {
|
|
|
decommissioned++;
|
|
|
} else {
|
|
|
- LightWeightLinkedSet<BlockInfo> blocksExcess = excessReplicateMap.get(
|
|
|
+ LightWeightHashSet<BlockInfo> blocksExcess = excessReplicateMap.get(
|
|
|
node.getDatanodeUuid());
|
|
|
if (blocksExcess != null && blocksExcess.contains(b)) {
|
|
|
excess++;
|
|
@@ -3988,7 +3982,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
private void removeFromExcessReplicateMap(Block block) {
|
|
|
for (DatanodeStorageInfo info : blocksMap.getStorages(block)) {
|
|
|
String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
|
|
|
- LightWeightLinkedSet<BlockInfo> excessReplicas = excessReplicateMap.get(uuid);
|
|
|
+ LightWeightHashSet<BlockInfo> excessReplicas =
|
|
|
+ excessReplicateMap.get(uuid);
|
|
|
if (excessReplicas != null) {
|
|
|
if (excessReplicas.remove(block)) {
|
|
|
excessBlocksCount.decrementAndGet();
|