|
@@ -17,6 +17,8 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
+import static org.apache.hadoop.util.Time.monotonicNow;
|
|
|
+
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collections;
|
|
@@ -44,7 +46,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlockStorageMovementCommand.BlockMovingInfo;
|
|
|
-import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMovementResult;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlocksStorageMoveAttemptFinished;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
@@ -82,25 +84,38 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
/**
|
|
|
* Represents the collective analysis status for all blocks.
|
|
|
*/
|
|
|
- private enum BlocksMovingAnalysisStatus {
|
|
|
- // Represents that, the analysis skipped due to some conditions. A such
|
|
|
- // condition is if block collection is in incomplete state.
|
|
|
- ANALYSIS_SKIPPED_FOR_RETRY,
|
|
|
- // Represents that, all block storage movement needed blocks found its
|
|
|
- // targets.
|
|
|
- ALL_BLOCKS_TARGETS_PAIRED,
|
|
|
- // Represents that, only fewer or none of the block storage movement needed
|
|
|
- // block found its eligible targets.
|
|
|
- FEW_BLOCKS_TARGETS_PAIRED,
|
|
|
- // Represents that, none of the blocks found for block storage movements.
|
|
|
- BLOCKS_ALREADY_SATISFIED,
|
|
|
- // Represents that, the analysis skipped due to some conditions.
|
|
|
- // Example conditions are if no blocks really exists in block collection or
|
|
|
- // if analysis is not required on ec files with unsuitable storage policies
|
|
|
- BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
- // Represents that, All the reported blocks are satisfied the policy but
|
|
|
- // some of the blocks are low redundant.
|
|
|
- FEW_LOW_REDUNDANCY_BLOCKS
|
|
|
+ private static class BlocksMovingAnalysis {
|
|
|
+
|
|
|
+ enum Status {
|
|
|
+ // Represents that, the analysis skipped due to some conditions. A such
|
|
|
+ // condition is if block collection is in incomplete state.
|
|
|
+ ANALYSIS_SKIPPED_FOR_RETRY,
|
|
|
+ // Represents that few or all blocks found respective target to do
|
|
|
+ // the storage movement.
|
|
|
+ BLOCKS_TARGETS_PAIRED,
|
|
|
+ // Represents that none of the blocks found respective target to do
|
|
|
+ // the storage movement.
|
|
|
+ NO_BLOCKS_TARGETS_PAIRED,
|
|
|
+ // Represents that, none of the blocks found for block storage movements.
|
|
|
+ BLOCKS_ALREADY_SATISFIED,
|
|
|
+ // Represents that, the analysis skipped due to some conditions.
|
|
|
+ // Example conditions are if no blocks really exists in block collection
|
|
|
+ // or
|
|
|
+ // if analysis is not required on ec files with unsuitable storage
|
|
|
+ // policies
|
|
|
+ BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
+ // Represents that, All the reported blocks are satisfied the policy but
|
|
|
+ // some of the blocks are low redundant.
|
|
|
+ FEW_LOW_REDUNDANCY_BLOCKS
|
|
|
+ }
|
|
|
+
|
|
|
+ private Status status = null;
|
|
|
+ private List<Block> assignedBlocks = null;
|
|
|
+
|
|
|
+ BlocksMovingAnalysis(Status status, List<Block> blockMovingInfo) {
|
|
|
+ this.status = status;
|
|
|
+ this.assignedBlocks = blockMovingInfo;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public StoragePolicySatisfier(final Namesystem namesystem,
|
|
@@ -118,8 +133,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
conf.getLong(
|
|
|
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
|
|
|
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
|
|
|
- storageMovementNeeded,
|
|
|
- this);
|
|
|
+ storageMovementNeeded);
|
|
|
this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
|
|
|
}
|
|
|
|
|
@@ -232,21 +246,25 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
namesystem.getBlockCollection(trackId);
|
|
|
// Check blockCollectionId existence.
|
|
|
if (blockCollection != null) {
|
|
|
- BlocksMovingAnalysisStatus status =
|
|
|
+ BlocksMovingAnalysis status =
|
|
|
analyseBlocksStorageMovementsAndAssignToDN(blockCollection);
|
|
|
- switch (status) {
|
|
|
+ switch (status.status) {
|
|
|
// Just add to monitor, so it will be retried after timeout
|
|
|
case ANALYSIS_SKIPPED_FOR_RETRY:
|
|
|
- // Just add to monitor, so it will be tracked for result and
|
|
|
- // be removed on successful storage movement result.
|
|
|
- case ALL_BLOCKS_TARGETS_PAIRED:
|
|
|
- this.storageMovementsMonitor.add(itemInfo, true);
|
|
|
+ // Just add to monitor, so it will be tracked for report and
|
|
|
+ // be removed on storage movement attempt finished report.
|
|
|
+ case BLOCKS_TARGETS_PAIRED:
|
|
|
+ this.storageMovementsMonitor.add(new AttemptedItemInfo(
|
|
|
+ itemInfo.getStartId(), itemInfo.getTrackId(),
|
|
|
+ monotonicNow(), status.assignedBlocks));
|
|
|
break;
|
|
|
- // Add to monitor with allBlcoksAttemptedToSatisfy flag false, so
|
|
|
- // that it will be tracked and still it will be consider for retry
|
|
|
- // as analysis was not found targets for storage movement blocks.
|
|
|
- case FEW_BLOCKS_TARGETS_PAIRED:
|
|
|
- this.storageMovementsMonitor.add(itemInfo, false);
|
|
|
+ case NO_BLOCKS_TARGETS_PAIRED:
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Adding trackID " + trackId
|
|
|
+ + " back to retry queue as none of the blocks"
|
|
|
+ + " found its eligible targets.");
|
|
|
+ }
|
|
|
+ this.storageMovementNeeded.add(itemInfo);
|
|
|
break;
|
|
|
case FEW_LOW_REDUNDANCY_BLOCKS:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -310,10 +328,10 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- private BlocksMovingAnalysisStatus analyseBlocksStorageMovementsAndAssignToDN(
|
|
|
+ private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
|
|
|
BlockCollection blockCollection) {
|
|
|
- BlocksMovingAnalysisStatus status =
|
|
|
- BlocksMovingAnalysisStatus.BLOCKS_ALREADY_SATISFIED;
|
|
|
+ BlocksMovingAnalysis.Status status =
|
|
|
+ BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
|
|
|
byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
|
|
|
BlockStoragePolicy existingStoragePolicy =
|
|
|
blockManager.getStoragePolicy(existingStoragePolicyID);
|
|
@@ -322,17 +340,18 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
// So, should we add back? or leave it to user
|
|
|
LOG.info("BlockCollectionID: {} file is under construction. So, postpone"
|
|
|
+ " this to the next retry iteration", blockCollection.getId());
|
|
|
- return BlocksMovingAnalysisStatus.ANALYSIS_SKIPPED_FOR_RETRY;
|
|
|
+ return new BlocksMovingAnalysis(
|
|
|
+ BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
|
|
|
+ new ArrayList<>());
|
|
|
}
|
|
|
|
|
|
- // First datanode will be chosen as the co-ordinator node for storage
|
|
|
- // movements. Later this can be optimized if needed.
|
|
|
- DatanodeDescriptor coordinatorNode = null;
|
|
|
BlockInfo[] blocks = blockCollection.getBlocks();
|
|
|
if (blocks.length == 0) {
|
|
|
LOG.info("BlockCollectionID: {} file is not having any blocks."
|
|
|
+ " So, skipping the analysis.", blockCollection.getId());
|
|
|
- return BlocksMovingAnalysisStatus.BLOCKS_TARGET_PAIRING_SKIPPED;
|
|
|
+ return new BlocksMovingAnalysis(
|
|
|
+ BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
+ new ArrayList<>());
|
|
|
}
|
|
|
List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
|
|
|
|
|
@@ -352,7 +371,9 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
LOG.warn("The storage policy " + existingStoragePolicy.getName()
|
|
|
+ " is not suitable for Striped EC files. "
|
|
|
+ "So, ignoring to move the blocks");
|
|
|
- return BlocksMovingAnalysisStatus.BLOCKS_TARGET_PAIRING_SKIPPED;
|
|
|
+ return new BlocksMovingAnalysis(
|
|
|
+ BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
+ new ArrayList<>());
|
|
|
}
|
|
|
} else {
|
|
|
expectedStorageTypes = existingStoragePolicy
|
|
@@ -370,30 +391,35 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
new LinkedList<StorageType>(Arrays.asList(storageTypes));
|
|
|
if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
|
|
|
existing, true)) {
|
|
|
- boolean computeStatus = computeBlockMovingInfos(blockMovingInfos,
|
|
|
+ boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
|
|
|
blockInfo, expectedStorageTypes, existing, storages);
|
|
|
- if (computeStatus
|
|
|
- && status != BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED
|
|
|
- && !blockManager.hasLowRedundancyBlocks(blockCollection)) {
|
|
|
- status = BlocksMovingAnalysisStatus.ALL_BLOCKS_TARGETS_PAIRED;
|
|
|
+ if (blocksPaired) {
|
|
|
+ status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
|
|
|
} else {
|
|
|
- status = BlocksMovingAnalysisStatus.FEW_BLOCKS_TARGETS_PAIRED;
|
|
|
+ // none of the blocks found its eligible targets for satisfying the
|
|
|
+ // storage policy.
|
|
|
+ status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
|
|
|
}
|
|
|
} else {
|
|
|
if (blockManager.hasLowRedundancyBlocks(blockCollection)) {
|
|
|
- status = BlocksMovingAnalysisStatus.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
+ status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(),
|
|
|
- blockMovingInfos, coordinatorNode);
|
|
|
- int count = 0;
|
|
|
+ List<Block> assignedBlockIds = new ArrayList<Block>();
|
|
|
for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
|
|
|
- count = count + blkMovingInfo.getSources().length;
|
|
|
+ // Check for at least one block storage movement has been chosen
|
|
|
+ if (blkMovingInfo.getTarget() != null) {
|
|
|
+ // assign block storage movement task to the target node
|
|
|
+ ((DatanodeDescriptor) blkMovingInfo.getTarget())
|
|
|
+ .addBlocksToMoveStorage(blkMovingInfo);
|
|
|
+ LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
|
|
|
+ assignedBlockIds.add(blkMovingInfo.getBlock());
|
|
|
+ blockCount++;
|
|
|
+ }
|
|
|
}
|
|
|
- blockCount = blockCount + count;
|
|
|
- return status;
|
|
|
+ return new BlocksMovingAnalysis(status, assignedBlockIds);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -468,41 +494,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return foundMatchingTargetNodesForBlock;
|
|
|
}
|
|
|
|
|
|
- private void assignBlockMovingInfosToCoordinatorDn(long blockCollectionID,
|
|
|
- List<BlockMovingInfo> blockMovingInfos,
|
|
|
- DatanodeDescriptor coordinatorNode) {
|
|
|
-
|
|
|
- if (blockMovingInfos.size() < 1) {
|
|
|
- // TODO: Major: handle this case. I think we need retry cases to
|
|
|
- // be implemented. Idea is, if some files are not getting storage movement
|
|
|
- // chances, then we can just retry limited number of times and exit.
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // For now, first datanode will be chosen as the co-ordinator. Later
|
|
|
- // this can be optimized if needed.
|
|
|
- coordinatorNode = (DatanodeDescriptor) blockMovingInfos.get(0)
|
|
|
- .getSources()[0];
|
|
|
-
|
|
|
- boolean needBlockStorageMovement = false;
|
|
|
- for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
|
|
|
- // Check for atleast one block storage movement has been chosen
|
|
|
- if (blkMovingInfo.getTargets().length > 0){
|
|
|
- needBlockStorageMovement = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- if (!needBlockStorageMovement) {
|
|
|
- // Simply return as there is no targets selected for scheduling the block
|
|
|
- // movement.
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // 'BlockCollectionId' is used as the tracking ID. All the blocks under this
|
|
|
- // blockCollectionID will be added to this datanode.
|
|
|
- coordinatorNode.addBlocksToMoveStorage(blockCollectionID, blockMovingInfos);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Find the good target node for each source node for which block storages was
|
|
|
* misplaced.
|
|
@@ -526,10 +517,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
List<StorageType> expected,
|
|
|
StorageTypeNodeMap locsForExpectedStorageTypes) {
|
|
|
boolean foundMatchingTargetNodesForBlock = true;
|
|
|
- List<DatanodeInfo> sourceNodes = new ArrayList<>();
|
|
|
- List<StorageType> sourceStorageTypes = new ArrayList<>();
|
|
|
- List<DatanodeInfo> targetNodes = new ArrayList<>();
|
|
|
- List<StorageType> targetStorageTypes = new ArrayList<>();
|
|
|
List<DatanodeDescriptor> excludeNodes = new ArrayList<>();
|
|
|
|
|
|
// Looping over all the source node locations and choose the target
|
|
@@ -544,10 +531,15 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
|
|
|
blockInfo, existingTypeNodePair.dn, expected);
|
|
|
if (chosenTarget != null) {
|
|
|
- sourceNodes.add(existingTypeNodePair.dn);
|
|
|
- sourceStorageTypes.add(existingTypeNodePair.storageType);
|
|
|
- targetNodes.add(chosenTarget.dn);
|
|
|
- targetStorageTypes.add(chosenTarget.storageType);
|
|
|
+ if (blockInfo.isStriped()) {
|
|
|
+ buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
+ existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
+ chosenTarget.storageType, blockMovingInfos);
|
|
|
+ } else {
|
|
|
+ buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
+ existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
+ chosenTarget.storageType, blockMovingInfos);
|
|
|
+ }
|
|
|
expected.remove(chosenTarget.storageType);
|
|
|
// TODO: We can increment scheduled block count for this node?
|
|
|
}
|
|
@@ -563,7 +555,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
StorageTypeNodePair chosenTarget = null;
|
|
|
// Chosen the target storage within same datanode. So just skipping this
|
|
|
// source node.
|
|
|
- if (sourceNodes.contains(existingTypeNodePair.dn)) {
|
|
|
+ if (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
|
|
|
continue;
|
|
|
}
|
|
|
if (chosenTarget == null && blockManager.getDatanodeManager()
|
|
@@ -586,10 +578,16 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
Matcher.ANY_OTHER, locsForExpectedStorageTypes, excludeNodes);
|
|
|
}
|
|
|
if (null != chosenTarget) {
|
|
|
- sourceNodes.add(existingTypeNodePair.dn);
|
|
|
- sourceStorageTypes.add(existingTypeNodePair.storageType);
|
|
|
- targetNodes.add(chosenTarget.dn);
|
|
|
- targetStorageTypes.add(chosenTarget.storageType);
|
|
|
+ if (blockInfo.isStriped()) {
|
|
|
+ buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
+ existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
+ chosenTarget.storageType, blockMovingInfos);
|
|
|
+ } else {
|
|
|
+ buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
+ existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
+ chosenTarget.storageType, blockMovingInfos);
|
|
|
+ }
|
|
|
+
|
|
|
expected.remove(chosenTarget.storageType);
|
|
|
excludeNodes.add(chosenTarget.dn);
|
|
|
// TODO: We can increment scheduled block count for this node?
|
|
@@ -605,47 +603,33 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
foundMatchingTargetNodesForBlock = false;
|
|
|
}
|
|
|
|
|
|
- blockMovingInfos.addAll(getBlockMovingInfos(blockInfo, sourceNodes,
|
|
|
- sourceStorageTypes, targetNodes, targetStorageTypes));
|
|
|
return foundMatchingTargetNodesForBlock;
|
|
|
}
|
|
|
|
|
|
- private List<BlockMovingInfo> getBlockMovingInfos(BlockInfo blockInfo,
|
|
|
- List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
|
|
|
- List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes) {
|
|
|
- List<BlockMovingInfo> blkMovingInfos = new ArrayList<>();
|
|
|
- // No source-target node pair exists.
|
|
|
- if (sourceNodes.size() <= 0) {
|
|
|
- return blkMovingInfos;
|
|
|
- }
|
|
|
-
|
|
|
- if (blockInfo.isStriped()) {
|
|
|
- buildStripedBlockMovingInfos(blockInfo, sourceNodes, sourceStorageTypes,
|
|
|
- targetNodes, targetStorageTypes, blkMovingInfos);
|
|
|
- } else {
|
|
|
- buildContinuousBlockMovingInfos(blockInfo, sourceNodes,
|
|
|
- sourceStorageTypes, targetNodes, targetStorageTypes, blkMovingInfos);
|
|
|
+ private boolean checkIfAlreadyChosen(List<BlockMovingInfo> blockMovingInfos,
|
|
|
+ DatanodeDescriptor dn) {
|
|
|
+ for (BlockMovingInfo blockMovingInfo : blockMovingInfos) {
|
|
|
+ if (blockMovingInfo.getSource().equals(dn)) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
}
|
|
|
- return blkMovingInfos;
|
|
|
+ return false;
|
|
|
}
|
|
|
|
|
|
private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
|
|
|
- List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
|
|
|
- List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
|
|
|
+ DatanodeInfo sourceNode, StorageType sourceStorageType,
|
|
|
+ DatanodeInfo targetNode, StorageType targetStorageType,
|
|
|
List<BlockMovingInfo> blkMovingInfos) {
|
|
|
Block blk = new Block(blockInfo.getBlockId(), blockInfo.getNumBytes(),
|
|
|
blockInfo.getGenerationStamp());
|
|
|
- BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk,
|
|
|
- sourceNodes.toArray(new DatanodeInfo[sourceNodes.size()]),
|
|
|
- targetNodes.toArray(new DatanodeInfo[targetNodes.size()]),
|
|
|
- sourceStorageTypes.toArray(new StorageType[sourceStorageTypes.size()]),
|
|
|
- targetStorageTypes.toArray(new StorageType[targetStorageTypes.size()]));
|
|
|
+ BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
|
|
|
+ targetNode, sourceStorageType, targetStorageType);
|
|
|
blkMovingInfos.add(blkMovingInfo);
|
|
|
}
|
|
|
|
|
|
private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
|
|
|
- List<DatanodeInfo> sourceNodes, List<StorageType> sourceStorageTypes,
|
|
|
- List<DatanodeInfo> targetNodes, List<StorageType> targetStorageTypes,
|
|
|
+ DatanodeInfo sourceNode, StorageType sourceStorageType,
|
|
|
+ DatanodeInfo targetNode, StorageType targetStorageType,
|
|
|
List<BlockMovingInfo> blkMovingInfos) {
|
|
|
// For a striped block, it needs to construct internal block at the given
|
|
|
// index of a block group. Here it is iterating over all the block indices
|
|
@@ -655,30 +639,17 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
for (StorageAndBlockIndex si : sBlockInfo.getStorageAndIndexInfos()) {
|
|
|
if (si.getBlockIndex() >= 0) {
|
|
|
DatanodeDescriptor dn = si.getStorage().getDatanodeDescriptor();
|
|
|
- DatanodeInfo[] srcNode = new DatanodeInfo[1];
|
|
|
- StorageType[] srcStorageType = new StorageType[1];
|
|
|
- DatanodeInfo[] targetNode = new DatanodeInfo[1];
|
|
|
- StorageType[] targetStorageType = new StorageType[1];
|
|
|
- for (int i = 0; i < sourceNodes.size(); i++) {
|
|
|
- DatanodeInfo node = sourceNodes.get(i);
|
|
|
- if (node.equals(dn)) {
|
|
|
- srcNode[0] = node;
|
|
|
- srcStorageType[0] = sourceStorageTypes.get(i);
|
|
|
- targetNode[0] = targetNodes.get(i);
|
|
|
- targetStorageType[0] = targetStorageTypes.get(i);
|
|
|
-
|
|
|
- // construct internal block
|
|
|
- long blockId = blockInfo.getBlockId() + si.getBlockIndex();
|
|
|
- long numBytes = StripedBlockUtil.getInternalBlockLength(
|
|
|
- sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
|
|
|
- sBlockInfo.getDataBlockNum(), si.getBlockIndex());
|
|
|
- Block blk = new Block(blockId, numBytes,
|
|
|
- blockInfo.getGenerationStamp());
|
|
|
- BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, srcNode,
|
|
|
- targetNode, srcStorageType, targetStorageType);
|
|
|
- blkMovingInfos.add(blkMovingInfo);
|
|
|
- break; // found matching source-target nodes
|
|
|
- }
|
|
|
+ if (sourceNode.equals(dn)) {
|
|
|
+ // construct internal block
|
|
|
+ long blockId = blockInfo.getBlockId() + si.getBlockIndex();
|
|
|
+ long numBytes = StripedBlockUtil.getInternalBlockLength(
|
|
|
+ sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
|
|
|
+ sBlockInfo.getDataBlockNum(), si.getBlockIndex());
|
|
|
+ Block blk = new Block(blockId, numBytes,
|
|
|
+ blockInfo.getGenerationStamp());
|
|
|
+ BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
|
|
|
+ targetNode, sourceStorageType, targetStorageType);
|
|
|
+ blkMovingInfos.add(blkMovingInfo);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -817,18 +788,18 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Receives the movement results of collection of blocks associated to a
|
|
|
- * trackId.
|
|
|
+ * Receives set of storage movement attempt finished blocks report.
|
|
|
*
|
|
|
- * @param blksMovementResults
|
|
|
- * movement status of the set of blocks associated to a trackId.
|
|
|
+ * @param moveAttemptFinishedBlks
|
|
|
+ * set of storage movement attempt finished blocks.
|
|
|
*/
|
|
|
- void handleBlocksStorageMovementResults(
|
|
|
- BlocksStorageMovementResult[] blksMovementResults) {
|
|
|
- if (blksMovementResults.length <= 0) {
|
|
|
+ void handleStorageMovementAttemptFinishedBlks(
|
|
|
+ BlocksStorageMoveAttemptFinished moveAttemptFinishedBlks) {
|
|
|
+ if (moveAttemptFinishedBlks.getBlocks().length <= 0) {
|
|
|
return;
|
|
|
}
|
|
|
- storageMovementsMonitor.addResults(blksMovementResults);
|
|
|
+ storageMovementsMonitor
|
|
|
+ .addReportedMovedBlocks(moveAttemptFinishedBlks.getBlocks());
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -906,4 +877,52 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return (startId != trackId);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This class contains information of an attempted blocks and its last
|
|
|
+ * attempted or reported time stamp. This is used by
|
|
|
+ * {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
|
|
|
+ */
|
|
|
+ final static class AttemptedItemInfo extends ItemInfo {
|
|
|
+ private long lastAttemptedOrReportedTime;
|
|
|
+ private final List<Block> blocks;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * AttemptedItemInfo constructor.
|
|
|
+ *
|
|
|
+ * @param rootId
|
|
|
+ * rootId for trackId
|
|
|
+ * @param trackId
|
|
|
+ * trackId for file.
|
|
|
+ * @param lastAttemptedOrReportedTime
|
|
|
+ * last attempted or reported time
|
|
|
+ */
|
|
|
+ AttemptedItemInfo(long rootId, long trackId,
|
|
|
+ long lastAttemptedOrReportedTime,
|
|
|
+ List<Block> blocks) {
|
|
|
+ super(rootId, trackId);
|
|
|
+ this.lastAttemptedOrReportedTime = lastAttemptedOrReportedTime;
|
|
|
+ this.blocks = blocks;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @return last attempted or reported time stamp.
|
|
|
+ */
|
|
|
+ long getLastAttemptedOrReportedTime() {
|
|
|
+ return lastAttemptedOrReportedTime;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Update lastAttemptedOrReportedTime, so that the expiration time will be
|
|
|
+ * postponed to future.
|
|
|
+ */
|
|
|
+ void touchLastReportedTimeStamp() {
|
|
|
+ this.lastAttemptedOrReportedTime = monotonicNow();
|
|
|
+ }
|
|
|
+
|
|
|
+ List<Block> getBlocks() {
|
|
|
+ return this.blocks;
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
}
|