|
@@ -24,9 +24,12 @@ import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
|
import java.util.EnumMap;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -50,7 +53,6 @@ import org.apache.hadoop.hdfs.server.balancer.Matcher;
|
|
|
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.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -83,8 +85,6 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
private BlockStorageMovementNeeded<T> storageMovementNeeded;
|
|
|
private BlockStorageMovementAttemptedItems<T> storageMovementsMonitor;
|
|
|
private volatile boolean isRunning = false;
|
|
|
- private volatile StoragePolicySatisfierMode spsMode =
|
|
|
- StoragePolicySatisfierMode.NONE;
|
|
|
private int spsWorkMultiplier;
|
|
|
private long blockCount = 0L;
|
|
|
private int blockMovementMaxRetry;
|
|
@@ -128,11 +128,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
}
|
|
|
|
|
|
private Status status = null;
|
|
|
- private List<Block> assignedBlocks = null;
|
|
|
+ private Map<Block, Set<StorageTypeNodePair>> assignedBlocks = null;
|
|
|
|
|
|
- BlocksMovingAnalysis(Status status, List<Block> blockMovingInfo) {
|
|
|
+ BlocksMovingAnalysis(Status status,
|
|
|
+ Map<Block, Set<StorageTypeNodePair>> assignedBlocks) {
|
|
|
this.status = status;
|
|
|
- this.assignedBlocks = blockMovingInfo;
|
|
|
+ this.assignedBlocks = assignedBlocks;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -164,7 +165,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
serviceMode);
|
|
|
return;
|
|
|
}
|
|
|
- if (spsMode == StoragePolicySatisfierMode.INTERNAL
|
|
|
+ if (serviceMode == StoragePolicySatisfierMode.INTERNAL
|
|
|
&& ctxt.isMoverRunning()) {
|
|
|
isRunning = false;
|
|
|
LOG.error(
|
|
@@ -175,14 +176,13 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
}
|
|
|
if (reconfigStart) {
|
|
|
LOG.info("Starting {} StoragePolicySatisfier, as admin requested to "
|
|
|
- + "start it.", StringUtils.toLowerCase(spsMode.toString()));
|
|
|
+ + "start it.", StringUtils.toLowerCase(serviceMode.toString()));
|
|
|
} else {
|
|
|
LOG.info("Starting {} StoragePolicySatisfier.",
|
|
|
- StringUtils.toLowerCase(spsMode.toString()));
|
|
|
+ StringUtils.toLowerCase(serviceMode.toString()));
|
|
|
}
|
|
|
|
|
|
isRunning = true;
|
|
|
- this.spsMode = serviceMode;
|
|
|
// Ensure that all the previously submitted block movements(if any) have to
|
|
|
// be stopped in all datanodes.
|
|
|
addDropSPSWorkCommandsToAllDNs();
|
|
@@ -297,36 +297,36 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
// be removed on storage movement attempt finished report.
|
|
|
case BLOCKS_TARGETS_PAIRED:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Block analysis status:{} for the file path:{}."
|
|
|
+ LOG.debug("Block analysis status:{} for the file id:{}."
|
|
|
+ " Adding to attempt monitor queue for the storage "
|
|
|
+ "movement attempt finished report",
|
|
|
- status.status, fileStatus.getPath());
|
|
|
+ status.status, fileStatus.getFileId());
|
|
|
}
|
|
|
- this.storageMovementsMonitor.add(new AttemptedItemInfo<T>(
|
|
|
- itemInfo.getStartPath(), itemInfo.getFile(), monotonicNow(),
|
|
|
- status.assignedBlocks, itemInfo.getRetryCount()));
|
|
|
+ this.storageMovementsMonitor.add(itemInfo.getStartPath(),
|
|
|
+ itemInfo.getFile(), monotonicNow(), status.assignedBlocks,
|
|
|
+ itemInfo.getRetryCount());
|
|
|
break;
|
|
|
case NO_BLOCKS_TARGETS_PAIRED:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Adding trackID:{} for the file path:{} back to"
|
|
|
+ LOG.debug("Adding trackID:{} for the file id:{} back to"
|
|
|
+ " retry queue as none of the blocks found its eligible"
|
|
|
- + " targets.", trackId, fileStatus.getPath());
|
|
|
+ + " targets.", trackId, fileStatus.getFileId());
|
|
|
}
|
|
|
retryItem = true;
|
|
|
break;
|
|
|
case FEW_LOW_REDUNDANCY_BLOCKS:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Adding trackID:{} for the file path:{} back to "
|
|
|
+ LOG.debug("Adding trackID:{} for the file id:{} back to "
|
|
|
+ "retry queue as some of the blocks are low redundant.",
|
|
|
- trackId, fileStatus.getPath());
|
|
|
+ trackId, fileStatus.getFileId());
|
|
|
}
|
|
|
retryItem = true;
|
|
|
break;
|
|
|
case BLOCKS_FAILED_TO_MOVE:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Adding trackID:{} for the file path:{} back to "
|
|
|
+ LOG.debug("Adding trackID:{} for the file id:{} back to "
|
|
|
+ "retry queue as some of the blocks movement failed.",
|
|
|
- trackId, fileStatus.getPath());
|
|
|
+ trackId, fileStatus.getFileId());
|
|
|
}
|
|
|
retryItem = true;
|
|
|
break;
|
|
@@ -334,9 +334,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
case BLOCKS_TARGET_PAIRING_SKIPPED:
|
|
|
case BLOCKS_ALREADY_SATISFIED:
|
|
|
default:
|
|
|
- LOG.info("Block analysis status:{} for the file path:{}."
|
|
|
+ LOG.info("Block analysis status:{} for the file id:{}."
|
|
|
+ " So, Cleaning up the Xattrs.", status.status,
|
|
|
- fileStatus.getPath());
|
|
|
+ fileStatus.getFileId());
|
|
|
storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
|
|
|
break;
|
|
|
}
|
|
@@ -389,19 +389,19 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
if (!lastBlkComplete) {
|
|
|
// Postpone, currently file is under construction
|
|
|
LOG.info("File: {} is under construction. So, postpone"
|
|
|
- + " this to the next retry iteration", fileInfo.getPath());
|
|
|
+ + " this to the next retry iteration", fileInfo.getFileId());
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
|
|
|
- new ArrayList<>());
|
|
|
+ new HashMap<>());
|
|
|
}
|
|
|
|
|
|
List<LocatedBlock> blocks = locatedBlocks.getLocatedBlocks();
|
|
|
if (blocks.size() == 0) {
|
|
|
LOG.info("File: {} is not having any blocks."
|
|
|
- + " So, skipping the analysis.", fileInfo.getPath());
|
|
|
+ + " So, skipping the analysis.", fileInfo.getFileId());
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
- new ArrayList<>());
|
|
|
+ new HashMap<>());
|
|
|
}
|
|
|
List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
|
|
|
boolean hasLowRedundancyBlocks = false;
|
|
@@ -432,7 +432,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
+ "So, ignoring to move the blocks");
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
- new ArrayList<>());
|
|
|
+ new HashMap<>());
|
|
|
}
|
|
|
} else {
|
|
|
expectedStorageTypes = existingStoragePolicy
|
|
@@ -465,13 +465,21 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
&& status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
|
|
|
status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
}
|
|
|
- List<Block> assignedBlockIds = new ArrayList<Block>();
|
|
|
+ Map<Block, Set<StorageTypeNodePair>> assignedBlocks = new HashMap<>();
|
|
|
for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
|
|
|
// Check for at least one block storage movement has been chosen
|
|
|
try {
|
|
|
blockMoveTaskHandler.submitMoveTask(blkMovingInfo);
|
|
|
LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
|
|
|
- assignedBlockIds.add(blkMovingInfo.getBlock());
|
|
|
+ StorageTypeNodePair nodeStorage = new StorageTypeNodePair(
|
|
|
+ blkMovingInfo.getTargetStorageType(), blkMovingInfo.getTarget());
|
|
|
+ Set<StorageTypeNodePair> nodesWithStorage = assignedBlocks
|
|
|
+ .get(blkMovingInfo.getBlock());
|
|
|
+ if (nodesWithStorage == null) {
|
|
|
+ nodesWithStorage = new HashSet<>();
|
|
|
+ assignedBlocks.put(blkMovingInfo.getBlock(), nodesWithStorage);
|
|
|
+ }
|
|
|
+ nodesWithStorage.add(nodeStorage);
|
|
|
blockCount++;
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Exception while scheduling movement task", e);
|
|
@@ -479,7 +487,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
status = BlocksMovingAnalysis.Status.BLOCKS_FAILED_TO_MOVE;
|
|
|
}
|
|
|
}
|
|
|
- return new BlocksMovingAnalysis(status, assignedBlockIds);
|
|
|
+ return new BlocksMovingAnalysis(status, assignedBlocks);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -545,6 +553,11 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
new ArrayList<StorageTypeNodePair>();
|
|
|
List<DatanodeInfo> existingBlockStorages = new ArrayList<DatanodeInfo>(
|
|
|
Arrays.asList(storages));
|
|
|
+
|
|
|
+ // Add existing storages into exclude nodes to avoid choosing this as
|
|
|
+ // remote target later.
|
|
|
+ List<DatanodeInfo> excludeNodes = new ArrayList<>(existingBlockStorages);
|
|
|
+
|
|
|
// if expected type exists in source node already, local movement would be
|
|
|
// possible, so lets find such sources first.
|
|
|
Iterator<DatanodeInfo> iterator = existingBlockStorages.iterator();
|
|
@@ -582,7 +595,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
|
|
|
blockMovingInfos, blockInfo, sourceWithStorageMap,
|
|
|
expectedStorageTypes, targetDns,
|
|
|
- ecPolicy);
|
|
|
+ ecPolicy, excludeNodes);
|
|
|
}
|
|
|
return foundMatchingTargetNodesForBlock;
|
|
|
}
|
|
@@ -601,6 +614,10 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
* - Expecting storages to move
|
|
|
* @param targetDns
|
|
|
* - Available DNs for expected storage types
|
|
|
+ * @param ecPolicy
|
|
|
+ * - erasure coding policy of sps invoked file
|
|
|
+ * @param excludeNodes
|
|
|
+ * - existing source nodes, which has replica copy
|
|
|
* @return false if some of the block locations failed to find target node to
|
|
|
* satisfy the storage policy
|
|
|
*/
|
|
@@ -609,9 +626,8 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
List<StorageTypeNodePair> sourceWithStorageList,
|
|
|
List<StorageType> expectedTypes,
|
|
|
EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>> targetDns,
|
|
|
- ErasureCodingPolicy ecPolicy) {
|
|
|
+ ErasureCodingPolicy ecPolicy, List<DatanodeInfo> excludeNodes) {
|
|
|
boolean foundMatchingTargetNodesForBlock = true;
|
|
|
- List<DatanodeInfo> excludeNodes = new ArrayList<>();
|
|
|
|
|
|
// Looping over all the source node locations and choose the target
|
|
|
// storage within same node if possible. This is done separately to
|
|
@@ -638,10 +654,12 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
expectedTypes.remove(chosenTarget.storageType);
|
|
|
}
|
|
|
}
|
|
|
- // To avoid choosing this excludeNodes as targets later
|
|
|
- excludeNodes.add(existingTypeNodePair.dn);
|
|
|
}
|
|
|
-
|
|
|
+ // If all the sources and targets are paired within same node, then simply
|
|
|
+ // return.
|
|
|
+ if (expectedTypes.size() <= 0) {
|
|
|
+ return foundMatchingTargetNodesForBlock;
|
|
|
+ }
|
|
|
// Looping over all the source node locations. Choose a remote target
|
|
|
// storage node if it was not found out within same node.
|
|
|
for (int i = 0; i < sourceWithStorageList.size(); i++) {
|
|
@@ -824,14 +842,29 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
/**
|
|
|
* Keeps datanode with its respective storage type.
|
|
|
*/
|
|
|
- private static final class StorageTypeNodePair {
|
|
|
+ static final class StorageTypeNodePair {
|
|
|
private final StorageType storageType;
|
|
|
private final DatanodeInfo dn;
|
|
|
|
|
|
- private StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
|
|
|
+ StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
|
|
|
this.storageType = storageType;
|
|
|
this.dn = dn;
|
|
|
}
|
|
|
+
|
|
|
+ public DatanodeInfo getDatanodeInfo() {
|
|
|
+ return dn;
|
|
|
+ }
|
|
|
+
|
|
|
+ public StorageType getStorageType() {
|
|
|
+ return storageType;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String toString() {
|
|
|
+ return new StringBuilder().append("StorageTypeNodePair(\n ")
|
|
|
+ .append("DatanodeInfo: ").append(dn).append(", StorageType: ")
|
|
|
+ .append(storageType).toString();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private EnumMap<StorageType, List<DatanodeWithStorage.StorageDetails>>
|
|
@@ -1043,18 +1076,19 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Receives set of storage movement attempt finished blocks report.
|
|
|
+ * Receives storage movement attempt finished block report.
|
|
|
*
|
|
|
- * @param moveAttemptFinishedBlks
|
|
|
- * set of storage movement attempt finished blocks.
|
|
|
+ * @param dnInfo
|
|
|
+ * reported datanode
|
|
|
+ * @param storageType
|
|
|
+ * - storage type
|
|
|
+ * @param block
|
|
|
+ * movement attempt finished block.
|
|
|
*/
|
|
|
- public void notifyStorageMovementAttemptFinishedBlks(
|
|
|
- BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
|
|
|
- if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
|
|
|
- return;
|
|
|
- }
|
|
|
- storageMovementsMonitor
|
|
|
- .notifyMovementTriedBlocks(moveAttemptFinishedBlks.getBlocks());
|
|
|
+ @Override
|
|
|
+ public void notifyStorageMovementAttemptFinishedBlk(DatanodeInfo dnInfo,
|
|
|
+ StorageType storageType, Block block) {
|
|
|
+ storageMovementsMonitor.notifyReportedBlock(dnInfo, storageType, block);
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -1086,7 +1120,7 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
*/
|
|
|
final static class AttemptedItemInfo<T> extends ItemInfo<T> {
|
|
|
private long lastAttemptedOrReportedTime;
|
|
|
- private final List<Block> blocks;
|
|
|
+ private final Set<Block> blocks;
|
|
|
|
|
|
/**
|
|
|
* AttemptedItemInfo constructor.
|
|
@@ -1097,10 +1131,14 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
* trackId for file.
|
|
|
* @param lastAttemptedOrReportedTime
|
|
|
* last attempted or reported time
|
|
|
+ * @param blocks
|
|
|
+ * scheduled blocks
|
|
|
+ * @param retryCount
|
|
|
+ * file retry count
|
|
|
*/
|
|
|
AttemptedItemInfo(T rootId, T trackId,
|
|
|
long lastAttemptedOrReportedTime,
|
|
|
- List<Block> blocks, int retryCount) {
|
|
|
+ Set<Block> blocks, int retryCount) {
|
|
|
super(rootId, trackId, retryCount);
|
|
|
this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
|
|
|
this.blocks = blocks;
|
|
@@ -1121,10 +1159,9 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
this.lastAttemptedOrReportedTime = monotonicNow();
|
|
|
}
|
|
|
|
|
|
- List<Block> getBlocks() {
|
|
|
+ Set<Block> getBlocks() {
|
|
|
return this.blocks;
|
|
|
}
|
|
|
-
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1241,4 +1278,15 @@ public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
"It should be a positive, non-zero integer value.");
|
|
|
return spsWorkMultiplier;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Sets external listener for testing.
|
|
|
+ *
|
|
|
+ * @param blkMovementListener
|
|
|
+ * block movement listener callback object
|
|
|
+ */
|
|
|
+ @VisibleForTesting
|
|
|
+ void setBlockMovementListener(BlockMovementListener blkMovementListener) {
|
|
|
+ storageMovementsMonitor.setBlockMovementListener(blkMovementListener);
|
|
|
+ }
|
|
|
}
|