|
@@ -71,8 +71,10 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier.AccessMode;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped.StorageAndBlockIndex;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas.StoredReplicaState;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
@@ -1816,76 +1818,62 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
containingNodes.clear();
|
|
|
nodesContainingLiveReplicas.clear();
|
|
|
List<DatanodeDescriptor> srcNodes = new ArrayList<>();
|
|
|
- int live = 0;
|
|
|
- int readonly = 0;
|
|
|
- int decommissioned = 0;
|
|
|
- int decommissioning = 0;
|
|
|
- int corrupt = 0;
|
|
|
- int excess = 0;
|
|
|
liveBlockIndices.clear();
|
|
|
final boolean isStriped = block.isStriped();
|
|
|
|
|
|
- Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
|
|
|
+ BitSet bitSet = isStriped ?
|
|
|
+ new BitSet(((BlockInfoStriped) block).getTotalBlockNum()) : null;
|
|
|
for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
|
|
|
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
|
- LightWeightHashSet<BlockInfo> excessBlocks =
|
|
|
- excessReplicateMap.get(node.getDatanodeUuid());
|
|
|
- int countableReplica = storage.getState() == State.NORMAL ? 1 : 0;
|
|
|
- if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
|
|
|
- corrupt += countableReplica;
|
|
|
- else if (node.isDecommissionInProgress()) {
|
|
|
- decommissioning += countableReplica;
|
|
|
- } else if (node.isDecommissioned()) {
|
|
|
- decommissioned += countableReplica;
|
|
|
- } else if (excessBlocks != null && excessBlocks.contains(block)) {
|
|
|
- excess += countableReplica;
|
|
|
- } else {
|
|
|
+ final StoredReplicaState state = checkReplicaOnStorage(numReplicas, block,
|
|
|
+ storage, corruptReplicas.getNodes(block), false);
|
|
|
+ if (state == StoredReplicaState.LIVE) {
|
|
|
nodesContainingLiveReplicas.add(storage);
|
|
|
- live += countableReplica;
|
|
|
- }
|
|
|
- if (storage.getState() == State.READ_ONLY_SHARED) {
|
|
|
- readonly++;
|
|
|
}
|
|
|
containingNodes.add(node);
|
|
|
- // Check if this replica is corrupt
|
|
|
- // If so, do not select the node as src node
|
|
|
- if ((nodesCorrupt != null) && nodesCorrupt.contains(node))
|
|
|
+
|
|
|
+ // do not select corrupted replica as src. also do not select the block
|
|
|
+ // that is already in excess map
|
|
|
+ if (state == StoredReplicaState.CORRUPT ||
|
|
|
+ state == StoredReplicaState.EXCESS) {
|
|
|
continue;
|
|
|
+ }
|
|
|
+
|
|
|
if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY
|
|
|
&& !node.isDecommissionInProgress()
|
|
|
- && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
|
|
|
- {
|
|
|
+ && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams) {
|
|
|
continue; // already reached replication limit
|
|
|
}
|
|
|
- if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit)
|
|
|
- {
|
|
|
+ if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit) {
|
|
|
continue;
|
|
|
}
|
|
|
- // the block must not be scheduled for removal on srcNode
|
|
|
- if(excessBlocks != null && excessBlocks.contains(block))
|
|
|
- continue;
|
|
|
// never use already decommissioned nodes
|
|
|
- if(node.isDecommissioned())
|
|
|
+ if (node.isDecommissioned()) {
|
|
|
continue;
|
|
|
+ }
|
|
|
|
|
|
if(isStriped || srcNodes.isEmpty()) {
|
|
|
srcNodes.add(node);
|
|
|
if (isStriped) {
|
|
|
- liveBlockIndices.add(((BlockInfoStriped) block).
|
|
|
- getStorageBlockIndex(storage));
|
|
|
+ byte blockIndex = ((BlockInfoStriped) block).
|
|
|
+ getStorageBlockIndex(storage);
|
|
|
+ liveBlockIndices.add(blockIndex);
|
|
|
+ if (!bitSet.get(blockIndex)) {
|
|
|
+ bitSet.set(blockIndex);
|
|
|
+ } else if (state == StoredReplicaState.LIVE) {
|
|
|
+ numReplicas.subtract(StoredReplicaState.LIVE, 1);
|
|
|
+ numReplicas.add(StoredReplicaState.REDUNDANT, 1);
|
|
|
+ }
|
|
|
}
|
|
|
continue;
|
|
|
}
|
|
|
// 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 (!isStriped && ThreadLocalRandom.current().nextBoolean()) {
|
|
|
+ if (ThreadLocalRandom.current().nextBoolean()) {
|
|
|
srcNodes.set(0, node);
|
|
|
}
|
|
|
}
|
|
|
- if(numReplicas != null)
|
|
|
- numReplicas.set(live, readonly, decommissioned, decommissioning, corrupt,
|
|
|
- excess, 0);
|
|
|
return srcNodes.toArray(new DatanodeDescriptor[srcNodes.size()]);
|
|
|
}
|
|
|
|
|
@@ -2872,8 +2860,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// Now check for completion of blocks and safe block count
|
|
|
NumberReplicas num = countNodes(storedBlock);
|
|
|
int numLiveReplicas = num.liveReplicas();
|
|
|
- int numCurrentReplica = numLiveReplicas
|
|
|
- + pendingReplications.getNumReplicas(storedBlock);
|
|
|
+ int pendingNum = pendingReplications.getNumReplicas(storedBlock);
|
|
|
+ int numCurrentReplica = numLiveReplicas + pendingNum;
|
|
|
|
|
|
if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
|
|
|
hasMinStorage(storedBlock, numLiveReplicas)) {
|
|
@@ -2907,7 +2895,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
} else {
|
|
|
updateNeededReplications(storedBlock, curReplicaDelta, 0);
|
|
|
}
|
|
|
- if (numCurrentReplica > fileReplication) {
|
|
|
+ if (shouldProcessOverReplicated(num, pendingNum, fileReplication)) {
|
|
|
processOverReplicatedBlock(storedBlock, fileReplication, node, delNodeHint);
|
|
|
}
|
|
|
// If the file replication has reached desired value
|
|
@@ -2925,6 +2913,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
return storedBlock;
|
|
|
}
|
|
|
|
|
|
+ private boolean shouldProcessOverReplicated(NumberReplicas num,
|
|
|
+ int pendingNum, int expectedNum) {
|
|
|
+ int numCurrent = num.liveReplicas() + pendingNum;
|
|
|
+ return numCurrent > expectedNum ||
|
|
|
+ (numCurrent == expectedNum && num.redundantInternalBlocks() > 0);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Invalidate corrupt replicas.
|
|
|
* <p>
|
|
@@ -3129,7 +3124,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
// calculate current replication
|
|
|
short expectedReplication = getExpectedReplicaNum(block);
|
|
|
NumberReplicas num = countNodes(block);
|
|
|
- int numCurrentReplica = num.liveReplicas();
|
|
|
+ final int numCurrentReplica = num.liveReplicas();
|
|
|
// add to under-replicated queue if need to be
|
|
|
if (isNeededReplication(block, numCurrentReplica)) {
|
|
|
if (neededReplications.add(block, numCurrentReplica, num.readOnlyReplicas(),
|
|
@@ -3138,7 +3133,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (numCurrentReplica > expectedReplication) {
|
|
|
+ if (shouldProcessOverReplicated(num, 0, expectedReplication)) {
|
|
|
if (num.replicasOnStaleNodes() > 0) {
|
|
|
// If any of the replicas of this block are on nodes that are
|
|
|
// considered "stale", then these replicas may in fact have
|
|
@@ -3666,46 +3661,94 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* Return the number of nodes hosting a given block, grouped
|
|
|
* by the state of those replicas.
|
|
|
* For a striped block, this includes nodes storing blocks belonging to the
|
|
|
- * striped block group.
|
|
|
+ * striped block group. But note we exclude duplicated internal block replicas
|
|
|
+ * for calculating {@link NumberReplicas#liveReplicas}.
|
|
|
*/
|
|
|
- public NumberReplicas countNodes(Block b) {
|
|
|
- int decommissioned = 0;
|
|
|
- int decommissioning = 0;
|
|
|
- int live = 0;
|
|
|
- int readonly = 0;
|
|
|
- int corrupt = 0;
|
|
|
- int excess = 0;
|
|
|
- int stale = 0;
|
|
|
+ public NumberReplicas countNodes(BlockInfo b) {
|
|
|
+ return countNodes(b, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ private NumberReplicas countNodes(BlockInfo b, boolean inStartupSafeMode) {
|
|
|
+ NumberReplicas numberReplicas = new NumberReplicas();
|
|
|
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
|
- for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
- if (storage.getState() == State.FAILED) {
|
|
|
- continue;
|
|
|
- } else if (storage.getState() == State.READ_ONLY_SHARED) {
|
|
|
- readonly++;
|
|
|
- continue;
|
|
|
+ if (b.isStriped()) {
|
|
|
+ countReplicasForStripedBlock(numberReplicas, (BlockInfoStriped) b,
|
|
|
+ nodesCorrupt, inStartupSafeMode);
|
|
|
+ } else {
|
|
|
+ for (DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
+ checkReplicaOnStorage(numberReplicas, b, storage, nodesCorrupt,
|
|
|
+ inStartupSafeMode);
|
|
|
}
|
|
|
+ }
|
|
|
+ return numberReplicas;
|
|
|
+ }
|
|
|
+
|
|
|
+ private StoredReplicaState checkReplicaOnStorage(NumberReplicas counters,
|
|
|
+ BlockInfo b, DatanodeStorageInfo storage,
|
|
|
+ Collection<DatanodeDescriptor> nodesCorrupt, boolean inStartupSafeMode) {
|
|
|
+ final StoredReplicaState s;
|
|
|
+ if (storage.getState() == State.NORMAL) {
|
|
|
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
|
- if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
|
|
|
- corrupt++;
|
|
|
+ if (nodesCorrupt != null && nodesCorrupt.contains(node)) {
|
|
|
+ s = StoredReplicaState.CORRUPT;
|
|
|
+ } else if (inStartupSafeMode) {
|
|
|
+ s = StoredReplicaState.LIVE;
|
|
|
+ counters.add(s, 1);
|
|
|
+ return s;
|
|
|
} else if (node.isDecommissionInProgress()) {
|
|
|
- decommissioning++;
|
|
|
+ s = StoredReplicaState.DECOMMISSIONING;
|
|
|
} else if (node.isDecommissioned()) {
|
|
|
- decommissioned++;
|
|
|
+ s = StoredReplicaState.DECOMMISSIONED;
|
|
|
+ } else if (isExcess(node, b)) {
|
|
|
+ s = StoredReplicaState.EXCESS;
|
|
|
} else {
|
|
|
- LightWeightHashSet<BlockInfo> blocksExcess = excessReplicateMap.get(
|
|
|
- node.getDatanodeUuid());
|
|
|
- if (blocksExcess != null && blocksExcess.contains(b)) {
|
|
|
- excess++;
|
|
|
- } else {
|
|
|
- live++;
|
|
|
- }
|
|
|
+ s = StoredReplicaState.LIVE;
|
|
|
}
|
|
|
+ counters.add(s, 1);
|
|
|
if (storage.areBlockContentsStale()) {
|
|
|
- stale++;
|
|
|
+ counters.add(StoredReplicaState.STALESTORAGE, 1);
|
|
|
+ }
|
|
|
+ } else if (!inStartupSafeMode &&
|
|
|
+ storage.getState() == State.READ_ONLY_SHARED) {
|
|
|
+ s = StoredReplicaState.READONLY;
|
|
|
+ counters.add(s, 1);
|
|
|
+ } else {
|
|
|
+ s = null;
|
|
|
+ }
|
|
|
+ return s;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * For a striped block, it is possible it contains full number of internal
|
|
|
+ * blocks (i.e., 9 by default), but with duplicated replicas of the same
|
|
|
+ * internal block. E.g., for the following list of internal blocks
|
|
|
+ * b0, b0, b1, b2, b3, b4, b5, b6, b7
|
|
|
+ * we have 9 internal blocks but we actually miss b8.
|
|
|
+ * We should use this method to detect the above scenario and schedule
|
|
|
+ * necessary reconstruction.
|
|
|
+ */
|
|
|
+ private void countReplicasForStripedBlock(NumberReplicas counters,
|
|
|
+ BlockInfoStriped block, Collection<DatanodeDescriptor> nodesCorrupt,
|
|
|
+ boolean inStartupSafeMode) {
|
|
|
+ BitSet bitSet = new BitSet(block.getTotalBlockNum());
|
|
|
+ for (StorageAndBlockIndex si : block.getStorageAndIndexInfos()) {
|
|
|
+ StoredReplicaState state = checkReplicaOnStorage(counters, block,
|
|
|
+ si.storage, nodesCorrupt, inStartupSafeMode);
|
|
|
+ if (state == StoredReplicaState.LIVE) {
|
|
|
+ if (!bitSet.get(si.blockIndex)) {
|
|
|
+ bitSet.set(si.blockIndex);
|
|
|
+ } else {
|
|
|
+ counters.subtract(StoredReplicaState.LIVE, 1);
|
|
|
+ counters.add(StoredReplicaState.REDUNDANT, 1);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
- return new NumberReplicas(live, readonly, decommissioned, decommissioning,
|
|
|
- corrupt, excess, stale);
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean isExcess(DatanodeDescriptor node, BlockInfo block) {
|
|
|
+ LightWeightHashSet<BlockInfo> blocksExcess = excessReplicateMap.get(
|
|
|
+ node.getDatanodeUuid());
|
|
|
+ return blocksExcess != null && blocksExcess.contains(block);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -3719,21 +3762,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* @return count of live nodes for this block
|
|
|
*/
|
|
|
int countLiveNodes(BlockInfo b) {
|
|
|
- if (!namesystem.isInStartupSafeMode()) {
|
|
|
- return countNodes(b).liveReplicas();
|
|
|
- }
|
|
|
- // else proceed with fast case
|
|
|
- int live = 0;
|
|
|
- Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
|
|
|
- for (DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
|
|
|
- if (storage.getState() != State.NORMAL) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- final DatanodeDescriptor node = storage.getDatanodeDescriptor();
|
|
|
- if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
|
|
|
- live++;
|
|
|
- }
|
|
|
- return live;
|
|
|
+ final boolean inStartupSafeMode = namesystem.isInStartupSafeMode();
|
|
|
+ return countNodes(b, inStartupSafeMode).liveReplicas();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -3752,9 +3782,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
final BlockInfo block = it.next();
|
|
|
int expectedReplication = this.getReplication(block);
|
|
|
NumberReplicas num = countNodes(block);
|
|
|
- int numCurrentReplica = num.liveReplicas();
|
|
|
- if (numCurrentReplica > expectedReplication) {
|
|
|
- // over-replicated block
|
|
|
+ if (shouldProcessOverReplicated(num, 0, expectedReplication)) {
|
|
|
+ // over-replicated block
|
|
|
processOverReplicatedBlock(block, (short) expectedReplication, null,
|
|
|
null);
|
|
|
numOverReplicated++;
|
|
@@ -3890,7 +3919,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
neededReplications.add(block, n.liveReplicas() + pending,
|
|
|
n.readOnlyReplicas(),
|
|
|
n.decommissionedAndDecommissioning(), expected);
|
|
|
- } else if (n.liveReplicas() > expected) {
|
|
|
+ } else if (shouldProcessOverReplicated(n, 0, expected)) {
|
|
|
processOverReplicatedBlock(block, expected, null, null);
|
|
|
}
|
|
|
}
|