|
@@ -27,6 +27,7 @@ import java.util.EnumMap;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
+import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -50,8 +51,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -92,10 +91,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
private Context<T> ctxt;
|
|
|
private BlockMoveTaskHandler blockMoveTaskHandler;
|
|
|
private final Configuration conf;
|
|
|
+ private DatanodeCacheManager<T> dnCacheMgr;
|
|
|
|
|
|
public StoragePolicySatisfier(Configuration conf) {
|
|
|
this.conf = conf;
|
|
|
}
|
|
|
+
|
|
|
/**
|
|
|
* Represents the collective analysis status for all blocks.
|
|
|
*/
|
|
@@ -190,6 +191,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
storagePolicySatisfierThread.start();
|
|
|
this.storageMovementsMonitor.start();
|
|
|
this.storageMovementNeeded.activate();
|
|
|
+ dnCacheMgr = new DatanodeCacheManager<T>(conf);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -271,7 +273,6 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
}
|
|
|
T trackId = itemInfo.getFile();
|
|
|
BlocksMovingAnalysis status = null;
|
|
|
- DatanodeStorageReport[] liveDnReports;
|
|
|
BlockStoragePolicy existingStoragePolicy;
|
|
|
// TODO: presently, context internally acquire the lock
|
|
|
// and returns the result. Need to discuss to move the lock outside?
|
|
@@ -282,14 +283,13 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
// just remove trackId from the queue
|
|
|
storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
|
|
|
} else {
|
|
|
- liveDnReports = ctxt.getLiveDatanodeStorageReport();
|
|
|
byte existingStoragePolicyID = fileStatus.getStoragePolicy();
|
|
|
existingStoragePolicy = ctxt
|
|
|
.getStoragePolicy(existingStoragePolicyID);
|
|
|
|
|
|
HdfsLocatedFileStatus file = (HdfsLocatedFileStatus) fileStatus;
|
|
|
status = analyseBlocksStorageMovementsAndAssignToDN(file,
|
|
|
- existingStoragePolicy, liveDnReports);
|
|
|
+ existingStoragePolicy);
|
|
|
switch (status.status) {
|
|
|
// Just add to monitor, so it will be retried after timeout
|
|
|
case ANALYSIS_SKIPPED_FOR_RETRY:
|
|
@@ -380,8 +380,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
|
|
|
private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
|
|
|
HdfsLocatedFileStatus fileInfo,
|
|
|
- BlockStoragePolicy existingStoragePolicy,
|
|
|
- DatanodeStorageReport[] liveDns) {
|
|
|
+ BlockStoragePolicy existingStoragePolicy) throws IOException {
|
|
|
BlocksMovingAnalysis.Status status =
|
|
|
BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
|
|
|
final ErasureCodingPolicy ecPolicy = fileInfo.getErasureCodingPolicy();
|
|
@@ -407,6 +406,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
|
|
|
boolean hasLowRedundancyBlocks = false;
|
|
|
int replication = fileInfo.getReplication();
|
|
|
+ DatanodeMap liveDns = dnCacheMgr.getLiveDatanodeStorageReport(ctxt);
|
|
|
for (int i = 0; i < blocks.size(); i++) {
|
|
|
LocatedBlock blockInfo = blocks.get(i);
|
|
|
|
|
@@ -462,7 +462,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
// If there is no block paired and few blocks are low redundant, so marking
|
|
|
// the status as FEW_LOW_REDUNDANCY_BLOCKS.
|
|
|
if (hasLowRedundancyBlocks
|
|
|
- && status == BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED) {
|
|
|
+ && status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
|
|
|
status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
}
|
|
|
List<Block> assignedBlockIds = new ArrayList<Block>();
|
|
@@ -526,13 +526,17 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
* - list to get existing storage types
|
|
|
* @param storages
|
|
|
* - available storages
|
|
|
+ * @param liveDns
|
|
|
+ * - live datanodes which can be used as targets
|
|
|
+ * @param ecPolicy
|
|
|
+ * - ec policy of sps invoked file
|
|
|
* @return false if some of the block locations failed to find target node to
|
|
|
* satisfy the storage policy, true otherwise
|
|
|
*/
|
|
|
private boolean computeBlockMovingInfos(
|
|
|
List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
|
|
|
List<StorageType> expectedStorageTypes, List<StorageType> existing,
|
|
|
- DatanodeInfo[] storages, DatanodeStorageReport[] liveDns,
|
|
|
+ DatanodeInfo[] storages, DatanodeMap liveDns,
|
|
|
ErasureCodingPolicy ecPolicy) {
|
|
|
boolean foundMatchingTargetNodesForBlock = true;
|
|
|
if (!removeOverlapBetweenStorageTypes(expectedStorageTypes,
|
|
@@ -572,12 +576,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- StorageTypeNodeMap locsForExpectedStorageTypes =
|
|
|
+ EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns =
|
|
|
findTargetsForExpectedStorageTypes(expectedStorageTypes, liveDns);
|
|
|
|
|
|
foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
|
|
|
blockMovingInfos, blockInfo, sourceWithStorageMap,
|
|
|
- expectedStorageTypes, locsForExpectedStorageTypes,
|
|
|
+ expectedStorageTypes, targetDns,
|
|
|
ecPolicy);
|
|
|
}
|
|
|
return foundMatchingTargetNodesForBlock;
|
|
@@ -593,9 +597,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
* - Block
|
|
|
* @param sourceWithStorageList
|
|
|
* - Source Datanode with storages list
|
|
|
- * @param expected
|
|
|
+ * @param expectedTypes
|
|
|
* - Expecting storages to move
|
|
|
- * @param locsForExpectedStorageTypes
|
|
|
+ * @param targetDns
|
|
|
* - Available DNs for expected storage types
|
|
|
* @return false if some of the block locations failed to find target node to
|
|
|
* satisfy the storage policy
|
|
@@ -603,8 +607,8 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
private boolean findSourceAndTargetToMove(
|
|
|
List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
|
|
|
List<StorageTypeNodePair> sourceWithStorageList,
|
|
|
- List<StorageType> expected,
|
|
|
- StorageTypeNodeMap locsForExpectedStorageTypes,
|
|
|
+ List<StorageType> expectedTypes,
|
|
|
+ EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns,
|
|
|
ErasureCodingPolicy ecPolicy) {
|
|
|
boolean foundMatchingTargetNodesForBlock = true;
|
|
|
List<DatanodeInfo> excludeNodes = new ArrayList<>();
|
|
@@ -617,9 +621,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
|
|
|
// Check whether the block replica is already placed in the expected
|
|
|
// storage type in this source datanode.
|
|
|
- if (!expected.contains(existingTypeNodePair.storageType)) {
|
|
|
+ if (!expectedTypes.contains(existingTypeNodePair.storageType)) {
|
|
|
StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(blockInfo,
|
|
|
- existingTypeNodePair.dn, expected);
|
|
|
+ existingTypeNodePair.dn, targetDns, expectedTypes);
|
|
|
if (chosenTarget != null) {
|
|
|
if (blockInfo.isStriped()) {
|
|
|
buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
@@ -631,7 +635,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
chosenTarget.storageType, blockMovingInfos);
|
|
|
}
|
|
|
- expected.remove(chosenTarget.storageType);
|
|
|
+ expectedTypes.remove(chosenTarget.storageType);
|
|
|
}
|
|
|
}
|
|
|
// To avoid choosing this excludeNodes as targets later
|
|
@@ -648,24 +652,23 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
if (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (chosenTarget == null && ctxt
|
|
|
- .getNetworkTopology().isNodeGroupAware()) {
|
|
|
+ if (chosenTarget == null && dnCacheMgr.getCluster().isNodeGroupAware()) {
|
|
|
chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
|
|
|
- expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
|
|
|
+ expectedTypes, Matcher.SAME_NODE_GROUP, targetDns,
|
|
|
excludeNodes);
|
|
|
}
|
|
|
|
|
|
// Then, match nodes on the same rack
|
|
|
if (chosenTarget == null) {
|
|
|
chosenTarget =
|
|
|
- chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
|
|
|
- Matcher.SAME_RACK, locsForExpectedStorageTypes, excludeNodes);
|
|
|
+ chooseTarget(blockInfo, existingTypeNodePair.dn, expectedTypes,
|
|
|
+ Matcher.SAME_RACK, targetDns, excludeNodes);
|
|
|
}
|
|
|
|
|
|
if (chosenTarget == null) {
|
|
|
chosenTarget =
|
|
|
- chooseTarget(blockInfo, existingTypeNodePair.dn, expected,
|
|
|
- Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
|
|
|
+ chooseTarget(blockInfo, existingTypeNodePair.dn, expectedTypes,
|
|
|
+ Matcher.ANY_OTHER, targetDns, excludeNodes);
|
|
|
}
|
|
|
if (null != chosenTarget) {
|
|
|
if (blockInfo.isStriped()) {
|
|
@@ -678,17 +681,17 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
chosenTarget.storageType, blockMovingInfos);
|
|
|
}
|
|
|
|
|
|
- expected.remove(chosenTarget.storageType);
|
|
|
+ expectedTypes.remove(chosenTarget.storageType);
|
|
|
excludeNodes.add(chosenTarget.dn);
|
|
|
} else {
|
|
|
LOG.warn(
|
|
|
"Failed to choose target datanode for the required"
|
|
|
+ " storage types {}, block:{}, existing storage type:{}",
|
|
|
- expected, blockInfo, existingTypeNodePair.storageType);
|
|
|
+ expectedTypes, blockInfo, existingTypeNodePair.storageType);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (expected.size() > 0) {
|
|
|
+ if (expectedTypes.size() > 0) {
|
|
|
foundMatchingTargetNodesForBlock = false;
|
|
|
}
|
|
|
|
|
@@ -750,20 +753,38 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
/**
|
|
|
* Choose the target storage within same datanode if possible.
|
|
|
*
|
|
|
- * @param block
|
|
|
+ * @param blockInfo
|
|
|
* - block info
|
|
|
* @param source
|
|
|
* - source datanode
|
|
|
+ * @param targetDns
|
|
|
+ * - set of target datanodes with its respective storage type
|
|
|
* @param targetTypes
|
|
|
* - list of target storage types
|
|
|
*/
|
|
|
private StorageTypeNodePair chooseTargetTypeInSameNode(LocatedBlock blockInfo,
|
|
|
- DatanodeInfo source, List<StorageType> targetTypes) {
|
|
|
+ DatanodeInfo source,
|
|
|
+ EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns,
|
|
|
+ List<StorageType> targetTypes) {
|
|
|
for (StorageType t : targetTypes) {
|
|
|
- boolean goodTargetDn =
|
|
|
- ctxt.checkDNSpaceForScheduling(source, t, blockInfo.getBlockSize());
|
|
|
- if (goodTargetDn) {
|
|
|
- return new StorageTypeNodePair(t, source);
|
|
|
+ List<DatanodeWithStorage.StorageDetails> targetNodeStorages =
|
|
|
+ targetDns.get(t);
|
|
|
+ if (targetNodeStorages == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ for (DatanodeWithStorage.StorageDetails targetNode : targetNodeStorages) {
|
|
|
+ if (targetNode.getDatanodeInfo().equals(source)) {
|
|
|
+ // Good target with enough space to write the given block size.
|
|
|
+ if (targetNode.hasSpaceForScheduling(blockInfo.getBlockSize())) {
|
|
|
+ targetNode.incScheduledSize(blockInfo.getBlockSize());
|
|
|
+ return new StorageTypeNodePair(t, source);
|
|
|
+ }
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Datanode:{} storage type:{} doesn't have sufficient "
|
|
|
+ + "space:{} to move the target block size:{}",
|
|
|
+ source, t, targetNode, blockInfo.getBlockSize());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
return null;
|
|
@@ -771,84 +792,105 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
|
|
|
private StorageTypeNodePair chooseTarget(LocatedBlock block,
|
|
|
DatanodeInfo source, List<StorageType> targetTypes, Matcher matcher,
|
|
|
- StorageTypeNodeMap locsForExpectedStorageTypes,
|
|
|
- List<DatanodeInfo> excludeNodes) {
|
|
|
+ EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>
|
|
|
+ locsForExpectedStorageTypes, List<DatanodeInfo> excludeNodes) {
|
|
|
for (StorageType t : targetTypes) {
|
|
|
- List<DatanodeInfo> nodesWithStorages =
|
|
|
- locsForExpectedStorageTypes.getNodesWithStorages(t);
|
|
|
+ List<DatanodeWithStorage.StorageDetails> nodesWithStorages =
|
|
|
+ locsForExpectedStorageTypes.get(t);
|
|
|
if (nodesWithStorages == null || nodesWithStorages.isEmpty()) {
|
|
|
continue; // no target nodes with the required storage type.
|
|
|
}
|
|
|
Collections.shuffle(nodesWithStorages);
|
|
|
- for (DatanodeInfo target : nodesWithStorages) {
|
|
|
+ for (DatanodeWithStorage.StorageDetails targetNode : nodesWithStorages) {
|
|
|
+ DatanodeInfo target = targetNode.getDatanodeInfo();
|
|
|
if (!excludeNodes.contains(target)
|
|
|
- && matcher.match(ctxt.getNetworkTopology(), source, target)) {
|
|
|
- boolean goodTargetDn =
|
|
|
- ctxt.checkDNSpaceForScheduling(target, t, block.getBlockSize());
|
|
|
- if (goodTargetDn) {
|
|
|
+ && matcher.match(dnCacheMgr.getCluster(), source, target)) {
|
|
|
+ // Good target with enough space to write the given block size.
|
|
|
+ if (targetNode.hasSpaceForScheduling(block.getBlockSize())) {
|
|
|
+ targetNode.incScheduledSize(block.getBlockSize());
|
|
|
return new StorageTypeNodePair(t, target);
|
|
|
}
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Datanode:{} storage type:{} doesn't have sufficient "
|
|
|
+ + "space:{} to move the target block size:{}",
|
|
|
+ target, t, targetNode, block.getBlockSize());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- private static class StorageTypeNodePair {
|
|
|
- private StorageType storageType = null;
|
|
|
- private DatanodeInfo dn = null;
|
|
|
+ /**
|
|
|
+ * Keeps datanode with its respective storage type.
|
|
|
+ */
|
|
|
+ private static final class StorageTypeNodePair {
|
|
|
+ private final StorageType storageType;
|
|
|
+ private final DatanodeInfo dn;
|
|
|
|
|
|
- StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
|
|
|
+ private StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
|
|
|
this.storageType = storageType;
|
|
|
this.dn = dn;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private StorageTypeNodeMap findTargetsForExpectedStorageTypes(
|
|
|
- List<StorageType> expected, DatanodeStorageReport[] liveDns) {
|
|
|
- StorageTypeNodeMap targetMap = new StorageTypeNodeMap();
|
|
|
- for (DatanodeStorageReport dn : liveDns) {
|
|
|
- StorageReport[] storageReports = dn.getStorageReports();
|
|
|
- for (StorageReport storageReport : storageReports) {
|
|
|
- StorageType t = storageReport.getStorage().getStorageType();
|
|
|
- if (expected.contains(t)) {
|
|
|
- final long maxRemaining = getMaxRemaining(dn.getStorageReports(), t);
|
|
|
- if (maxRemaining > 0L) {
|
|
|
- targetMap.add(t, dn.getDatanodeInfo());
|
|
|
- }
|
|
|
+ private EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>
|
|
|
+ findTargetsForExpectedStorageTypes(List<StorageType> expected,
|
|
|
+ DatanodeMap liveDns) {
|
|
|
+ EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetsMap =
|
|
|
+ new EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>(
|
|
|
+ StorageType.class);
|
|
|
+
|
|
|
+ for (StorageType storageType : expected) {
|
|
|
+ List<DatanodeWithStorage> nodes = liveDns.getTarget(storageType);
|
|
|
+ if (nodes == null) {
|
|
|
+ return targetsMap;
|
|
|
+ }
|
|
|
+ List<DatanodeWithStorage.StorageDetails> listNodes = targetsMap
|
|
|
+ .get(storageType);
|
|
|
+ if (listNodes == null) {
|
|
|
+ listNodes = new ArrayList<>();
|
|
|
+ targetsMap.put(storageType, listNodes);
|
|
|
+ }
|
|
|
+
|
|
|
+ for (DatanodeWithStorage n : nodes) {
|
|
|
+ final DatanodeWithStorage.StorageDetails node = getMaxRemaining(n,
|
|
|
+ storageType);
|
|
|
+ if (node != null) {
|
|
|
+ listNodes.add(node);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- return targetMap;
|
|
|
+ return targetsMap;
|
|
|
}
|
|
|
|
|
|
- private static long getMaxRemaining(StorageReport[] storageReports,
|
|
|
- StorageType t) {
|
|
|
+ private static DatanodeWithStorage.StorageDetails getMaxRemaining(
|
|
|
+ DatanodeWithStorage node, StorageType storageType) {
|
|
|
long max = 0L;
|
|
|
- for (StorageReport r : storageReports) {
|
|
|
- if (r.getStorage().getStorageType() == t) {
|
|
|
- if (r.getRemaining() > max) {
|
|
|
- max = r.getRemaining();
|
|
|
- }
|
|
|
+ DatanodeWithStorage.StorageDetails nodeInfo = null;
|
|
|
+ List<DatanodeWithStorage.StorageDetails> storages = node
|
|
|
+ .getNodesWithStorages(storageType);
|
|
|
+ for (DatanodeWithStorage.StorageDetails n : storages) {
|
|
|
+ if (n.availableSizeToMove() > max) {
|
|
|
+ max = n.availableSizeToMove();
|
|
|
+ nodeInfo = n;
|
|
|
}
|
|
|
}
|
|
|
- return max;
|
|
|
+ return nodeInfo;
|
|
|
}
|
|
|
|
|
|
private boolean checkSourceAndTargetTypeExists(DatanodeInfo dn,
|
|
|
- List<StorageType> existing, List<StorageType> expectedStorageTypes,
|
|
|
- DatanodeStorageReport[] liveDns) {
|
|
|
+ List<StorageType> existingStorageTypes,
|
|
|
+ List<StorageType> expectedStorageTypes, DatanodeMap liveDns) {
|
|
|
boolean isExpectedTypeAvailable = false;
|
|
|
boolean isExistingTypeAvailable = false;
|
|
|
- for (DatanodeStorageReport liveDn : liveDns) {
|
|
|
- if (dn.equals(liveDn.getDatanodeInfo())) {
|
|
|
- StorageReport[] storageReports = liveDn.getStorageReports();
|
|
|
- for (StorageReport eachStorage : storageReports) {
|
|
|
- StorageType storageType = eachStorage.getStorage().getStorageType();
|
|
|
- if (existing.contains(storageType)) {
|
|
|
+ for (DatanodeWithStorage liveDn : liveDns.getTargets()) {
|
|
|
+ if (dn.equals(liveDn.datanode)) {
|
|
|
+ for (StorageType eachType : liveDn.getStorageTypes()) {
|
|
|
+ if (existingStorageTypes.contains(eachType)) {
|
|
|
isExistingTypeAvailable = true;
|
|
|
}
|
|
|
- if (expectedStorageTypes.contains(storageType)) {
|
|
|
+ if (expectedStorageTypes.contains(eachType)) {
|
|
|
isExpectedTypeAvailable = true;
|
|
|
}
|
|
|
if (isExistingTypeAvailable && isExpectedTypeAvailable) {
|
|
@@ -860,29 +902,143 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
return isExistingTypeAvailable && isExpectedTypeAvailable;
|
|
|
}
|
|
|
|
|
|
- private static class StorageTypeNodeMap {
|
|
|
- private final EnumMap<StorageType, List<DatanodeInfo>> typeNodeMap =
|
|
|
- new EnumMap<StorageType, List<DatanodeInfo>>(StorageType.class);
|
|
|
+ /**
|
|
|
+ * Maintains storage type map with the available datanodes in the cluster.
|
|
|
+ */
|
|
|
+ public static class DatanodeMap {
|
|
|
+ private final EnumMap<StorageType, List<DatanodeWithStorage>> targetsMap =
|
|
|
+ new EnumMap<StorageType, List<DatanodeWithStorage>>(StorageType.class);
|
|
|
+
|
|
|
+ private List<DatanodeWithStorage> targets = new ArrayList<>();
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Build datanode map with the available storage types.
|
|
|
+ *
|
|
|
+ * @param node
|
|
|
+ * datanode
|
|
|
+ * @param storageTypes
|
|
|
+ * list of available storage types in the given datanode
|
|
|
+ * @param maxSize2Move
|
|
|
+ * available space which can be used for scheduling block move
|
|
|
+ */
|
|
|
+ void addTarget(DatanodeInfo node, List<StorageType> storageTypes,
|
|
|
+ List<Long> maxSize2Move) {
|
|
|
+ DatanodeWithStorage nodeStorage = new DatanodeWithStorage(node);
|
|
|
+ targets.add(nodeStorage);
|
|
|
+ for (int i = 0; i < storageTypes.size(); i++) {
|
|
|
+ StorageType type = storageTypes.get(i);
|
|
|
+ List<DatanodeWithStorage> nodeStorages = targetsMap.get(type);
|
|
|
+ nodeStorage.addStorageType(type, maxSize2Move.get(i));
|
|
|
+ if (nodeStorages == null) {
|
|
|
+ nodeStorages = new LinkedList<>();
|
|
|
+ targetsMap.put(type, nodeStorages);
|
|
|
+ }
|
|
|
+ nodeStorages.add(nodeStorage);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ List<DatanodeWithStorage> getTarget(StorageType storageType) {
|
|
|
+ return targetsMap.get(storageType);
|
|
|
+ }
|
|
|
+
|
|
|
+ public List<DatanodeWithStorage> getTargets() {
|
|
|
+ return targets;
|
|
|
+ }
|
|
|
+
|
|
|
+ void reset() {
|
|
|
+ targetsMap.clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Keeps datanode with its respective set of supported storage types. It holds
|
|
|
+ * the available space in each volumes and will be used while pairing the
|
|
|
+ * target datanodes.
|
|
|
+ */
|
|
|
+ public static final class DatanodeWithStorage {
|
|
|
+ private final EnumMap<StorageType, List<StorageDetails>> storageMap =
|
|
|
+ new EnumMap<StorageType, List<StorageDetails>>(StorageType.class);
|
|
|
+ private final DatanodeInfo datanode;
|
|
|
+
|
|
|
+ private DatanodeWithStorage(DatanodeInfo datanode) {
|
|
|
+ this.datanode = datanode;
|
|
|
+ }
|
|
|
+
|
|
|
+ public DatanodeInfo getDatanodeInfo() {
|
|
|
+ return datanode;
|
|
|
+ }
|
|
|
+
|
|
|
+ Set<StorageType> getStorageTypes() {
|
|
|
+ return storageMap.keySet();
|
|
|
+ }
|
|
|
|
|
|
- private void add(StorageType t, DatanodeInfo dn) {
|
|
|
- List<DatanodeInfo> nodesWithStorages = getNodesWithStorages(t);
|
|
|
- LinkedList<DatanodeInfo> value = null;
|
|
|
+ private void addStorageType(StorageType t, long maxSize2Move) {
|
|
|
+ List<StorageDetails> nodesWithStorages = getNodesWithStorages(t);
|
|
|
if (nodesWithStorages == null) {
|
|
|
- value = new LinkedList<DatanodeInfo>();
|
|
|
- value.add(dn);
|
|
|
- typeNodeMap.put(t, value);
|
|
|
- } else {
|
|
|
- nodesWithStorages.add(dn);
|
|
|
+ nodesWithStorages = new LinkedList<StorageDetails>();
|
|
|
+ storageMap.put(t, nodesWithStorages);
|
|
|
}
|
|
|
+ nodesWithStorages.add(new StorageDetails(maxSize2Move));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
+ * Returns datanode storages which has the given storage type.
|
|
|
+ *
|
|
|
* @param type
|
|
|
- * - Storage type
|
|
|
- * @return datanodes which has the given storage type
|
|
|
+ * - storage type
|
|
|
+ * @return datanodes for the given storage type
|
|
|
*/
|
|
|
- private List<DatanodeInfo> getNodesWithStorages(StorageType type) {
|
|
|
- return typeNodeMap.get(type);
|
|
|
+ private List<StorageDetails> getNodesWithStorages(StorageType type) {
|
|
|
+ return storageMap.get(type);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String toString() {
|
|
|
+ return new StringBuilder().append("DatanodeWithStorageInfo(\n ")
|
|
|
+ .append("Datanode: ").append(datanode).append(" StorageTypeNodeMap: ")
|
|
|
+ .append(storageMap).append(")").toString();
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Storage details in a datanode storage type. */
|
|
|
+ final class StorageDetails {
|
|
|
+ private final long maxSize2Move;
|
|
|
+ private long scheduledSize = 0L;
|
|
|
+
|
|
|
+ private StorageDetails(long maxSize2Move) {
|
|
|
+ this.maxSize2Move = maxSize2Move;
|
|
|
+ }
|
|
|
+
|
|
|
+ private DatanodeInfo getDatanodeInfo() {
|
|
|
+ return DatanodeWithStorage.this.datanode;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Checks whether this datanode storage has sufficient space to occupy the
|
|
|
+ * given block size.
|
|
|
+ */
|
|
|
+ private synchronized boolean hasSpaceForScheduling(long size) {
|
|
|
+ return availableSizeToMove() > size;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @return the total number of bytes that need to be moved.
|
|
|
+ */
|
|
|
+ private synchronized long availableSizeToMove() {
|
|
|
+ return maxSize2Move - scheduledSize;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Increment scheduled size. */
|
|
|
+ private synchronized void incScheduledSize(long size) {
|
|
|
+ scheduledSize += size;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String toString() {
|
|
|
+ return new StringBuilder().append("StorageDetails(\n ")
|
|
|
+ .append("maxSize2Move: ").append(maxSize2Move)
|
|
|
+ .append(" scheduledSize: ").append(scheduledSize).append(")")
|
|
|
+ .toString();
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|