|
@@ -66,7 +66,7 @@ import com.google.common.base.Preconditions;
|
|
|
* 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 a dedidated daemon (can
|
|
|
+ * where users may want to physically move the blocks by a dedicated 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
|
|
@@ -77,19 +77,19 @@ import com.google.common.base.Preconditions;
|
|
|
* physical block movements.
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
-public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
+public class StoragePolicySatisfier<T> implements SPSService<T>, Runnable {
|
|
|
public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(StoragePolicySatisfier.class);
|
|
|
private Daemon storagePolicySatisfierThread;
|
|
|
- private BlockStorageMovementNeeded storageMovementNeeded;
|
|
|
- private BlockStorageMovementAttemptedItems storageMovementsMonitor;
|
|
|
+ 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;
|
|
|
- private Context ctxt;
|
|
|
+ private Context<T> ctxt;
|
|
|
private BlockMoveTaskHandler blockMoveTaskHandler;
|
|
|
private final Configuration conf;
|
|
|
|
|
@@ -135,15 +135,15 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void init(final Context context, final FileIdCollector fileIDCollector,
|
|
|
+ public void init(final Context<T> context,
|
|
|
+ final FileCollector<T> fileIDCollector,
|
|
|
final BlockMoveTaskHandler blockMovementTaskHandler,
|
|
|
final BlockMovementListener blockMovementListener) {
|
|
|
this.ctxt = context;
|
|
|
- this.storageMovementNeeded =
|
|
|
- new BlockStorageMovementNeeded(context, fileIDCollector);
|
|
|
- this.storageMovementsMonitor =
|
|
|
- new BlockStorageMovementAttemptedItems(this,
|
|
|
- storageMovementNeeded, blockMovementListener);
|
|
|
+ this.storageMovementNeeded = new BlockStorageMovementNeeded<T>(context,
|
|
|
+ fileIDCollector);
|
|
|
+ this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems<T>(
|
|
|
+ this, storageMovementNeeded, blockMovementListener);
|
|
|
this.blockMoveTaskHandler = blockMovementTaskHandler;
|
|
|
this.spsWorkMultiplier = getSPSWorkMultiplier(getConf());
|
|
|
this.blockMovementMaxRetry = getConf().getInt(
|
|
@@ -257,24 +257,24 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
continue;
|
|
|
}
|
|
|
try {
|
|
|
+ ItemInfo<T> itemInfo = null;
|
|
|
+ boolean retryItem = false;
|
|
|
if (!ctxt.isInSafeMode()) {
|
|
|
- ItemInfo itemInfo = storageMovementNeeded.get();
|
|
|
+ itemInfo = storageMovementNeeded.get();
|
|
|
if (itemInfo != null) {
|
|
|
if(itemInfo.getRetryCount() >= blockMovementMaxRetry){
|
|
|
LOG.info("Failed to satisfy the policy after "
|
|
|
+ blockMovementMaxRetry + " retries. Removing inode "
|
|
|
- + itemInfo.getFileId() + " from the queue");
|
|
|
+ + itemInfo.getFile() + " from the queue");
|
|
|
storageMovementNeeded.removeItemTrackInfo(itemInfo, false);
|
|
|
continue;
|
|
|
}
|
|
|
- long trackId = itemInfo.getFileId();
|
|
|
+ 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?
|
|
|
- boolean hasLowRedundancyBlocks = ctxt
|
|
|
- .hasLowRedundancyBlocks(trackId);
|
|
|
HdfsFileStatus fileStatus = ctxt.getFileInfo(trackId);
|
|
|
// Check path existence.
|
|
|
if (fileStatus == null || fileStatus.isDir()) {
|
|
@@ -289,7 +289,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
|
|
|
HdfsLocatedFileStatus file = (HdfsLocatedFileStatus) fileStatus;
|
|
|
status = analyseBlocksStorageMovementsAndAssignToDN(file,
|
|
|
- hasLowRedundancyBlocks, existingStoragePolicy, liveDnReports);
|
|
|
+ existingStoragePolicy, liveDnReports);
|
|
|
switch (status.status) {
|
|
|
// Just add to monitor, so it will be retried after timeout
|
|
|
case ANALYSIS_SKIPPED_FOR_RETRY:
|
|
@@ -302,8 +302,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
+ "movement attempt finished report",
|
|
|
status.status, fileStatus.getPath());
|
|
|
}
|
|
|
- this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
|
|
|
- .getStartId(), itemInfo.getFileId(), monotonicNow(),
|
|
|
+ this.storageMovementsMonitor.add(new AttemptedItemInfo<T>(
|
|
|
+ itemInfo.getStartPath(), itemInfo.getFile(), monotonicNow(),
|
|
|
status.assignedBlocks, itemInfo.getRetryCount()));
|
|
|
break;
|
|
|
case NO_BLOCKS_TARGETS_PAIRED:
|
|
@@ -312,8 +312,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
+ " retry queue as none of the blocks found its eligible"
|
|
|
+ " targets.", trackId, fileStatus.getPath());
|
|
|
}
|
|
|
- itemInfo.increRetryCount();
|
|
|
- this.storageMovementNeeded.add(itemInfo);
|
|
|
+ retryItem = true;
|
|
|
break;
|
|
|
case FEW_LOW_REDUNDANCY_BLOCKS:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -321,8 +320,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
+ "retry queue as some of the blocks are low redundant.",
|
|
|
trackId, fileStatus.getPath());
|
|
|
}
|
|
|
- itemInfo.increRetryCount();
|
|
|
- this.storageMovementNeeded.add(itemInfo);
|
|
|
+ retryItem = true;
|
|
|
break;
|
|
|
case BLOCKS_FAILED_TO_MOVE:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -330,7 +328,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
+ "retry queue as some of the blocks movement failed.",
|
|
|
trackId, fileStatus.getPath());
|
|
|
}
|
|
|
- this.storageMovementNeeded.add(itemInfo);
|
|
|
+ retryItem = true;
|
|
|
break;
|
|
|
// Just clean Xattrs
|
|
|
case BLOCKS_TARGET_PAIRING_SKIPPED:
|
|
@@ -354,6 +352,10 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
Thread.sleep(3000);
|
|
|
blockCount = 0L;
|
|
|
}
|
|
|
+ if (retryItem) {
|
|
|
+ itemInfo.increRetryCount();
|
|
|
+ this.storageMovementNeeded.add(itemInfo);
|
|
|
+ }
|
|
|
} catch (IOException e) {
|
|
|
LOG.error("Exception during StoragePolicySatisfier execution - "
|
|
|
+ "will continue next cycle", e);
|
|
@@ -377,7 +379,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
}
|
|
|
|
|
|
private BlocksMovingAnalysis analyseBlocksStorageMovementsAndAssignToDN(
|
|
|
- HdfsLocatedFileStatus fileInfo, boolean hasLowRedundancyBlocks,
|
|
|
+ HdfsLocatedFileStatus fileInfo,
|
|
|
BlockStoragePolicy existingStoragePolicy,
|
|
|
DatanodeStorageReport[] liveDns) {
|
|
|
BlocksMovingAnalysis.Status status =
|
|
@@ -403,9 +405,17 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
new ArrayList<>());
|
|
|
}
|
|
|
List<BlockMovingInfo> blockMovingInfos = new ArrayList<BlockMovingInfo>();
|
|
|
-
|
|
|
+ boolean hasLowRedundancyBlocks = false;
|
|
|
+ int replication = fileInfo.getReplication();
|
|
|
for (int i = 0; i < blocks.size(); i++) {
|
|
|
LocatedBlock blockInfo = blocks.get(i);
|
|
|
+
|
|
|
+ // Block is considered as low redundancy when the block locations array
|
|
|
+ // length is less than expected replication factor. If any of the block is
|
|
|
+ // low redundant, then hasLowRedundancyBlocks will be marked as true.
|
|
|
+ hasLowRedundancyBlocks |= isLowRedundancyBlock(blockInfo, replication,
|
|
|
+ ecPolicy);
|
|
|
+
|
|
|
List<StorageType> expectedStorageTypes;
|
|
|
if (blockInfo.isStriped()) {
|
|
|
if (ErasureCodingPolicyManager
|
|
@@ -446,13 +456,15 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
// policy.
|
|
|
status = BlocksMovingAnalysis.Status.NO_BLOCKS_TARGETS_PAIRED;
|
|
|
}
|
|
|
- } else if (hasLowRedundancyBlocks
|
|
|
- && status != BlocksMovingAnalysis.Status.BLOCKS_TARGETS_PAIRED) {
|
|
|
- // Check if the previous block was successfully paired.
|
|
|
- status = BlocksMovingAnalysis.Status.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // 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.FEW_LOW_REDUNDANCY_BLOCKS;
|
|
|
+ }
|
|
|
List<Block> assignedBlockIds = new ArrayList<Block>();
|
|
|
for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
|
|
|
// Check for at least one block storage movement has been chosen
|
|
@@ -470,6 +482,33 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
return new BlocksMovingAnalysis(status, assignedBlockIds);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The given block is considered as low redundancy when the block locations
|
|
|
+ * length is less than expected replication factor. For EC blocks, redundancy
|
|
|
+ * is the summation of data + parity blocks.
|
|
|
+ *
|
|
|
+ * @param blockInfo
|
|
|
+ * block
|
|
|
+ * @param replication
|
|
|
+ * replication factor of the given file block
|
|
|
+ * @param ecPolicy
|
|
|
+ * erasure coding policy of the given file block
|
|
|
+ * @return true if the given block is low redundant.
|
|
|
+ */
|
|
|
+ private boolean isLowRedundancyBlock(LocatedBlock blockInfo, int replication,
|
|
|
+ ErasureCodingPolicy ecPolicy) {
|
|
|
+ boolean hasLowRedundancyBlock = false;
|
|
|
+ if (blockInfo.isStriped()) {
|
|
|
+ // For EC blocks, redundancy is the summation of data + parity blocks.
|
|
|
+ replication = ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits();
|
|
|
+ }
|
|
|
+ // block is considered as low redundancy when the block locations length is
|
|
|
+ // less than expected replication factor.
|
|
|
+ hasLowRedundancyBlock = blockInfo.getLocations().length < replication ? true
|
|
|
+ : false;
|
|
|
+ return hasLowRedundancyBlock;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Compute the list of block moving information corresponding to the given
|
|
|
* blockId. This will check that each block location of the given block is
|
|
@@ -863,7 +902,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- BlockStorageMovementAttemptedItems getAttemptedItemsMonitor() {
|
|
|
+ public BlockStorageMovementAttemptedItems<T> getAttemptedItemsMonitor() {
|
|
|
return storageMovementsMonitor;
|
|
|
}
|
|
|
|
|
@@ -880,7 +919,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
/**
|
|
|
* Clear queues for given track id.
|
|
|
*/
|
|
|
- public void clearQueue(long trackId) {
|
|
|
+ public void clearQueue(T trackId) {
|
|
|
storageMovementNeeded.clearQueue(trackId);
|
|
|
}
|
|
|
|
|
@@ -889,7 +928,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
* attempted or reported time stamp. This is used by
|
|
|
* {@link BlockStorageMovementAttemptedItems#storageMovementAttemptedItems}.
|
|
|
*/
|
|
|
- final static class AttemptedItemInfo extends ItemInfo {
|
|
|
+ final static class AttemptedItemInfo<T> extends ItemInfo<T> {
|
|
|
private long lastAttemptedOrReportedTime;
|
|
|
private final List<Block> blocks;
|
|
|
|
|
@@ -903,7 +942,7 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
* @param lastAttemptedOrReportedTime
|
|
|
* last attempted or reported time
|
|
|
*/
|
|
|
- AttemptedItemInfo(long rootId, long trackId,
|
|
|
+ AttemptedItemInfo(T rootId, T trackId,
|
|
|
long lastAttemptedOrReportedTime,
|
|
|
List<Block> blocks, int retryCount) {
|
|
|
super(rootId, trackId, retryCount);
|
|
@@ -932,24 +971,33 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns sps invoked path status. This method is used by internal satisfy
|
|
|
+ * storage policy service.
|
|
|
+ *
|
|
|
+ * @param path
|
|
|
+ * sps path
|
|
|
+ * @return storage policy satisfy path status
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
public StoragePolicySatisfyPathStatus checkStoragePolicySatisfyPathStatus(
|
|
|
String path) throws IOException {
|
|
|
return storageMovementNeeded.getStatus(ctxt.getFileID(path));
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void addFileIdToProcess(ItemInfo trackInfo, boolean scanCompleted) {
|
|
|
+ public void addFileToProcess(ItemInfo<T> trackInfo, boolean scanCompleted) {
|
|
|
storageMovementNeeded.add(trackInfo, scanCompleted);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Added track info for inode {} to block "
|
|
|
- + "storageMovementNeeded queue", trackInfo.getFileId());
|
|
|
+ + "storageMovementNeeded queue", trackInfo.getFile());
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void addAllFileIdsToProcess(long startId, List<ItemInfo> itemInfoList,
|
|
|
+ public void addAllFilesToProcess(T startPath, List<ItemInfo<T>> itemInfoList,
|
|
|
boolean scanCompleted) {
|
|
|
- getStorageMovementQueue().addAll(startId, itemInfoList, scanCompleted);
|
|
|
+ getStorageMovementQueue().addAll(startPath, itemInfoList, scanCompleted);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -963,12 +1011,12 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public BlockStorageMovementNeeded getStorageMovementQueue() {
|
|
|
+ public BlockStorageMovementNeeded<T> getStorageMovementQueue() {
|
|
|
return storageMovementNeeded;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void markScanCompletedForPath(Long inodeId) {
|
|
|
+ public void markScanCompletedForPath(T inodeId) {
|
|
|
getStorageMovementQueue().markScanCompletedForDir(inodeId);
|
|
|
}
|
|
|
|
|
@@ -976,7 +1024,6 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
* Join main SPS thread.
|
|
|
*/
|
|
|
public void join() throws InterruptedException {
|
|
|
- //TODO Add join here on SPS rpc server also
|
|
|
storagePolicySatisfierThread.join();
|
|
|
}
|
|
|
|