|
@@ -29,6 +29,7 @@ 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;
|
|
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
|
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.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;
|
|
@@ -64,28 +64,34 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
* storage policy type in Namespace, but physical block storage movement will
|
|
|
* not happen until user runs "Mover Tool" explicitly for such files. The
|
|
|
* StoragePolicySatisfier Daemon thread implemented for addressing the case
|
|
|
- * where users may want to physically move the blocks by HDFS itself instead of
|
|
|
- * running mover tool explicitly. Just calling client API to
|
|
|
- * satisfyStoragePolicy on a file/dir will automatically trigger to move its
|
|
|
- * physical storage locations as expected in asynchronous manner. Here Namenode
|
|
|
- * will pick the file blocks which are expecting to change its storages, then it
|
|
|
- * will build the mapping of source block location and expected storage type and
|
|
|
- * location to move. After that this class will also prepare commands to send to
|
|
|
- * Datanode for processing the physical block movements.
|
|
|
+ * where users may want to physically move the blocks by a dedidated daemon (can
|
|
|
+ * run inside Namenode or stand alone) instead of running mover tool explicitly.
|
|
|
+ * Just calling client API to satisfyStoragePolicy on a file/dir will
|
|
|
+ * automatically trigger to move its physical storage locations as expected in
|
|
|
+ * asynchronous manner. Here SPS will pick the file blocks which are expecting
|
|
|
+ * to change its storages, then it will build the mapping of source block
|
|
|
+ * location and expected storage type and location to move. After that this
|
|
|
+ * class will also prepare requests to send to Datanode for processing the
|
|
|
+ * physical block movements.
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
-public class StoragePolicySatisfier implements Runnable {
|
|
|
+public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(StoragePolicySatisfier.class);
|
|
|
private Daemon storagePolicySatisfierThread;
|
|
|
- private final BlockStorageMovementNeeded storageMovementNeeded;
|
|
|
- private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
|
|
|
+ private BlockStorageMovementNeeded storageMovementNeeded;
|
|
|
+ private BlockStorageMovementAttemptedItems storageMovementsMonitor;
|
|
|
private volatile boolean isRunning = false;
|
|
|
private int spsWorkMultiplier;
|
|
|
private long blockCount = 0L;
|
|
|
private int blockMovementMaxRetry;
|
|
|
- private final Context ctxt;
|
|
|
+ private Context ctxt;
|
|
|
+ private BlockMoveTaskHandler blockMoveTaskHandler;
|
|
|
+ private Configuration conf;
|
|
|
|
|
|
+ public StoragePolicySatisfier(Configuration conf) {
|
|
|
+ this.conf = conf;
|
|
|
+ }
|
|
|
/**
|
|
|
* Represents the collective analysis status for all blocks.
|
|
|
*/
|
|
@@ -125,13 +131,17 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public StoragePolicySatisfier(Context ctxt) {
|
|
|
- this.ctxt = ctxt;
|
|
|
- this.storageMovementNeeded = new BlockStorageMovementNeeded(ctxt);
|
|
|
- this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(ctxt,
|
|
|
+ public void init(final Context context, final FileIdCollector fileIDCollector,
|
|
|
+ final BlockMoveTaskHandler blockMovementTaskHandler) {
|
|
|
+ this.ctxt = context;
|
|
|
+ this.storageMovementNeeded =
|
|
|
+ new BlockStorageMovementNeeded(context, fileIDCollector);
|
|
|
+ this.storageMovementsMonitor =
|
|
|
+ new BlockStorageMovementAttemptedItems(this,
|
|
|
storageMovementNeeded);
|
|
|
- this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(ctxt.getConf());
|
|
|
- this.blockMovementMaxRetry = ctxt.getConf().getInt(
|
|
|
+ this.blockMoveTaskHandler = blockMovementTaskHandler;
|
|
|
+ this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(getConf());
|
|
|
+ this.blockMovementMaxRetry = getConf().getInt(
|
|
|
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_KEY,
|
|
|
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT);
|
|
|
}
|
|
@@ -139,12 +149,10 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
/**
|
|
|
* 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, FSDirectory fsd) {
|
|
|
+ @Override
|
|
|
+ public synchronized void start(boolean reconfigStart) {
|
|
|
isRunning = true;
|
|
|
- ctxt.setSPSRunning(this::isRunning);
|
|
|
if (ctxt.isMoverRunning()) {
|
|
|
isRunning = false;
|
|
|
LOG.error(
|
|
@@ -163,20 +171,14 @@ 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(fsd);
|
|
|
storagePolicySatisfierThread = new Daemon(this);
|
|
|
storagePolicySatisfierThread.setName("StoragePolicySatisfier");
|
|
|
storagePolicySatisfierThread.start();
|
|
|
this.storageMovementsMonitor.start();
|
|
|
+ this.storageMovementNeeded.activate();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Disables storage policy satisfier by stopping its services.
|
|
|
- *
|
|
|
- * @param forceStop
|
|
|
- * true represents that it should stop SPS service by clearing all
|
|
|
- * pending SPS work
|
|
|
- */
|
|
|
+ @Override
|
|
|
public synchronized void disable(boolean forceStop) {
|
|
|
isRunning = false;
|
|
|
if (storagePolicySatisfierThread == null) {
|
|
@@ -195,14 +197,15 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Timed wait to stop storage policy satisfier daemon threads.
|
|
|
- */
|
|
|
+ @Override
|
|
|
public synchronized void stopGracefully() {
|
|
|
if (isRunning) {
|
|
|
disable(true);
|
|
|
}
|
|
|
- this.storageMovementsMonitor.stopGracefully();
|
|
|
+
|
|
|
+ if (this.storageMovementsMonitor != null) {
|
|
|
+ this.storageMovementsMonitor.stopGracefully();
|
|
|
+ }
|
|
|
|
|
|
if (storagePolicySatisfierThread == null) {
|
|
|
return;
|
|
@@ -213,10 +216,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Check whether StoragePolicySatisfier is running.
|
|
|
- * @return true if running
|
|
|
- */
|
|
|
+ @Override
|
|
|
public boolean isRunning() {
|
|
|
return isRunning;
|
|
|
}
|
|
@@ -239,11 +239,11 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
|
|
|
LOG.info("Failed to satisfy the policy after "
|
|
|
+ blockMovementMaxRetry + " retries. Removing inode "
|
|
|
- + itemInfo.getTrackId() + " from the queue");
|
|
|
+ + itemInfo.getFileId() + " from the queue");
|
|
|
storageMovementNeeded.removeItemTrackInfo(itemInfo, false);
|
|
|
continue;
|
|
|
}
|
|
|
- long trackId = itemInfo.getTrackId();
|
|
|
+ long trackId = itemInfo.getFileId();
|
|
|
BlocksMovingAnalysis status = null;
|
|
|
DatanodeStorageReport[] liveDnReports;
|
|
|
BlockStoragePolicy existingStoragePolicy;
|
|
@@ -273,7 +273,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
// be removed on storage movement attempt finished report.
|
|
|
case BLOCKS_TARGETS_PAIRED:
|
|
|
this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
|
|
|
- .getStartId(), itemInfo.getTrackId(), monotonicNow(),
|
|
|
+ .getStartId(), itemInfo.getFileId(), monotonicNow(),
|
|
|
status.assignedBlocks, itemInfo.getRetryCount()));
|
|
|
break;
|
|
|
case NO_BLOCKS_TARGETS_PAIRED:
|
|
@@ -282,7 +282,7 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
+ " back to retry queue as none of the blocks"
|
|
|
+ " found its eligible targets.");
|
|
|
}
|
|
|
- itemInfo.retryCount++;
|
|
|
+ itemInfo.increRetryCount();
|
|
|
this.storageMovementNeeded.add(itemInfo);
|
|
|
break;
|
|
|
case FEW_LOW_REDUNDANCY_BLOCKS:
|
|
@@ -426,7 +426,8 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
|
|
|
// Check for at least one block storage movement has been chosen
|
|
|
try {
|
|
|
- ctxt.assignBlockMoveTaskToTargetNode(blkMovingInfo);
|
|
|
+ blockMoveTaskHandler.submitMoveTask(blkMovingInfo,
|
|
|
+ storageMovementsMonitor);
|
|
|
LOG.debug("BlockMovingInfo: {}", blkMovingInfo);
|
|
|
assignedBlockIds.add(blkMovingInfo.getBlock());
|
|
|
blockCount++;
|
|
@@ -611,7 +612,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
|
|
|
expected.remove(chosenTarget.storageType);
|
|
|
excludeNodes.add(chosenTarget.dn);
|
|
|
- // TODO: We can increment scheduled block count for this node?
|
|
|
} else {
|
|
|
LOG.warn(
|
|
|
"Failed to choose target datanode for the required"
|
|
@@ -830,11 +830,11 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
return;
|
|
|
}
|
|
|
storageMovementsMonitor
|
|
|
- .addReportedMovedBlocks(moveAttemptFinishedBlks.getBlocks());
|
|
|
+ .notifyMovementTriedBlocks(moveAttemptFinishedBlks.getBlocks());
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
|
|
|
+ BlockMovementListener getAttemptedItemsMonitor() {
|
|
|
return storageMovementsMonitor;
|
|
|
}
|
|
|
|
|
@@ -863,10 +863,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void addInodeToPendingDirQueue(long id) {
|
|
|
- storageMovementNeeded.addToPendingDirQueue(id);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Clear queues for given track id.
|
|
|
*/
|
|
@@ -874,57 +870,6 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
storageMovementNeeded.clearQueue(trackId);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * ItemInfo is a file info object for which need to satisfy the
|
|
|
- * policy.
|
|
|
- */
|
|
|
- public static class ItemInfo {
|
|
|
- private long startId;
|
|
|
- private long trackId;
|
|
|
- private int retryCount;
|
|
|
-
|
|
|
- public ItemInfo(long startId, long trackId) {
|
|
|
- this.startId = startId;
|
|
|
- this.trackId = trackId;
|
|
|
- //set 0 when item is getting added first time in queue.
|
|
|
- this.retryCount = 0;
|
|
|
- }
|
|
|
-
|
|
|
- public ItemInfo(long startId, long trackId, int retryCount) {
|
|
|
- this.startId = startId;
|
|
|
- this.trackId = trackId;
|
|
|
- this.retryCount = retryCount;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Return the start inode id of the current track Id.
|
|
|
- */
|
|
|
- public long getStartId() {
|
|
|
- return startId;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Return the File inode Id for which needs to satisfy the policy.
|
|
|
- */
|
|
|
- public long getTrackId() {
|
|
|
- return trackId;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns true if the tracking path is a directory, false otherwise.
|
|
|
- */
|
|
|
- public boolean isDir() {
|
|
|
- return (startId != trackId);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Get the attempted retry count of the block for satisfy the policy.
|
|
|
- */
|
|
|
- public int getRetryCount() {
|
|
|
- return retryCount;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* This class contains information of an attempted blocks and its last
|
|
|
* attempted or reported time stamp. This is used by
|
|
@@ -977,4 +922,30 @@ public class StoragePolicySatisfier implements Runnable {
|
|
|
String path) throws IOException {
|
|
|
return storageMovementNeeded.getStatus(ctxt.getFileID(path));
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void addFileIdToProcess(ItemInfo trackInfo) {
|
|
|
+ storageMovementNeeded.add(trackInfo);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void addAllFileIdsToProcess(long startId, List<ItemInfo> itemInfoList,
|
|
|
+ boolean scanCompleted) {
|
|
|
+ getStorageMovementQueue().addAll(startId, itemInfoList, scanCompleted);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int processingQueueSize() {
|
|
|
+ return storageMovementNeeded.size();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Configuration getConf() {
|
|
|
+ return conf;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public BlockStorageMovementNeeded getStorageMovementQueue() {
|
|
|
+ return storageMovementNeeded;
|
|
|
+ }
|
|
|
}
|