|
@@ -55,6 +55,7 @@ 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;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -174,10 +175,11 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
return;
|
|
|
}
|
|
|
if (reconfigStart) {
|
|
|
- LOG.info("Starting StoragePolicySatisfier, as admin requested to "
|
|
|
- + "start it.");
|
|
|
+ LOG.info("Starting {} StoragePolicySatisfier, as admin requested to "
|
|
|
+ + "start it.", StringUtils.toLowerCase(spsMode.toString()));
|
|
|
} else {
|
|
|
- LOG.info("Starting StoragePolicySatisfier.");
|
|
|
+ LOG.info("Starting {} StoragePolicySatisfier.",
|
|
|
+ StringUtils.toLowerCase(spsMode.toString()));
|
|
|
}
|
|
|
|
|
|
// Ensure that all the previously submitted block movements(if any) have to
|
|
@@ -243,7 +245,14 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
|
|
|
@Override
|
|
|
public void run() {
|
|
|
- while (ctxt.isRunning()) {
|
|
|
+ while (isRunning) {
|
|
|
+ // Check if dependent service is running
|
|
|
+ if (!ctxt.isRunning()) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Upstream service is down, skipping the sps work.");
|
|
|
+ }
|
|
|
+ continue;
|
|
|
+ }
|
|
|
try {
|
|
|
if (!ctxt.isInSafeMode()) {
|
|
|
ItemInfo itemInfo = storageMovementNeeded.get();
|
|
@@ -284,33 +293,39 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
// Just add to monitor, so it will be tracked for report and
|
|
|
// be removed on storage movement attempt finished report.
|
|
|
case BLOCKS_TARGETS_PAIRED:
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Block analysis status:{} for the file path:{}."
|
|
|
+ + " Adding to attempt monitor queue for the storage "
|
|
|
+ + "movement attempt finished report",
|
|
|
+ status.status, fileStatus.getPath());
|
|
|
+ }
|
|
|
this.storageMovementsMonitor.add(new AttemptedItemInfo(itemInfo
|
|
|
.getStartId(), itemInfo.getFileId(), monotonicNow(),
|
|
|
status.assignedBlocks, itemInfo.getRetryCount()));
|
|
|
break;
|
|
|
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.");
|
|
|
+ LOG.debug("Adding trackID:{} for the file path:{} back to"
|
|
|
+ + " retry queue as none of the blocks found its eligible"
|
|
|
+ + " targets.", trackId, fileStatus.getPath());
|
|
|
}
|
|
|
itemInfo.increRetryCount();
|
|
|
this.storageMovementNeeded.add(itemInfo);
|
|
|
break;
|
|
|
case FEW_LOW_REDUNDANCY_BLOCKS:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Adding trackID " + trackId
|
|
|
- + " back to retry queue as some of the blocks"
|
|
|
- + " are low redundant.");
|
|
|
+ LOG.debug("Adding trackID:{} for the file path:{} back to "
|
|
|
+ + "retry queue as some of the blocks are low redundant.",
|
|
|
+ trackId, fileStatus.getPath());
|
|
|
}
|
|
|
itemInfo.increRetryCount();
|
|
|
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.");
|
|
|
+ LOG.debug("Adding trackID:{} for the file path:{} back to "
|
|
|
+ + "retry queue as some of the blocks movement failed.",
|
|
|
+ trackId, fileStatus.getPath());
|
|
|
}
|
|
|
this.storageMovementNeeded.add(itemInfo);
|
|
|
break;
|
|
@@ -318,8 +333,9 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
case BLOCKS_TARGET_PAIRING_SKIPPED:
|
|
|
case BLOCKS_ALREADY_SATISFIED:
|
|
|
default:
|
|
|
- LOG.info("Block analysis skipped or blocks already satisfied"
|
|
|
- + " with storages. So, Cleaning up the Xattrs.");
|
|
|
+ LOG.info("Block analysis status:{} for the file path:{}."
|
|
|
+ + " So, Cleaning up the Xattrs.", status.status,
|
|
|
+ fileStatus.getPath());
|
|
|
storageMovementNeeded.removeItemTrackInfo(itemInfo, true);
|
|
|
break;
|
|
|
}
|
|
@@ -346,20 +362,20 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
if (isRunning) {
|
|
|
synchronized (this) {
|
|
|
if (isRunning) {
|
|
|
- isRunning = false;
|
|
|
- // Stopping monitor thread and clearing queues as well
|
|
|
- this.clearQueues();
|
|
|
- this.storageMovementsMonitor.stopGracefully();
|
|
|
- if (!(t instanceof InterruptedException)) {
|
|
|
- LOG.info("StoragePolicySatisfier received an exception"
|
|
|
- + " while shutting down.", t);
|
|
|
+ if (t instanceof InterruptedException) {
|
|
|
+ isRunning = false;
|
|
|
+ LOG.info("Stopping StoragePolicySatisfier.");
|
|
|
+ // Stopping monitor thread and clearing queues as well
|
|
|
+ this.clearQueues();
|
|
|
+ this.storageMovementsMonitor.stopGracefully();
|
|
|
+ } else {
|
|
|
+ LOG.error(
|
|
|
+ "StoragePolicySatisfier thread received runtime exception, "
|
|
|
+ + "ignoring", t);
|
|
|
}
|
|
|
- LOG.info("Stopping StoragePolicySatisfier.");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- LOG.error("StoragePolicySatisfier thread received runtime exception. "
|
|
|
- + "Stopping Storage policy satisfier work", t);
|
|
|
return;
|
|
|
}
|
|
|
|
|
@@ -374,9 +390,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
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", fileInfo.getFileId());
|
|
|
+ LOG.info("File: {} is under construction. So, postpone"
|
|
|
+ + " this to the next retry iteration", fileInfo.getPath());
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.ANALYSIS_SKIPPED_FOR_RETRY,
|
|
|
new ArrayList<>());
|
|
@@ -384,8 +399,8 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
|
|
|
List<LocatedBlock> blocks = locatedBlocks.getLocatedBlocks();
|
|
|
if (blocks.size() == 0) {
|
|
|
- LOG.info("BlockCollectionID: {} file is not having any blocks."
|
|
|
- + " So, skipping the analysis.", fileInfo.getFileId());
|
|
|
+ LOG.info("File: {} is not having any blocks."
|
|
|
+ + " So, skipping the analysis.", fileInfo.getPath());
|
|
|
return new BlocksMovingAnalysis(
|
|
|
BlocksMovingAnalysis.Status.BLOCKS_TARGET_PAIRING_SKIPPED,
|
|
|
new ArrayList<>());
|
|
@@ -970,4 +985,12 @@ public class StoragePolicySatisfier implements SPSService, Runnable {
|
|
|
public void markScanCompletedForPath(Long inodeId) {
|
|
|
getStorageMovementQueue().markScanCompletedForDir(inodeId);
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Join main SPS thread.
|
|
|
+ */
|
|
|
+ public void join() throws InterruptedException {
|
|
|
+ //TODO Add join here on SPS rpc server also
|
|
|
+ storagePolicySatisfierThread.join();
|
|
|
+ }
|
|
|
}
|