|
@@ -43,47 +43,38 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
* schedule the block collection IDs for movement. It track the info of
|
|
|
* scheduled items and remove the SPS xAttr from the file/Directory once
|
|
|
* movement is success.
|
|
|
- *
|
|
|
- * @param <T>
|
|
|
- * is identifier of inode or full path name of inode. Internal sps will
|
|
|
- * use the file inodeId for the block movement. External sps will use
|
|
|
- * file string path representation for the block movement.
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
-public class BlockStorageMovementNeeded<T> {
|
|
|
+public class BlockStorageMovementNeeded {
|
|
|
|
|
|
public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(BlockStorageMovementNeeded.class);
|
|
|
|
|
|
- private final Queue<ItemInfo<T>> storageMovementNeeded =
|
|
|
- new LinkedList<ItemInfo<T>>();
|
|
|
+ private final Queue<ItemInfo> storageMovementNeeded =
|
|
|
+ new LinkedList<ItemInfo>();
|
|
|
|
|
|
/**
|
|
|
* Map of startPath and number of child's. Number of child's indicate the
|
|
|
* number of files pending to satisfy the policy.
|
|
|
*/
|
|
|
- private final Map<T, DirPendingWorkInfo> pendingWorkForDirectory =
|
|
|
+ private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
|
|
|
new HashMap<>();
|
|
|
|
|
|
- private final Map<T, StoragePolicySatisfyPathStatusInfo> spsStatus =
|
|
|
+ private final Map<Long, StoragePolicySatisfyPathStatusInfo> spsStatus =
|
|
|
new ConcurrentHashMap<>();
|
|
|
|
|
|
- private final Context<T> ctxt;
|
|
|
+ private final Context ctxt;
|
|
|
|
|
|
private Daemon pathIdCollector;
|
|
|
|
|
|
- private FileCollector<T> fileCollector;
|
|
|
-
|
|
|
private SPSPathIdProcessor pathIDProcessor;
|
|
|
|
|
|
// Amount of time to cache the SUCCESS status of path before turning it to
|
|
|
// NOT_AVAILABLE.
|
|
|
private static long statusClearanceElapsedTimeMs = 300000;
|
|
|
|
|
|
- public BlockStorageMovementNeeded(Context<T> context,
|
|
|
- FileCollector<T> fileCollector) {
|
|
|
+ public BlockStorageMovementNeeded(Context context) {
|
|
|
this.ctxt = context;
|
|
|
- this.fileCollector = fileCollector;
|
|
|
pathIDProcessor = new SPSPathIdProcessor();
|
|
|
}
|
|
|
|
|
@@ -94,7 +85,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
* @param trackInfo
|
|
|
* - track info for satisfy the policy
|
|
|
*/
|
|
|
- public synchronized void add(ItemInfo<T> trackInfo) {
|
|
|
+ public synchronized void add(ItemInfo trackInfo) {
|
|
|
spsStatus.put(trackInfo.getFile(),
|
|
|
new StoragePolicySatisfyPathStatusInfo(
|
|
|
StoragePolicySatisfyPathStatus.IN_PROGRESS));
|
|
@@ -114,7 +105,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
* scan.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- public synchronized void addAll(T startPath, List<ItemInfo<T>> itemInfoList,
|
|
|
+ public synchronized void addAll(long startPath, List<ItemInfo> itemInfoList,
|
|
|
boolean scanCompleted) {
|
|
|
storageMovementNeeded.addAll(itemInfoList);
|
|
|
updatePendingDirScanStats(startPath, itemInfoList.size(), scanCompleted);
|
|
@@ -131,7 +122,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
* elements to scan.
|
|
|
*/
|
|
|
@VisibleForTesting
|
|
|
- public synchronized void add(ItemInfo<T> itemInfo, boolean scanCompleted) {
|
|
|
+ public synchronized void add(ItemInfo itemInfo, boolean scanCompleted) {
|
|
|
storageMovementNeeded.add(itemInfo);
|
|
|
// This represents sps start id is file, so no need to update pending dir
|
|
|
// stats.
|
|
@@ -141,7 +132,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
updatePendingDirScanStats(itemInfo.getFile(), 1, scanCompleted);
|
|
|
}
|
|
|
|
|
|
- private void updatePendingDirScanStats(T startPath, int numScannedFiles,
|
|
|
+ private void updatePendingDirScanStats(long startPath, int numScannedFiles,
|
|
|
boolean scanCompleted) {
|
|
|
DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startPath);
|
|
|
if (pendingWork == null) {
|
|
@@ -160,7 +151,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
*
|
|
|
* @return satisfier files
|
|
|
*/
|
|
|
- public synchronized ItemInfo<T> get() {
|
|
|
+ public synchronized ItemInfo get() {
|
|
|
return storageMovementNeeded.poll();
|
|
|
}
|
|
|
|
|
@@ -181,12 +172,12 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
* Decrease the pending child count for directory once one file blocks moved
|
|
|
* successfully. Remove the SPS xAttr if pending child count is zero.
|
|
|
*/
|
|
|
- public synchronized void removeItemTrackInfo(ItemInfo<T> trackInfo,
|
|
|
+ public synchronized void removeItemTrackInfo(ItemInfo trackInfo,
|
|
|
boolean isSuccess) throws IOException {
|
|
|
if (trackInfo.isDir()) {
|
|
|
// If track is part of some start inode then reduce the pending
|
|
|
// directory work count.
|
|
|
- T startId = trackInfo.getStartPath();
|
|
|
+ long startId = trackInfo.getStartPath();
|
|
|
if (!ctxt.isFileExist(startId)) {
|
|
|
// directory deleted just remove it.
|
|
|
this.pendingWorkForDirectory.remove(startId);
|
|
@@ -212,11 +203,11 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public synchronized void clearQueue(T trackId) {
|
|
|
+ public synchronized void clearQueue(long trackId) {
|
|
|
ctxt.removeSPSPathId(trackId);
|
|
|
- Iterator<ItemInfo<T>> iterator = storageMovementNeeded.iterator();
|
|
|
+ Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
|
|
|
while (iterator.hasNext()) {
|
|
|
- ItemInfo<T> next = iterator.next();
|
|
|
+ ItemInfo next = iterator.next();
|
|
|
if (next.getFile() == trackId) {
|
|
|
iterator.remove();
|
|
|
}
|
|
@@ -227,7 +218,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
/**
|
|
|
* Mark inode status as SUCCESS in map.
|
|
|
*/
|
|
|
- private void updateStatus(T startId, boolean isSuccess){
|
|
|
+ private void updateStatus(long startId, boolean isSuccess){
|
|
|
StoragePolicySatisfyPathStatusInfo spsStatusInfo =
|
|
|
spsStatus.get(startId);
|
|
|
if (spsStatusInfo == null) {
|
|
@@ -249,7 +240,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
*/
|
|
|
public synchronized void clearQueuesWithNotification() {
|
|
|
// Remove xAttr from directories
|
|
|
- T trackId;
|
|
|
+ Long trackId;
|
|
|
while ((trackId = ctxt.getNextSPSPath()) != null) {
|
|
|
try {
|
|
|
// Remove xAttr for file
|
|
@@ -261,7 +252,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
|
|
|
// File's directly added to storageMovementNeeded, So try to remove
|
|
|
// xAttr for file
|
|
|
- ItemInfo<T> itemInfo;
|
|
|
+ ItemInfo itemInfo;
|
|
|
while ((itemInfo = get()) != null) {
|
|
|
try {
|
|
|
// Remove xAttr for file
|
|
@@ -287,7 +278,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
public void run() {
|
|
|
LOG.info("Starting SPSPathIdProcessor!.");
|
|
|
long lastStatusCleanTime = 0;
|
|
|
- T startINode = null;
|
|
|
+ Long startINode = null;
|
|
|
while (ctxt.isRunning()) {
|
|
|
try {
|
|
|
if (!ctxt.isInSafeMode()) {
|
|
@@ -301,7 +292,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
spsStatus.put(startINode,
|
|
|
new StoragePolicySatisfyPathStatusInfo(
|
|
|
StoragePolicySatisfyPathStatus.IN_PROGRESS));
|
|
|
- fileCollector.scanAndCollectFiles(startINode);
|
|
|
+ ctxt.scanAndCollectFiles(startINode);
|
|
|
// check if directory was empty and no child added to queue
|
|
|
DirPendingWorkInfo dirPendingWorkInfo =
|
|
|
pendingWorkForDirectory.get(startINode);
|
|
@@ -339,9 +330,9 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
}
|
|
|
|
|
|
private synchronized void cleanSPSStatus() {
|
|
|
- for (Iterator<Entry<T, StoragePolicySatisfyPathStatusInfo>> it = spsStatus
|
|
|
- .entrySet().iterator(); it.hasNext();) {
|
|
|
- Entry<T, StoragePolicySatisfyPathStatusInfo> entry = it.next();
|
|
|
+ for (Iterator<Entry<Long, StoragePolicySatisfyPathStatusInfo>> it =
|
|
|
+ spsStatus.entrySet().iterator(); it.hasNext();) {
|
|
|
+ Entry<Long, StoragePolicySatisfyPathStatusInfo> entry = it.next();
|
|
|
if (entry.getValue().canRemove()) {
|
|
|
it.remove();
|
|
|
}
|
|
@@ -477,7 +468,7 @@ public class BlockStorageMovementNeeded<T> {
|
|
|
return statusClearanceElapsedTimeMs;
|
|
|
}
|
|
|
|
|
|
- public void markScanCompletedForDir(T inode) {
|
|
|
+ public void markScanCompletedForDir(long inode) {
|
|
|
DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(inode);
|
|
|
if (pendingWork != null) {
|
|
|
pendingWork.markScanCompleted();
|