|
@@ -252,9 +252,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
final float blocksInvalidateWorkPct;
|
|
|
final int blocksReplWorkMultiplier;
|
|
|
|
|
|
- /** variable to enable check for enough racks */
|
|
|
- final boolean shouldCheckForEnoughRacks;
|
|
|
-
|
|
|
// whether or not to issue block encryption keys.
|
|
|
final boolean encryptDataTransfer;
|
|
|
|
|
@@ -355,10 +352,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
conf.getInt(
|
|
|
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT);
|
|
|
- this.shouldCheckForEnoughRacks =
|
|
|
- conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null
|
|
|
- ? false : true;
|
|
|
-
|
|
|
this.blocksInvalidateWorkPct = DFSUtil.getInvalidateWorkPctPerIteration(conf);
|
|
|
this.blocksReplWorkMultiplier = DFSUtil.getReplWorkMultiplier(conf);
|
|
|
|
|
@@ -382,7 +375,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
LOG.info("maxReplication = " + maxReplication);
|
|
|
LOG.info("minReplication = " + minReplication);
|
|
|
LOG.info("maxReplicationStreams = " + maxReplicationStreams);
|
|
|
- LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
|
|
|
LOG.info("replicationRecheckInterval = " + replicationRecheckInterval);
|
|
|
LOG.info("encryptDataTransfer = " + encryptDataTransfer);
|
|
|
LOG.info("maxNumBlocksToLog = " + maxNumBlocksToLog);
|
|
@@ -1531,7 +1523,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
NumberReplicas numReplicas, int pendingReplicaNum, int required) {
|
|
|
int numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplicaNum;
|
|
|
return (numEffectiveReplicas >= required) &&
|
|
|
- (pendingReplicaNum > 0 || blockHasEnoughRacks(block, required));
|
|
|
+ (pendingReplicaNum > 0 || isPlacementPolicySatisfied(block));
|
|
|
}
|
|
|
|
|
|
private BlockRecoveryWork scheduleRecovery(BlockInfo block, int priority) {
|
|
@@ -1627,7 +1619,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
|
|
|
DatanodeStorageInfo[] targets = rw.getTargets();
|
|
|
if ( (numReplicas.liveReplicas() >= requiredReplication) &&
|
|
|
- (!blockHasEnoughRacks(block, requiredReplication)) ) {
|
|
|
+ (!isPlacementPolicySatisfied(block)) ) {
|
|
|
if (rw.getSrcNodes()[0].getNetworkLocation().equals(
|
|
|
targets[0].getDatanodeDescriptor().getNetworkLocation())) {
|
|
|
//No use continuing, unless a new rack in this case
|
|
@@ -3145,8 +3137,8 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
bc.getStoragePolicyID());
|
|
|
final List<StorageType> excessTypes = storagePolicy.chooseExcess(
|
|
|
replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
|
|
|
- chooseExcessReplicasContiguous(bc, nonExcess, storedBlock,
|
|
|
- replication, addedNode, delNodeHint, excessTypes);
|
|
|
+ chooseExcessReplicasContiguous(nonExcess, storedBlock, replication,
|
|
|
+ addedNode, delNodeHint, excessTypes);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3164,45 +3156,16 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
* If no such a node is available,
|
|
|
* then pick a node with least free space
|
|
|
*/
|
|
|
- private void chooseExcessReplicasContiguous(BlockCollection bc,
|
|
|
- final Collection<DatanodeStorageInfo> nonExcess,
|
|
|
- BlockInfo storedBlock, short replication,
|
|
|
- DatanodeDescriptor addedNode,
|
|
|
- DatanodeDescriptor delNodeHint,
|
|
|
- List<StorageType> excessTypes) {
|
|
|
+ private void chooseExcessReplicasContiguous(
|
|
|
+ final Collection<DatanodeStorageInfo> nonExcess, BlockInfo storedBlock,
|
|
|
+ short replication, DatanodeDescriptor addedNode,
|
|
|
+ DatanodeDescriptor delNodeHint, List<StorageType> excessTypes) {
|
|
|
BlockPlacementPolicy replicator = placementPolicies.getPolicy(false);
|
|
|
- final Map<String, List<DatanodeStorageInfo>> rackMap = new HashMap<>();
|
|
|
- final List<DatanodeStorageInfo> moreThanOne = new ArrayList<>();
|
|
|
- final List<DatanodeStorageInfo> exactlyOne = new ArrayList<>();
|
|
|
-
|
|
|
- // split nodes into two sets
|
|
|
- // moreThanOne contains nodes on rack with more than one replica
|
|
|
- // exactlyOne contains the remaining nodes
|
|
|
- replicator.splitNodesWithRack(nonExcess, rackMap, moreThanOne, exactlyOne);
|
|
|
-
|
|
|
- // pick one node to delete that favors the delete hint
|
|
|
- // otherwise pick one with least space from priSet if it is not empty
|
|
|
- // otherwise one node with least space from remains
|
|
|
- boolean firstOne = true;
|
|
|
- final DatanodeStorageInfo delNodeHintStorage
|
|
|
- = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, delNodeHint);
|
|
|
- final DatanodeStorageInfo addedNodeStorage
|
|
|
- = DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, addedNode);
|
|
|
- while (nonExcess.size() - replication > 0) {
|
|
|
- final DatanodeStorageInfo cur;
|
|
|
- if (useDelHint(firstOne, delNodeHintStorage, addedNodeStorage,
|
|
|
- moreThanOne, excessTypes)) {
|
|
|
- cur = delNodeHintStorage;
|
|
|
- } else { // regular excessive replica removal
|
|
|
- cur = replicator.chooseReplicaToDelete(bc, storedBlock, replication,
|
|
|
- moreThanOne, exactlyOne, excessTypes);
|
|
|
- }
|
|
|
- firstOne = false;
|
|
|
- // adjust rackmap, moreThanOne, and exactlyOne
|
|
|
- replicator.adjustSetsWithChosenReplica(rackMap, moreThanOne,
|
|
|
- exactlyOne, cur);
|
|
|
-
|
|
|
- processChosenExcessReplica(nonExcess, cur, storedBlock);
|
|
|
+ List<DatanodeStorageInfo> replicasToDelete = replicator
|
|
|
+ .chooseReplicasToDelete(nonExcess, replication, excessTypes,
|
|
|
+ addedNode, delNodeHint);
|
|
|
+ for (DatanodeStorageInfo choosenReplica : replicasToDelete) {
|
|
|
+ processChosenExcessReplica(nonExcess, choosenReplica, storedBlock);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3223,7 +3186,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
BlockInfoStriped sblk = (BlockInfoStriped) storedBlock;
|
|
|
short groupSize = sblk.getTotalBlockNum();
|
|
|
BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true);
|
|
|
- List<DatanodeStorageInfo> empty = new ArrayList<>(0);
|
|
|
|
|
|
// find all duplicated indices
|
|
|
BitSet found = new BitSet(groupSize); //indices found
|
|
@@ -3270,10 +3232,13 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
Block internalBlock = new Block(storedBlock);
|
|
|
internalBlock.setBlockId(storedBlock.getBlockId() + targetIndex);
|
|
|
while (candidates.size() > 1) {
|
|
|
- DatanodeStorageInfo target = placementPolicy.chooseReplicaToDelete(bc,
|
|
|
- internalBlock, (short)1, candidates, empty, excessTypes);
|
|
|
- processChosenExcessReplica(nonExcess, target, storedBlock);
|
|
|
- candidates.remove(target);
|
|
|
+ List<DatanodeStorageInfo> replicasToDelete = placementPolicy
|
|
|
+ .chooseReplicasToDelete(candidates, (short) 1, excessTypes, null,
|
|
|
+ null);
|
|
|
+ for (DatanodeStorageInfo chosen : replicasToDelete) {
|
|
|
+ processChosenExcessReplica(nonExcess, chosen, storedBlock);
|
|
|
+ candidates.remove(chosen);
|
|
|
+ }
|
|
|
}
|
|
|
duplicated.clear(targetIndex);
|
|
|
}
|
|
@@ -3299,27 +3264,6 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
+ "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
|
|
|
}
|
|
|
|
|
|
- /** Check if we can use delHint */
|
|
|
- static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
|
|
|
- DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
|
|
|
- List<StorageType> excessTypes) {
|
|
|
- if (!isFirst) {
|
|
|
- return false; // only consider delHint for the first case
|
|
|
- } else if (delHint == null) {
|
|
|
- return false; // no delHint
|
|
|
- } else if (!excessTypes.contains(delHint.getStorageType())) {
|
|
|
- return false; // delHint storage type is not an excess type
|
|
|
- } else {
|
|
|
- // check if removing delHint reduces the number of racks
|
|
|
- if (moreThan1Racks.contains(delHint)) {
|
|
|
- return true; // delHint and some other nodes are under the same rack
|
|
|
- } else if (added != null && !moreThan1Racks.contains(added)) {
|
|
|
- return true; // the added node adds a new rack
|
|
|
- }
|
|
|
- return false; // removing delHint reduces the number of racks;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) {
|
|
|
assert namesystem.hasWriteLock();
|
|
|
LightWeightHashSet<BlockInfo> excessBlocks = excessReplicateMap.get(
|
|
@@ -3888,74 +3832,23 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
return invalidateBlocks.contains(dn, block);
|
|
|
}
|
|
|
|
|
|
- boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) {
|
|
|
- if (!this.shouldCheckForEnoughRacks) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- Collection<DatanodeDescriptor> corruptNodes =
|
|
|
- corruptReplicas.getNodes(storedBlock);
|
|
|
-
|
|
|
- if (storedBlock.isStriped()) {
|
|
|
- return blockHasEnoughRacksStriped(storedBlock, corruptNodes);
|
|
|
- } else {
|
|
|
- return blockHashEnoughRacksContiguous(storedBlock, expectedStorageNum,
|
|
|
- corruptNodes);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Verify whether given striped block is distributed through enough racks.
|
|
|
- * As dicussed in HDFS-7613, ec file requires racks at least as many as
|
|
|
- * the number of data block number.
|
|
|
- */
|
|
|
- boolean blockHasEnoughRacksStriped(BlockInfo storedBlock,
|
|
|
- Collection<DatanodeDescriptor> corruptNodes) {
|
|
|
- if (!datanodeManager.hasClusterEverBeenMultiRack()) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- boolean enoughRacks = false;
|
|
|
- Set<String> rackNameSet = new HashSet<>();
|
|
|
- int dataBlockNum = ((BlockInfoStriped)storedBlock).getRealDataBlockNum();
|
|
|
+ boolean isPlacementPolicySatisfied(BlockInfo storedBlock) {
|
|
|
+ List<DatanodeDescriptor> liveNodes = new ArrayList<>();
|
|
|
+ Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
|
|
|
+ .getNodes(storedBlock);
|
|
|
for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
|
|
|
final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
|
|
|
- if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
|
|
- if ((corruptNodes == null) || !corruptNodes.contains(cur)) {
|
|
|
- String rackNameNew = cur.getNetworkLocation();
|
|
|
- rackNameSet.add(rackNameNew);
|
|
|
- if (rackNameSet.size() >= dataBlockNum) {
|
|
|
- enoughRacks = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return enoughRacks;
|
|
|
- }
|
|
|
-
|
|
|
- boolean blockHashEnoughRacksContiguous(BlockInfo storedBlock,
|
|
|
- int expectedStorageNum, Collection<DatanodeDescriptor> corruptNodes) {
|
|
|
- boolean enoughRacks = false;
|
|
|
- String rackName = null;
|
|
|
- for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
|
|
|
- final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
|
|
|
- if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
|
|
|
- if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
|
|
|
- if (expectedStorageNum == 1 || (expectedStorageNum > 1 &&
|
|
|
- !datanodeManager.hasClusterEverBeenMultiRack())) {
|
|
|
- enoughRacks = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- String rackNameNew = cur.getNetworkLocation();
|
|
|
- if (rackName == null) {
|
|
|
- rackName = rackNameNew;
|
|
|
- } else if (!rackName.equals(rackNameNew)) {
|
|
|
- enoughRacks = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
+ if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()
|
|
|
+ && ((corruptNodes == null) || !corruptNodes.contains(cur))) {
|
|
|
+ liveNodes.add(cur);
|
|
|
}
|
|
|
}
|
|
|
- return enoughRacks;
|
|
|
+ DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]);
|
|
|
+ BlockPlacementPolicy placementPolicy = placementPolicies
|
|
|
+ .getPolicy(storedBlock.isStriped());
|
|
|
+ int numReplicas = storedBlock.isStriped() ? ((BlockInfoStriped) storedBlock)
|
|
|
+ .getRealDataBlockNum() : storedBlock.getReplication();
|
|
|
+ return placementPolicy.verifyBlockPlacement(locs, numReplicas).isPlacementPolicySatisfied();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -3964,7 +3857,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|
|
*/
|
|
|
boolean isNeededReplication(BlockInfo storedBlock, int current) {
|
|
|
int expected = getExpectedReplicaNum(storedBlock);
|
|
|
- return current < expected || !blockHasEnoughRacks(storedBlock, expected);
|
|
|
+ return current < expected || !isPlacementPolicySatisfied(storedBlock);
|
|
|
}
|
|
|
|
|
|
public short getExpectedReplicaNum(BlockInfo block) {
|