|
@@ -29,29 +29,28 @@ import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfyPathStatus;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.balancer.Matcher;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped.StorageAndBlockIndex;
|
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
-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.namenode.ErasureCodingPolicyManager;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
|
|
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;
|
|
@@ -79,8 +78,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(StoragePolicySatisfier.class);
|
|
|
private Daemon storagePolicySatisfierThread;
|
|
|
- private final Namesystem namesystem;
|
|
|
- private final BlockManager blockManager;
|
|
|
private final BlockStorageMovementNeeded storageMovementNeeded;
|
|
|
private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
|
|
|
private volatile boolean isRunning = false;
|
|
@@ -89,16 +86,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
private int blockMovementMaxRetry;
|
|
|
private final Context ctxt;
|
|
|
|
|
|
- /**
|
|
|
- * An interface for analyzing and assigning the block storage movements to
|
|
|
- * worker nodes.
|
|
|
- */
|
|
|
- // TODO: Now, added one API which is required for sps package. Will refine
|
|
|
- // this interface via HDFS-12911.
|
|
|
- public interface Context {
|
|
|
- int getNumLiveDataNodes();
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Represents the collective analysis status for all blocks.
|
|
|
*/
|
|
@@ -124,7 +111,9 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
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
|
|
|
+ FEW_LOW_REDUNDANCY_BLOCKS,
|
|
|
+ // Represents that, movement failures due to unexpected errors.
|
|
|
+ BLOCKS_FAILED_TO_MOVE
|
|
|
}
|
|
|
|
|
|
private Status status = null;
|
|
@@ -136,36 +125,27 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public StoragePolicySatisfier(final Namesystem namesystem,
|
|
|
- final BlockManager blkManager, Configuration conf, Context ctxt) {
|
|
|
- this.namesystem = namesystem;
|
|
|
- this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
|
|
|
- this, conf.getInt(
|
|
|
- DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
|
|
|
- DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT));
|
|
|
- this.blockManager = blkManager;
|
|
|
- this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
|
|
|
- conf.getLong(
|
|
|
- DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY,
|
|
|
- DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT),
|
|
|
- conf.getLong(
|
|
|
- DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_KEY,
|
|
|
- DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
|
|
|
+ public StoragePolicySatisfier(Context ctxt) {
|
|
|
+ this.ctxt = ctxt;
|
|
|
+ this.storageMovementNeeded = new BlockStorageMovementNeeded(ctxt);
|
|
|
+ this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(ctxt,
|
|
|
storageMovementNeeded);
|
|
|
- this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
|
|
|
- this.blockMovementMaxRetry = conf.getInt(
|
|
|
+ this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(ctxt.getConf());
|
|
|
+ this.blockMovementMaxRetry = ctxt.getConf().getInt(
|
|
|
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
|
|
|
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
|
|
|
- this.ctxt = ctxt;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Start storage policy satisfier demon thread. Also start block storage
|
|
|
* movements monitor for retry the attempts if needed.
|
|
|
+ *
|
|
|
+ * // TODO: FSDirectory will get removed via HDFS-12911 modularization work.
|
|
|
*/
|
|
|
- public synchronized void start(boolean reconfigStart) {
|
|
|
+ public synchronized void start(boolean reconfigStart, FSDirectory fsd) {
|
|
|
isRunning = true;
|
|
|
- if (checkIfMoverRunning()) {
|
|
|
+ ctxt.setSPSRunning(this::isRunning);
|
|
|
+ if (ctxt.isMoverRunning()) {
|
|
|
isRunning = false;
|
|
|
LOG.error(
|
|
|
"Stopping StoragePolicySatisfier thread " + "as Mover ID file "
|
|
@@ -183,7 +163,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
// Ensure that all the previously submitted block movements(if any) have to
|
|
|
// be stopped in all datanodes.
|
|
|
addDropSPSWorkCommandsToAllDNs();
|
|
|
- storageMovementNeeded.init();
|
|
|
+ storageMovementNeeded.init(fsd);
|
|
|
storagePolicySatisfierThread = new Daemon(this);
|
|
|
storagePolicySatisfierThread.setName("StoragePolicySatisfier");
|
|
|
storagePolicySatisfierThread.start();
|
|
@@ -199,7 +179,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
*/
|
|
|
public synchronized void disable(boolean forceStop) {
|
|
|
isRunning = false;
|
|
|
-
|
|
|
if (storagePolicySatisfierThread == null) {
|
|
|
return;
|
|
|
}
|
|
@@ -242,25 +221,19 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return isRunning;
|
|
|
}
|
|
|
|
|
|
- // Return true if a Mover instance is running
|
|
|
- private boolean checkIfMoverRunning() {
|
|
|
- String moverId = HdfsServerConstants.MOVER_ID_PATH.toString();
|
|
|
- return namesystem.isFileOpenedForWrite(moverId);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Adding drop commands to all datanodes to stop performing the satisfier
|
|
|
* block movements, if any.
|
|
|
*/
|
|
|
private void addDropSPSWorkCommandsToAllDNs() {
|
|
|
- this.blockManager.getDatanodeManager().addDropSPSWorkCommandsToAllDNs();
|
|
|
+ ctxt.addDropPreviousSPSWorkAtDNs();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void run() {
|
|
|
- while (namesystem.isRunning() && isRunning) {
|
|
|
+ while (ctxt.isRunning()) {
|
|
|
try {
|
|
|
- if (!namesystem.isInSafeMode()) {
|
|
|
+ if (!ctxt.isInSafeMode()) {
|
|
|
ItemInfo itemInfo = storageMovementNeeded.get();
|
|
|
if (itemInfo != null) {
|
|
|
if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
|
|
@@ -271,25 +244,28 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
continue;
|
|
|
}
|
|
|
long trackId = itemInfo.getTrackId();
|
|
|
- BlockCollection blockCollection;
|
|
|
BlocksMovingAnalysis status = null;
|
|
|
- try {
|
|
|
- namesystem.readLock();
|
|
|
- blockCollection = namesystem.getBlockCollection(trackId);
|
|
|
- // Check blockCollectionId existence.
|
|
|
- if (blockCollection == null) {
|
|
|
- // File doesn't exists (maybe got deleted), remove trackId from
|
|
|
- // the queue
|
|
|
- storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
|
|
|
- } else {
|
|
|
- status =
|
|
|
- analyseBlocksStorageMovementsAndAssignToDN(
|
|
|
- blockCollection);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- namesystem.readUnlock();
|
|
|
- }
|
|
|
- if (blockCollection != null) {
|
|
|
+ DatanodeStorageReport[] liveDnReports;
|
|
|
+ BlockStoragePolicy existingStoragePolicy;
|
|
|
+ // TODO: presently, context internally acquire the lock
|
|
|
+ // and returns the result. Need to discuss to move the lock outside?
|
|
|
+ boolean hasLowRedundancyBlocks = ctxt
|
|
|
+ .hasLowRedundancyBlocks(trackId);
|
|
|
+ HdfsFileStatus fileStatus = ctxt.getFileInfo(trackId);
|
|
|
+ // Check path existence.
|
|
|
+ if (fileStatus == null || fileStatus.isDir()) {
|
|
|
+ // File doesn't exists (maybe got deleted) or its a directory,
|
|
|
+ // 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,
|
|
|
+ hasLowRedundancyBlocks, existingStoragePolicy, liveDnReports);
|
|
|
switch (status.status) {
|
|
|
// Just add to monitor, so it will be retried after timeout
|
|
|
case ANALYSIS_SKIPPED_FOR_RETRY:
|
|
@@ -317,6 +293,14 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
this.storageMovementNeeded.add(itemInfo);
|
|
|
break;
|
|
|
+ case BLOCKS_FAILED_TO_MOVE:
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Adding trackID " + trackId
|
|
|
+ + " back to retry queue as some of the blocks"
|
|
|
+ + " movement failed.");
|
|
|
+ }
|
|
|
+ this.storageMovementNeeded.add(itemInfo);
|
|
|
+ break;
|
|
|
// Just clean Xattrs
|
|
|
case BLOCKS_TARGET_PAIRING_SKIPPED:
|
|
|
case BLOCKS_ALREADY_SATISFIED:
|
|
@@ -350,14 +334,11 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
// Stopping monitor thread and clearing queues as well
|
|
|
this.clearQueues();
|
|
|
this.storageMovementsMonitor.stopGracefully();
|
|
|
- if (!namesystem.isRunning()) {
|
|
|
- LOG.info("Stopping StoragePolicySatisfier.");
|
|
|
- if (!(t instanceof InterruptedException)) {
|
|
|
- LOG.info("StoragePolicySatisfier received an exception"
|
|
|
- + " while shutting down.", t);
|
|
|
- }
|
|
|
- return;
|
|
|
+ if (!(t instanceof InterruptedException)) {
|
|
|
+ LOG.info("StoragePolicySatisfier received an exception"
|
|
|
+ + " while shutting down.", t);
|
|
|
}
|
|
|
+ LOG.info("Stopping StoragePolicySatisfier.");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -367,41 +348,43 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
|
|
|
private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
|
|
|
- BlockCollection blockCollection) {
|
|
|
+ HdfsLocatedFileStatus fileInfo, boolean hasLowRedundancyBlocks,
|
|
|
+ BlockStoragePolicy existingStoragePolicy,
|
|
|
+ DatanodeStorageReport[] liveDns) {
|
|
|
BlocksMovingAnalysis.Status status =
|
|
|
BlocksMovingAnalysis.Status.BLOCKS_ALREADY_SATISFIED;
|
|
|
- byte existingStoragePolicyID = blockCollection.getStoragePolicyID();
|
|
|
- BlockStoragePolicy existingStoragePolicy =
|
|
|
- blockManager.getStoragePolicy(existingStoragePolicyID);
|
|
|
- if (!blockCollection.getLastBlock().isComplete()) {
|
|
|
+ final ErasureCodingPolicy ecPolicy = fileInfo.getErasureCodingPolicy();
|
|
|
+ final LocatedBlocks locatedBlocks = fileInfo.getLocatedBlocks();
|
|
|
+ final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
|
|
|
+ if (!lastBlkComplete) {
|
|
|
// Postpone, currently file is under construction
|
|
|
// 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());
|
|
|
+ + " this to the next retry iteration", fileInfo.getFileId());
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
|
|
|
new ArrayList<>());
|
|
|
}
|
|
|
|
|
|
- BlockInfo[] blocks = blockCollection.getBlocks();
|
|
|
- if (blocks.length == 0) {
|
|
|
+ List<LocatedBlock> blocks = locatedBlocks.getLocatedBlocks();
|
|
|
+ if (blocks.size() == 0) {
|
|
|
LOG.info("BlockCollectionID: {} file is not having any blocks."
|
|
|
- + " So, skipping the analysis.", blockCollection.getId());
|
|
|
+ + " So, skipping the analysis.", fileInfo.getFileId());
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
new ArrayList<>());
|
|
|
}
|
|
|
List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
|
|
|
|
|
|
- for (int i = 0; i < blocks.length; i++) {
|
|
|
- BlockInfo blockInfo = blocks[i];
|
|
|
+ for (int i = 0; i < blocks.size(); i++) {
|
|
|
+ LocatedBlock blockInfo = blocks.get(i);
|
|
|
List<StorageType> expectedStorageTypes;
|
|
|
if (blockInfo.isStriped()) {
|
|
|
if (ErasureCodingPolicyManager
|
|
|
.checkStoragePolicySuitableForECStripedMode(
|
|
|
- existingStoragePolicyID)) {
|
|
|
+ existingStoragePolicy.getId())) {
|
|
|
expectedStorageTypes = existingStoragePolicy
|
|
|
- .chooseStorageTypes((short) blockInfo.getCapacity());
|
|
|
+ .chooseStorageTypes((short) blockInfo.getLocations().length);
|
|
|
} else {
|
|
|
// Currently we support only limited policies (HOT, COLD, ALLSSD)
|
|
|
// for EC striped mode files. SPS will ignore to move the blocks if
|
|
@@ -415,22 +398,16 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
} else {
|
|
|
expectedStorageTypes = existingStoragePolicy
|
|
|
- .chooseStorageTypes(blockInfo.getReplication());
|
|
|
+ .chooseStorageTypes(fileInfo.getReplication());
|
|
|
}
|
|
|
|
|
|
- DatanodeStorageInfo[] storages = blockManager.getStorages(blockInfo);
|
|
|
- StorageType[] storageTypes = new StorageType[storages.length];
|
|
|
- for (int j = 0; j < storages.length; j++) {
|
|
|
- DatanodeStorageInfo datanodeStorageInfo = storages[j];
|
|
|
- StorageType storageType = datanodeStorageInfo.getStorageType();
|
|
|
- storageTypes[j] = storageType;
|
|
|
- }
|
|
|
- List<StorageType> existing =
|
|
|
- new LinkedList<StorageType>(Arrays.asList(storageTypes));
|
|
|
+ List<StorageType> existing = new LinkedList<StorageType>(
|
|
|
+ Arrays.asList(blockInfo.getStorageTypes()));
|
|
|
if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
|
|
|
existing, true)) {
|
|
|
boolean blocksPaired = computeBlockMovingInfos(blockMovingInfos,
|
|
|
- blockInfo, expectedStorageTypes, existing, storages);
|
|
|
+ blockInfo, expectedStorageTypes, existing, blockInfo.getLocations(),
|
|
|
+ liveDns, ecPolicy);
|
|
|
if (blocksPaired) {
|
|
|
status = BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED;
|
|
|
} else {
|
|
@@ -439,7 +416,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
|
|
|
}
|
|
|
} else {
|
|
|
- if (blockManager.hasLowRedundancyBlocks(blockCollection)) {
|
|
|
+ if (hasLowRedundancyBlocks) {
|
|
|
status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
}
|
|
|
}
|
|
@@ -448,13 +425,15 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
List<Block> assignedBlockIds = new ArrayList<Block>();
|
|
|
for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
|
|
|
// 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);
|
|
|
+ try {
|
|
|
+ ctxt.assignBlockMoveTaskToTargetNode(blkMovingInfo);
|
|
|
LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
|
|
|
assignedBlockIds.add(blkMovingInfo.getBlock());
|
|
|
blockCount++;
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Exception while scheduling movement task", e);
|
|
|
+ // failed to move the block.
|
|
|
+ status = BlocksMovingAnalysis.Status.BLOCKS_FAILED_TO_MOVE;
|
|
|
}
|
|
|
}
|
|
|
return new BlocksMovingAnalysis(status, assignedBlockIds);
|
|
@@ -481,29 +460,29 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
* satisfy the storage policy, true otherwise
|
|
|
*/
|
|
|
private boolean computeBlockMovingInfos(
|
|
|
- List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
|
|
|
+ List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
|
|
|
List<StorageType> expectedStorageTypes, List<StorageType> existing,
|
|
|
- DatanodeStorageInfo[] storages) {
|
|
|
+ DatanodeInfo[] storages, DatanodeStorageReport[] liveDns,
|
|
|
+ ErasureCodingPolicy ecPolicy) {
|
|
|
boolean foundMatchingTargetNodesForBlock = true;
|
|
|
if (!DFSUtil.removeOverlapBetweenStorageTypes(expectedStorageTypes,
|
|
|
existing, true)) {
|
|
|
List<StorageTypeNodePair> sourceWithStorageMap =
|
|
|
new ArrayList<StorageTypeNodePair>();
|
|
|
- List<DatanodeStorageInfo> existingBlockStorages =
|
|
|
- new ArrayList<DatanodeStorageInfo>(Arrays.asList(storages));
|
|
|
+ List<DatanodeInfo> existingBlockStorages = new ArrayList<DatanodeInfo>(
|
|
|
+ Arrays.asList(storages));
|
|
|
// if expected type exists in source node already, local movement would be
|
|
|
// possible, so lets find such sources first.
|
|
|
- Iterator<DatanodeStorageInfo> iterator = existingBlockStorages.iterator();
|
|
|
+ Iterator<DatanodeInfo> iterator = existingBlockStorages.iterator();
|
|
|
while (iterator.hasNext()) {
|
|
|
- DatanodeStorageInfo datanodeStorageInfo = iterator.next();
|
|
|
- if (checkSourceAndTargetTypeExists(
|
|
|
- datanodeStorageInfo.getDatanodeDescriptor(), existing,
|
|
|
- expectedStorageTypes)) {
|
|
|
+ DatanodeInfoWithStorage dnInfo = (DatanodeInfoWithStorage) iterator
|
|
|
+ .next();
|
|
|
+ if (checkSourceAndTargetTypeExists(dnInfo, existing,
|
|
|
+ expectedStorageTypes, liveDns)) {
|
|
|
sourceWithStorageMap
|
|
|
- .add(new StorageTypeNodePair(datanodeStorageInfo.getStorageType(),
|
|
|
- datanodeStorageInfo.getDatanodeDescriptor()));
|
|
|
+ .add(new StorageTypeNodePair(dnInfo.getStorageType(), dnInfo));
|
|
|
iterator.remove();
|
|
|
- existing.remove(datanodeStorageInfo.getStorageType());
|
|
|
+ existing.remove(dnInfo.getStorageType());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -511,23 +490,25 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
for (StorageType existingType : existing) {
|
|
|
iterator = existingBlockStorages.iterator();
|
|
|
while (iterator.hasNext()) {
|
|
|
- DatanodeStorageInfo datanodeStorageInfo = iterator.next();
|
|
|
- StorageType storageType = datanodeStorageInfo.getStorageType();
|
|
|
+ DatanodeInfoWithStorage dnStorageInfo =
|
|
|
+ (DatanodeInfoWithStorage) iterator.next();
|
|
|
+ StorageType storageType = dnStorageInfo.getStorageType();
|
|
|
if (storageType == existingType) {
|
|
|
iterator.remove();
|
|
|
sourceWithStorageMap.add(new StorageTypeNodePair(storageType,
|
|
|
- datanodeStorageInfo.getDatanodeDescriptor()));
|
|
|
+ dnStorageInfo));
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
StorageTypeNodeMap locsForExpectedStorageTypes =
|
|
|
- findTargetsForExpectedStorageTypes(expectedStorageTypes);
|
|
|
+ findTargetsForExpectedStorageTypes(expectedStorageTypes, liveDns);
|
|
|
|
|
|
foundMatchingTargetNodesForBlock |= findSourceAndTargetToMove(
|
|
|
blockMovingInfos, blockInfo, sourceWithStorageMap,
|
|
|
- expectedStorageTypes, locsForExpectedStorageTypes);
|
|
|
+ expectedStorageTypes, locsForExpectedStorageTypes,
|
|
|
+ ecPolicy);
|
|
|
}
|
|
|
return foundMatchingTargetNodesForBlock;
|
|
|
}
|
|
@@ -550,12 +531,13 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
* satisfy the storage policy
|
|
|
*/
|
|
|
private boolean findSourceAndTargetToMove(
|
|
|
- List<BlockMovingInfo> blockMovingInfos, BlockInfo blockInfo,
|
|
|
+ List<BlockMovingInfo> blockMovingInfos, LocatedBlock blockInfo,
|
|
|
List<StorageTypeNodePair> sourceWithStorageList,
|
|
|
List<StorageType> expected,
|
|
|
- StorageTypeNodeMap locsForExpectedStorageTypes) {
|
|
|
+ StorageTypeNodeMap locsForExpectedStorageTypes,
|
|
|
+ ErasureCodingPolicy ecPolicy) {
|
|
|
boolean foundMatchingTargetNodesForBlock = true;
|
|
|
- List<DatanodeDescriptor> excludeNodes = new ArrayList<>();
|
|
|
+ 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
|
|
@@ -566,13 +548,14 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
// Check whether the block replica is already placed in the expected
|
|
|
// storage type in this source datanode.
|
|
|
if (!expected.contains(existingTypeNodePair.storageType)) {
|
|
|
- StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(
|
|
|
- blockInfo, existingTypeNodePair.dn, expected);
|
|
|
+ StorageTypeNodePair chosenTarget = chooseTargetTypeInSameNode(blockInfo,
|
|
|
+ existingTypeNodePair.dn, expected);
|
|
|
if (chosenTarget != null) {
|
|
|
if (blockInfo.isStriped()) {
|
|
|
buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
- chosenTarget.storageType, blockMovingInfos);
|
|
|
+ chosenTarget.storageType, blockMovingInfos,
|
|
|
+ ecPolicy);
|
|
|
} else {
|
|
|
buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
existingTypeNodePair.storageType, chosenTarget.dn,
|
|
@@ -596,7 +579,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
if (checkIfAlreadyChosen(blockMovingInfos, existingTypeNodePair.dn)) {
|
|
|
continue;
|
|
|
}
|
|
|
- if (chosenTarget == null && blockManager.getDatanodeManager()
|
|
|
+ if (chosenTarget == null && ctxt
|
|
|
.getNetworkTopology().isNodeGroupAware()) {
|
|
|
chosenTarget = chooseTarget(blockInfo, existingTypeNodePair.dn,
|
|
|
expected, Matcher.SAME_NODE_GROUP, locsForExpectedStorageTypes,
|
|
@@ -619,7 +602,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
if (blockInfo.isStriped()) {
|
|
|
buildStripedBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
existingTypeNodePair.storageType, chosenTarget.dn,
|
|
|
- chosenTarget.storageType, blockMovingInfos);
|
|
|
+ chosenTarget.storageType, blockMovingInfos, ecPolicy);
|
|
|
} else {
|
|
|
buildContinuousBlockMovingInfos(blockInfo, existingTypeNodePair.dn,
|
|
|
existingTypeNodePair.storageType, chosenTarget.dn,
|
|
@@ -645,7 +628,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
|
|
|
private boolean checkIfAlreadyChosen(List<BlockMovingInfo> blockMovingInfos,
|
|
|
- DatanodeDescriptor dn) {
|
|
|
+ DatanodeInfo dn) {
|
|
|
for (BlockMovingInfo blockMovingInfo : blockMovingInfos) {
|
|
|
if (blockMovingInfo.getSource().equals(dn)) {
|
|
|
return true;
|
|
@@ -654,37 +637,40 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- private void buildContinuousBlockMovingInfos(BlockInfo blockInfo,
|
|
|
+ private void buildContinuousBlockMovingInfos(LocatedBlock blockInfo,
|
|
|
DatanodeInfo sourceNode, StorageType sourceStorageType,
|
|
|
DatanodeInfo targetNode, StorageType targetStorageType,
|
|
|
List<BlockMovingInfo> blkMovingInfos) {
|
|
|
- Block blk = new Block(blockInfo.getBlockId(), blockInfo.getNumBytes(),
|
|
|
- blockInfo.getGenerationStamp());
|
|
|
+ Block blk = ExtendedBlock.getLocalBlock(blockInfo.getBlock());
|
|
|
BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
|
|
|
targetNode, sourceStorageType, targetStorageType);
|
|
|
blkMovingInfos.add(blkMovingInfo);
|
|
|
}
|
|
|
|
|
|
- private void buildStripedBlockMovingInfos(BlockInfo blockInfo,
|
|
|
+ private void buildStripedBlockMovingInfos(LocatedBlock blockInfo,
|
|
|
DatanodeInfo sourceNode, StorageType sourceStorageType,
|
|
|
DatanodeInfo targetNode, StorageType targetStorageType,
|
|
|
- List<BlockMovingInfo> blkMovingInfos) {
|
|
|
+ List<BlockMovingInfo> blkMovingInfos, ErasureCodingPolicy ecPolicy) {
|
|
|
// 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
|
|
|
// and construct internal blocks which can be then considered for block
|
|
|
// movement.
|
|
|
- BlockInfoStriped sBlockInfo = (BlockInfoStriped) blockInfo;
|
|
|
- for (StorageAndBlockIndex si : sBlockInfo.getStorageAndIndexInfos()) {
|
|
|
- if (si.getBlockIndex() >= 0) {
|
|
|
- DatanodeDescriptor dn = si.getStorage().getDatanodeDescriptor();
|
|
|
- if (sourceNode.equals(dn)) {
|
|
|
+ LocatedStripedBlock sBlockInfo = (LocatedStripedBlock) blockInfo;
|
|
|
+ byte[] indices = sBlockInfo.getBlockIndices();
|
|
|
+ DatanodeInfo[] locations = sBlockInfo.getLocations();
|
|
|
+ for (int i = 0; i < indices.length; i++) {
|
|
|
+ byte blkIndex = indices[i];
|
|
|
+ if (blkIndex >= 0) {
|
|
|
+ // pick block movement only for the given source node.
|
|
|
+ if (sourceNode.equals(locations[i])) {
|
|
|
// construct internal block
|
|
|
- long blockId = blockInfo.getBlockId() + si.getBlockIndex();
|
|
|
+ ExtendedBlock extBlock = sBlockInfo.getBlock();
|
|
|
long numBytes = StripedBlockUtil.getInternalBlockLength(
|
|
|
- sBlockInfo.getNumBytes(), sBlockInfo.getCellSize(),
|
|
|
- sBlockInfo.getDataBlockNum(), si.getBlockIndex());
|
|
|
- Block blk = new Block(blockId, numBytes,
|
|
|
- blockInfo.getGenerationStamp());
|
|
|
+ extBlock.getNumBytes(), ecPolicy, blkIndex);
|
|
|
+ Block blk = new Block(ExtendedBlock.getLocalBlock(extBlock));
|
|
|
+ long blkId = blk.getBlockId() + blkIndex;
|
|
|
+ blk.setBlockId(blkId);
|
|
|
+ blk.setNumBytes(numBytes);
|
|
|
BlockMovingInfo blkMovingInfo = new BlockMovingInfo(blk, sourceNode,
|
|
|
targetNode, sourceStorageType, targetStorageType);
|
|
|
blkMovingInfos.add(blkMovingInfo);
|
|
@@ -703,34 +689,35 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
* @param targetTypes
|
|
|
* - list of target storage types
|
|
|
*/
|
|
|
- private StorageTypeNodePair chooseTargetTypeInSameNode(Block block,
|
|
|
- DatanodeDescriptor source, List<StorageType> targetTypes) {
|
|
|
+ private StorageTypeNodePair chooseTargetTypeInSameNode(LocatedBlock blockInfo,
|
|
|
+ DatanodeInfo source, List<StorageType> targetTypes) {
|
|
|
for (StorageType t : targetTypes) {
|
|
|
- DatanodeStorageInfo chooseStorage4Block =
|
|
|
- source.chooseStorage4Block(t, block.getNumBytes());
|
|
|
- if (chooseStorage4Block != null) {
|
|
|
+ boolean goodTargetDn = ctxt.verifyTargetDatanodeHasSpaceForScheduling(
|
|
|
+ source, t, blockInfo.getBlockSize());
|
|
|
+ if (goodTargetDn) {
|
|
|
return new StorageTypeNodePair(t, source);
|
|
|
}
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- private StorageTypeNodePair chooseTarget(Block block,
|
|
|
- DatanodeDescriptor source, List<StorageType> targetTypes, Matcher matcher,
|
|
|
+ private StorageTypeNodePair chooseTarget(LocatedBlock block,
|
|
|
+ DatanodeInfo source, List<StorageType> targetTypes, Matcher matcher,
|
|
|
StorageTypeNodeMap locsForExpectedStorageTypes,
|
|
|
- List<DatanodeDescriptor> excludeNodes) {
|
|
|
+ List<DatanodeInfo> excludeNodes) {
|
|
|
for (StorageType t : targetTypes) {
|
|
|
- List<DatanodeDescriptor> nodesWithStorages =
|
|
|
- locsForExpectedStorageTypes.getNodesWithStorages(t);
|
|
|
+ List<DatanodeInfo> nodesWithStorages = locsForExpectedStorageTypes
|
|
|
+ .getNodesWithStorages(t);
|
|
|
if (nodesWithStorages == null || nodesWithStorages.isEmpty()) {
|
|
|
continue; // no target nodes with the required storage type.
|
|
|
}
|
|
|
Collections.shuffle(nodesWithStorages);
|
|
|
- for (DatanodeDescriptor target : nodesWithStorages) {
|
|
|
- if (!excludeNodes.contains(target) && matcher.match(
|
|
|
- blockManager.getDatanodeManager().getNetworkTopology(), source,
|
|
|
- target)) {
|
|
|
- if (null != target.chooseStorage4Block(t, block.getNumBytes())) {
|
|
|
+ for (DatanodeInfo target : nodesWithStorages) {
|
|
|
+ if (!excludeNodes.contains(target)
|
|
|
+ && matcher.match(ctxt.getNetworkTopology(), source, target)) {
|
|
|
+ boolean goodTargetDn = ctxt.verifyTargetDatanodeHasSpaceForScheduling(
|
|
|
+ target, t, block.getBlockSize());
|
|
|
+ if (goodTargetDn) {
|
|
|
return new StorageTypeNodePair(t, target);
|
|
|
}
|
|
|
}
|
|
@@ -741,27 +728,25 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
|
|
|
private static class StorageTypeNodePair {
|
|
|
private StorageType storageType = null;
|
|
|
- private DatanodeDescriptor dn = null;
|
|
|
+ private DatanodeInfo dn = null;
|
|
|
|
|
|
- StorageTypeNodePair(StorageType storageType, DatanodeDescriptor dn) {
|
|
|
+ StorageTypeNodePair(StorageType storageType, DatanodeInfo dn) {
|
|
|
this.storageType = storageType;
|
|
|
this.dn = dn;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private StorageTypeNodeMap findTargetsForExpectedStorageTypes(
|
|
|
- List<StorageType> expected) {
|
|
|
+ List<StorageType> expected, DatanodeStorageReport[] liveDns) {
|
|
|
StorageTypeNodeMap targetMap = new StorageTypeNodeMap();
|
|
|
- List<DatanodeDescriptor> reports = blockManager.getDatanodeManager()
|
|
|
- .getDatanodeListForReport(DatanodeReportType.LIVE);
|
|
|
- for (DatanodeDescriptor dn : reports) {
|
|
|
+ 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);
|
|
|
+ targetMap.add(t, dn.getDatanodeInfo());
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -782,32 +767,40 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return max;
|
|
|
}
|
|
|
|
|
|
- private boolean checkSourceAndTargetTypeExists(DatanodeDescriptor dn,
|
|
|
- List<StorageType> existing, List<StorageType> expectedStorageTypes) {
|
|
|
- DatanodeStorageInfo[] allDNStorageInfos = dn.getStorageInfos();
|
|
|
+ private boolean checkSourceAndTargetTypeExists(DatanodeInfo dn,
|
|
|
+ List<StorageType> existing, List<StorageType> expectedStorageTypes,
|
|
|
+ DatanodeStorageReport[] liveDns) {
|
|
|
boolean isExpectedTypeAvailable = false;
|
|
|
boolean isExistingTypeAvailable = false;
|
|
|
- for (DatanodeStorageInfo dnInfo : allDNStorageInfos) {
|
|
|
- StorageType storageType = dnInfo.getStorageType();
|
|
|
- if (existing.contains(storageType)) {
|
|
|
- isExistingTypeAvailable = true;
|
|
|
- }
|
|
|
- if (expectedStorageTypes.contains(storageType)) {
|
|
|
- isExpectedTypeAvailable = true;
|
|
|
+ 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)) {
|
|
|
+ isExistingTypeAvailable = true;
|
|
|
+ }
|
|
|
+ if (expectedStorageTypes.contains(storageType)) {
|
|
|
+ isExpectedTypeAvailable = true;
|
|
|
+ }
|
|
|
+ if (isExistingTypeAvailable && isExpectedTypeAvailable) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
return isExistingTypeAvailable && isExpectedTypeAvailable;
|
|
|
}
|
|
|
|
|
|
private static class StorageTypeNodeMap {
|
|
|
- private final EnumMap<StorageType, List<DatanodeDescriptor>> typeNodeMap =
|
|
|
- new EnumMap<StorageType, List<DatanodeDescriptor>>(StorageType.class);
|
|
|
+ private final EnumMap<StorageType, List<DatanodeInfo>> typeNodeMap =
|
|
|
+ new EnumMap<StorageType, List<DatanodeInfo>>(StorageType.class);
|
|
|
|
|
|
- private void add(StorageType t, DatanodeDescriptor dn) {
|
|
|
- List<DatanodeDescriptor> nodesWithStorages = getNodesWithStorages(t);
|
|
|
- LinkedList<DatanodeDescriptor> value = null;
|
|
|
+ private void add(StorageType t, DatanodeInfo dn) {
|
|
|
+ List<DatanodeInfo> nodesWithStorages = getNodesWithStorages(t);
|
|
|
+ LinkedList<DatanodeInfo> value = null;
|
|
|
if (nodesWithStorages == null) {
|
|
|
- value = new LinkedList<DatanodeDescriptor>();
|
|
|
+ value = new LinkedList<DatanodeInfo>();
|
|
|
value.add(dn);
|
|
|
typeNodeMap.put(t, value);
|
|
|
} else {
|
|
@@ -820,7 +813,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
* - Storage type
|
|
|
* @return datanodes which has the given storage type
|
|
|
*/
|
|
|
- private List<DatanodeDescriptor> getNodesWithStorages(StorageType type) {
|
|
|
+ private List<DatanodeInfo> getNodesWithStorages(StorageType type) {
|
|
|
return typeNodeMap.get(type);
|
|
|
}
|
|
|
}
|
|
@@ -982,7 +975,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
|
|
|
public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
|
|
|
String path) throws IOException {
|
|
|
- INode inode = namesystem.getFSDirectory().getINode(path);
|
|
|
- return storageMovementNeeded.getStatus(inode.getId());
|
|
|
+ return storageMovementNeeded.getStatus(ctxt.getFileID(path));
|
|
|
}
|
|
|
}
|