HDFS-12291: [SPS]: Provide a mechanism to recursively iterate and satisfy storage policy of all the files under the given dir. Contributed by Surendra Singh Lilhore.

This commit is contained in:
Uma Maheswara Rao G 2017-09-30 06:31:52 -07:00 committed by Uma Maheswara Rao Gangumalla
parent 7ea24fc06c
commit bfd3f8bd8a
11 changed files with 689 additions and 83 deletions

View File

@ -618,6 +618,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
"dfs.storage.policy.satisfier.enabled"; "dfs.storage.policy.satisfier.enabled";
public static final boolean DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT = public static final boolean DFS_STORAGE_POLICY_SATISFIER_ENABLED_DEFAULT =
false; false;
public static final String DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY =
"dfs.storage.policy.satisfier.queue.limit";
public static final int DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT =
1000;
public static final String DFS_SPS_WORK_MULTIPLIER_PER_ITERATION =
"dfs.storage.policy.satisfier.work.multiplier.per.iteration";
public static final int DFS_SPS_WORK_MULTIPLIER_PER_ITERATION_DEFAULT =
1;
public static final String DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY = public static final String DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_KEY =
"dfs.storage.policy.satisfier.recheck.timeout.millis"; "dfs.storage.policy.satisfier.recheck.timeout.millis";
public static final int DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT = public static final int DFS_STORAGE_POLICY_SATISFIER_RECHECK_TIMEOUT_MILLIS_DEFAULT =

View File

@ -1457,7 +1457,27 @@ public class DFSUtil {
"It should be a positive, non-zero integer value."); "It should be a positive, non-zero integer value.");
return blocksReplWorkMultiplier; return blocksReplWorkMultiplier;
} }
/**
* Get DFS_SPS_WORK_MULTIPLIER_PER_ITERATION from
* configuration.
*
* @param conf Configuration
* @return Value of DFS_SPS_WORK_MULTIPLIER_PER_ITERATION
*/
public static int getSPSWorkMultiplier(Configuration conf) {
int spsWorkMultiplier = conf
.getInt(
DFSConfigKeys.DFS_SPS_WORK_MULTIPLIER_PER_ITERATION,
DFSConfigKeys.DFS_SPS_WORK_MULTIPLIER_PER_ITERATION_DEFAULT);
Preconditions.checkArgument(
(spsWorkMultiplier > 0),
DFSConfigKeys.DFS_SPS_WORK_MULTIPLIER_PER_ITERATION +
" = '" + spsWorkMultiplier + "' is invalid. " +
"It should be a positive, non-zero integer value.");
return spsWorkMultiplier;
}
/** /**
* Get SPNEGO keytab Key from configuration * Get SPNEGO keytab Key from configuration
* *

View File

@ -101,7 +101,7 @@ public class BlockStorageMovementAttemptedItems {
public void add(ItemInfo itemInfo, boolean allBlockLocsAttemptedToSatisfy) { public void add(ItemInfo itemInfo, boolean allBlockLocsAttemptedToSatisfy) {
synchronized (storageMovementAttemptedItems) { synchronized (storageMovementAttemptedItems) {
AttemptedItemInfo attemptedItemInfo = new AttemptedItemInfo( AttemptedItemInfo attemptedItemInfo = new AttemptedItemInfo(
itemInfo.getRootId(), itemInfo.getTrackId(), monotonicNow(), itemInfo.getStartId(), itemInfo.getTrackId(), monotonicNow(),
allBlockLocsAttemptedToSatisfy); allBlockLocsAttemptedToSatisfy);
storageMovementAttemptedItems.put(itemInfo.getTrackId(), storageMovementAttemptedItems.put(itemInfo.getTrackId(),
attemptedItemInfo); attemptedItemInfo);
@ -260,7 +260,7 @@ public class BlockStorageMovementAttemptedItems {
synchronized (storageMovementAttemptedResults) { synchronized (storageMovementAttemptedResults) {
if (!isExistInResult(blockCollectionID)) { if (!isExistInResult(blockCollectionID)) {
ItemInfo candidate = new ItemInfo( ItemInfo candidate = new ItemInfo(
itemInfo.getRootId(), blockCollectionID); itemInfo.getStartId(), blockCollectionID);
blockStorageMovementNeeded.add(candidate); blockStorageMovementNeeded.add(candidate);
iter.remove(); iter.remove();
LOG.info("TrackID: {} becomes timed out and moved to needed " LOG.info("TrackID: {} becomes timed out and moved to needed "
@ -315,7 +315,7 @@ public class BlockStorageMovementAttemptedItems {
// blockStorageMovementNeeded#removeIteamTrackInfo() for cleaning // blockStorageMovementNeeded#removeIteamTrackInfo() for cleaning
// the xAttr // the xAttr
ItemInfo itemInfo = new ItemInfo((attemptedItemInfo != null) ItemInfo itemInfo = new ItemInfo((attemptedItemInfo != null)
? attemptedItemInfo.getRootId() : trackId, trackId); ? attemptedItemInfo.getStartId() : trackId, trackId);
switch (status) { switch (status) {
case FAILURE: case FAILURE:
if (attemptedItemInfo != null) { if (attemptedItemInfo != null) {
@ -345,7 +345,7 @@ public class BlockStorageMovementAttemptedItems {
if (attemptedItemInfo != null) { if (attemptedItemInfo != null) {
if (!attemptedItemInfo.isAllBlockLocsAttemptedToSatisfy()) { if (!attemptedItemInfo.isAllBlockLocsAttemptedToSatisfy()) {
blockStorageMovementNeeded blockStorageMovementNeeded
.add(new ItemInfo(attemptedItemInfo.getRootId(), trackId)); .add(new ItemInfo(attemptedItemInfo.getStartId(), trackId));
LOG.warn("{} But adding trackID back to retry queue as some of" LOG.warn("{} But adding trackID back to retry queue as some of"
+ " the blocks couldn't find matching target nodes in" + " the blocks couldn't find matching target nodes in"
+ " previous SPS iteration.", msg); + " previous SPS iteration.", msg);

View File

@ -29,12 +29,15 @@ import java.util.Map;
import java.util.Queue; import java.util.Queue;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.namenode.FSTreeTraverser.TraverseInfo;
import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo; import org.apache.hadoop.hdfs.server.namenode.StoragePolicySatisfier.ItemInfo;
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
/** /**
* A Class to track the block collection IDs (Inode's ID) for which physical * A Class to track the block collection IDs (Inode's ID) for which physical
* storage movement needed as per the Namespace and StorageReports from DN. * storage movement needed as per the Namespace and StorageReports from DN.
@ -53,11 +56,11 @@ public class BlockStorageMovementNeeded {
new LinkedList<ItemInfo>(); new LinkedList<ItemInfo>();
/** /**
* Map of rootId and number of child's. Number of child's indicate the number * Map of startId and number of child's. Number of child's indicate the
* of files pending to satisfy the policy. * number of files pending to satisfy the policy.
*/ */
private final Map<Long, Integer> pendingWorkForDirectory = private final Map<Long, DirPendingWorkInfo> pendingWorkForDirectory =
new HashMap<Long, Integer>(); new HashMap<Long, DirPendingWorkInfo>();
private final Namesystem namesystem; private final Namesystem namesystem;
@ -66,12 +69,15 @@ public class BlockStorageMovementNeeded {
private final StoragePolicySatisfier sps; private final StoragePolicySatisfier sps;
private Daemon fileInodeIdCollector; private Daemon inodeIdCollector;
private final int maxQueuedItem;
public BlockStorageMovementNeeded(Namesystem namesystem, public BlockStorageMovementNeeded(Namesystem namesystem,
StoragePolicySatisfier sps) { StoragePolicySatisfier sps, int queueLimit) {
this.namesystem = namesystem; this.namesystem = namesystem;
this.sps = sps; this.sps = sps;
this.maxQueuedItem = queueLimit;
} }
/** /**
@ -88,15 +94,24 @@ public class BlockStorageMovementNeeded {
/** /**
* Add the itemInfo to tracking list for which storage movement * Add the itemInfo to tracking list for which storage movement
* expected if necessary. * expected if necessary.
* @param rootId * @param startId
* - root inode id * - start id
* @param itemInfoList * @param itemInfoList
* - List of child in the directory * - List of child in the directory
*/ */
private synchronized void addAll(Long rootId, @VisibleForTesting
List<ItemInfo> itemInfoList) { public synchronized void addAll(long startId,
List<ItemInfo> itemInfoList, boolean scanCompleted) {
storageMovementNeeded.addAll(itemInfoList); storageMovementNeeded.addAll(itemInfoList);
pendingWorkForDirectory.put(rootId, itemInfoList.size()); DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
if (pendingWork == null) {
pendingWork = new DirPendingWorkInfo();
pendingWorkForDirectory.put(startId, pendingWork);
}
pendingWork.addPendingWorkCount(itemInfoList.size());
if (scanCompleted) {
pendingWork.markScanCompleted();
}
} }
/** /**
@ -118,6 +133,25 @@ public class BlockStorageMovementNeeded {
} }
} }
/**
* Returns queue remaining capacity.
*/
public synchronized int remainingCapacity() {
int size = storageMovementNeeded.size();
if (size >= maxQueuedItem) {
return 0;
} else {
return (maxQueuedItem - size);
}
}
/**
* Returns queue size.
*/
public synchronized int size() {
return storageMovementNeeded.size();
}
public synchronized void clearAll() { public synchronized void clearAll() {
spsDirsToBeTraveresed.clear(); spsDirsToBeTraveresed.clear();
storageMovementNeeded.clear(); storageMovementNeeded.clear();
@ -131,20 +165,20 @@ public class BlockStorageMovementNeeded {
public synchronized void removeItemTrackInfo(ItemInfo trackInfo) public synchronized void removeItemTrackInfo(ItemInfo trackInfo)
throws IOException { throws IOException {
if (trackInfo.isDir()) { if (trackInfo.isDir()) {
// If track is part of some root then reduce the pending directory work // If track is part of some start inode then reduce the pending
// count. // directory work count.
long rootId = trackInfo.getRootId(); long startId = trackInfo.getStartId();
INode inode = namesystem.getFSDirectory().getInode(rootId); INode inode = namesystem.getFSDirectory().getInode(startId);
if (inode == null) { if (inode == null) {
// directory deleted just remove it. // directory deleted just remove it.
this.pendingWorkForDirectory.remove(rootId); this.pendingWorkForDirectory.remove(startId);
} else { } else {
if (pendingWorkForDirectory.get(rootId) != null) { DirPendingWorkInfo pendingWork = pendingWorkForDirectory.get(startId);
Integer pendingWork = pendingWorkForDirectory.get(rootId) - 1; if (pendingWork != null) {
pendingWorkForDirectory.put(rootId, pendingWork); pendingWork.decrementPendingWorkCount();
if (pendingWork <= 0) { if (pendingWork.isDirWorkDone()) {
namesystem.removeXattr(rootId, XATTR_SATISFY_STORAGE_POLICY); namesystem.removeXattr(startId, XATTR_SATISFY_STORAGE_POLICY);
pendingWorkForDirectory.remove(rootId); pendingWorkForDirectory.remove(startId);
} }
} }
} }
@ -161,7 +195,7 @@ public class BlockStorageMovementNeeded {
Iterator<ItemInfo> iterator = storageMovementNeeded.iterator(); Iterator<ItemInfo> iterator = storageMovementNeeded.iterator();
while (iterator.hasNext()) { while (iterator.hasNext()) {
ItemInfo next = iterator.next(); ItemInfo next = iterator.next();
if (next.getRootId() == trackId) { if (next.getStartId() == trackId) {
iterator.remove(); iterator.remove();
} }
} }
@ -208,7 +242,17 @@ public class BlockStorageMovementNeeded {
* Take dir tack ID from the spsDirsToBeTraveresed queue and collect child * Take dir tack ID from the spsDirsToBeTraveresed queue and collect child
* ID's to process for satisfy the policy. * ID's to process for satisfy the policy.
*/ */
private class FileInodeIdCollector implements Runnable { private class StorageMovementPendingInodeIdCollector extends FSTreeTraverser
implements Runnable {
private int remainingCapacity = 0;
private List<ItemInfo> currentBatch = new ArrayList<>(maxQueuedItem);
StorageMovementPendingInodeIdCollector(FSDirectory dir) {
super(dir);
}
@Override @Override
public void run() { public void run() {
LOG.info("Starting FileInodeIdCollector!."); LOG.info("Starting FileInodeIdCollector!.");
@ -216,38 +260,36 @@ public class BlockStorageMovementNeeded {
try { try {
if (!namesystem.isInSafeMode()) { if (!namesystem.isInSafeMode()) {
FSDirectory fsd = namesystem.getFSDirectory(); FSDirectory fsd = namesystem.getFSDirectory();
Long rootINodeId = spsDirsToBeTraveresed.poll(); Long startINodeId = spsDirsToBeTraveresed.poll();
if (rootINodeId == null) { if (startINodeId == null) {
// Waiting for SPS path // Waiting for SPS path
synchronized (spsDirsToBeTraveresed) { synchronized (spsDirsToBeTraveresed) {
spsDirsToBeTraveresed.wait(5000); spsDirsToBeTraveresed.wait(5000);
} }
} else { } else {
INode rootInode = fsd.getInode(rootINodeId); INode startInode = fsd.getInode(startINodeId);
if (rootInode != null) { if (startInode != null) {
// TODO : HDFS-12291 try {
// 1. Implement an efficient recursive directory iteration remainingCapacity = remainingCapacity();
// mechanism and satisfies storage policy for all the files readLock();
// under the given directory. traverseDir(startInode.asDirectory(), startINodeId,
// 2. Process files in batches,so datanodes workload can be HdfsFileStatus.EMPTY_NAME,
// handled. new SPSTraverseInfo(startINodeId));
List<ItemInfo> itemInfoList = } finally {
new ArrayList<>(); readUnlock();
for (INode childInode : rootInode.asDirectory()
.getChildrenList(Snapshot.CURRENT_STATE_ID)) {
if (childInode.isFile()
&& childInode.asFile().numBlocks() != 0) {
itemInfoList.add(
new ItemInfo(rootINodeId, childInode.getId()));
}
} }
if (itemInfoList.isEmpty()) { // Mark startInode traverse is done
// satisfy track info is empty, so remove the xAttr from the addAll(startInode.getId(), currentBatch, true);
// directory currentBatch.clear();
namesystem.removeXattr(rootINodeId,
// check if directory was empty and no child added to queue
DirPendingWorkInfo dirPendingWorkInfo =
pendingWorkForDirectory.get(startInode.getId());
if (dirPendingWorkInfo.isDirWorkDone()) {
namesystem.removeXattr(startInode.getId(),
XATTR_SATISFY_STORAGE_POLICY); XATTR_SATISFY_STORAGE_POLICY);
pendingWorkForDirectory.remove(startInode.getId());
} }
addAll(rootINodeId, itemInfoList);
} }
} }
} }
@ -256,17 +298,140 @@ public class BlockStorageMovementNeeded {
} }
} }
} }
@Override
protected void checkPauseForTesting() throws InterruptedException {
// TODO implement if needed
}
@Override
protected boolean processFileInode(INode inode, TraverseInfo traverseInfo)
throws IOException, InterruptedException {
assert getFSDirectory().hasReadLock();
if (LOG.isTraceEnabled()) {
LOG.trace("Processing {} for statisy the policy",
inode.getFullPathName());
}
if (!inode.isFile()) {
return false;
}
if (inode.isFile() && inode.asFile().numBlocks() != 0) {
currentBatch.add(new ItemInfo(
((SPSTraverseInfo) traverseInfo).getStartId(), inode.getId()));
remainingCapacity--;
}
return true;
}
@Override
protected boolean canSubmitCurrentBatch() {
return remainingCapacity <= 0;
}
@Override
protected void checkINodeReady(long startId) throws IOException {
FSNamesystem fsn = ((FSNamesystem) namesystem);
fsn.checkNameNodeSafeMode("NN is in safe mode,"
+ "cannot satisfy the policy.");
// SPS work should be cancelled when NN goes to standby. Just
// double checking for sanity.
fsn.checkOperation(NameNode.OperationCategory.WRITE);
}
@Override
protected void submitCurrentBatch(long startId)
throws IOException, InterruptedException {
// Add current child's to queue
addAll(startId, currentBatch, false);
currentBatch.clear();
}
@Override
protected void throttle() throws InterruptedException {
assert !getFSDirectory().hasReadLock();
assert !namesystem.hasReadLock();
if (LOG.isDebugEnabled()) {
LOG.debug("StorageMovementNeeded queue remaining capacity is zero,"
+ " waiting for some free slots.");
}
remainingCapacity = remainingCapacity();
// wait for queue to be free
while (remainingCapacity <= 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Waiting for storageMovementNeeded queue to be free!");
}
Thread.sleep(5000);
remainingCapacity = remainingCapacity();
}
}
@Override
protected boolean canTraverseDir(INode inode) throws IOException {
return true;
}
} }
public void start() { /**
fileInodeIdCollector = new Daemon(new FileInodeIdCollector()); * Info for directory recursive scan.
fileInodeIdCollector.setName("FileInodeIdCollector"); */
fileInodeIdCollector.start(); public static class DirPendingWorkInfo {
private int pendingWorkCount = 0;
private boolean fullyScanned = false;
/**
* Increment the pending work count for directory.
*/
public synchronized void addPendingWorkCount(int count) {
this.pendingWorkCount = this.pendingWorkCount + count;
}
/**
* Decrement the pending work count for directory one track info is
* completed.
*/
public synchronized void decrementPendingWorkCount() {
this.pendingWorkCount--;
}
/**
* Return true if all the pending work is done and directory fully
* scanned, otherwise false.
*/
public synchronized boolean isDirWorkDone() {
return (pendingWorkCount <= 0 && fullyScanned);
}
/**
* Mark directory scan is completed.
*/
public synchronized void markScanCompleted() {
this.fullyScanned = true;
}
} }
public void stop() { public void init() {
if (fileInodeIdCollector != null) { inodeIdCollector = new Daemon(new StorageMovementPendingInodeIdCollector(
fileInodeIdCollector.interrupt(); namesystem.getFSDirectory()));
inodeIdCollector.setName("FileInodeIdCollector");
inodeIdCollector.start();
}
public void close() {
if (inodeIdCollector != null) {
inodeIdCollector.interrupt();
}
}
class SPSTraverseInfo extends TraverseInfo {
private long startId;
SPSTraverseInfo(long startId) {
this.startId = startId;
}
public long getStartId() {
return startId;
} }
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;

View File

@ -77,7 +77,8 @@ public class StoragePolicySatisfier implements Runnable {
private final BlockStorageMovementNeeded storageMovementNeeded; private final BlockStorageMovementNeeded storageMovementNeeded;
private final BlockStorageMovementAttemptedItems storageMovementsMonitor; private final BlockStorageMovementAttemptedItems storageMovementsMonitor;
private volatile boolean isRunning = false; private volatile boolean isRunning = false;
private int spsWorkMultiplier;
private long blockCount = 0L;
/** /**
* Represents the collective analysis status for all blocks. * Represents the collective analysis status for all blocks.
*/ */
@ -106,7 +107,9 @@ public class StoragePolicySatisfier implements Runnable {
final BlockManager blkManager, Configuration conf) { final BlockManager blkManager, Configuration conf) {
this.namesystem = namesystem; this.namesystem = namesystem;
this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem, this.storageMovementNeeded = new BlockStorageMovementNeeded(namesystem,
this); this, conf.getInt(
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_DEFAULT));
this.blockManager = blkManager; this.blockManager = blkManager;
this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems( this.storageMovementsMonitor = new BlockStorageMovementAttemptedItems(
conf.getLong( conf.getLong(
@ -117,6 +120,7 @@ public class StoragePolicySatisfier implements Runnable {
DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT), DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_SELF_RETRY_TIMEOUT_MILLIS_DEFAULT),
storageMovementNeeded, storageMovementNeeded,
this); this);
this.spsWorkMultiplier = DFSUtil.getSPSWorkMultiplier(conf);
} }
/** /**
@ -143,7 +147,7 @@ public class StoragePolicySatisfier implements Runnable {
// Ensure that all the previously submitted block movements(if any) have to // Ensure that all the previously submitted block movements(if any) have to
// be stopped in all datanodes. // be stopped in all datanodes.
addDropSPSWorkCommandsToAllDNs(); addDropSPSWorkCommandsToAllDNs();
storageMovementNeeded.start(); storageMovementNeeded.init();
storagePolicySatisfierThread = new Daemon(this); storagePolicySatisfierThread = new Daemon(this);
storagePolicySatisfierThread.setName("StoragePolicySatisfier"); storagePolicySatisfierThread.setName("StoragePolicySatisfier");
storagePolicySatisfierThread.start(); storagePolicySatisfierThread.start();
@ -164,7 +168,7 @@ public class StoragePolicySatisfier implements Runnable {
return; return;
} }
storageMovementNeeded.stop(); storageMovementNeeded.close();
storagePolicySatisfierThread.interrupt(); storagePolicySatisfierThread.interrupt();
this.storageMovementsMonitor.stop(); this.storageMovementsMonitor.stop();
@ -268,9 +272,13 @@ public class StoragePolicySatisfier implements Runnable {
} }
} }
} }
// TODO: We can think to make this as configurable later, how frequently int numLiveDn = namesystem.getFSDirectory().getBlockManager()
// we want to check block movements. .getDatanodeManager().getNumLiveDataNodes();
Thread.sleep(3000); if (storageMovementNeeded.size() == 0
|| blockCount > (numLiveDn * spsWorkMultiplier)) {
Thread.sleep(3000);
blockCount = 0L;
}
} catch (Throwable t) { } catch (Throwable t) {
handleException(t); handleException(t);
} }
@ -380,6 +388,11 @@ public class StoragePolicySatisfier implements Runnable {
assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(), assignBlockMovingInfosToCoordinatorDn(blockCollection.getId(),
blockMovingInfos, coordinatorNode); blockMovingInfos, coordinatorNode);
int count = 0;
for (BlockMovingInfo blkMovingInfo : blockMovingInfos) {
count = count + blkMovingInfo.getSources().length;
}
blockCount = blockCount + count;
return status; return status;
} }
@ -840,7 +853,7 @@ public class StoragePolicySatisfier implements Runnable {
* - file inode/blockcollection id. * - file inode/blockcollection id.
*/ */
public void satisfyStoragePolicy(Long inodeId) { public void satisfyStoragePolicy(Long inodeId) {
//For file rootId and trackId is same //For file startId and trackId is same
storageMovementNeeded.add(new ItemInfo(inodeId, inodeId)); storageMovementNeeded.add(new ItemInfo(inodeId, inodeId));
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Added track info for inode {} to block " LOG.debug("Added track info for inode {} to block "
@ -864,19 +877,19 @@ public class StoragePolicySatisfier implements Runnable {
* policy. * policy.
*/ */
public static class ItemInfo { public static class ItemInfo {
private long rootId; private long startId;
private long trackId; private long trackId;
public ItemInfo(long rootId, long trackId) { public ItemInfo(long startId, long trackId) {
this.rootId = rootId; this.startId = startId;
this.trackId = trackId; this.trackId = trackId;
} }
/** /**
* Return the root of the current track Id. * Return the start inode id of the current track Id.
*/ */
public long getRootId() { public long getStartId() {
return rootId; return startId;
} }
/** /**
@ -890,7 +903,7 @@ public class StoragePolicySatisfier implements Runnable {
* Returns true if the tracking path is a directory, false otherwise. * Returns true if the tracking path is a directory, false otherwise.
*/ */
public boolean isDir() { public boolean isDir() {
return (rootId != trackId); return (startId != trackId);
} }
} }
} }

View File

@ -4509,6 +4509,29 @@
</description> </description>
</property> </property>
<property>
<name>dfs.storage.policy.satisfier.queue.limit</name>
<value>1000</value>
<description>
Storage policy satisfier queue size. This queue contains the currently
scheduled file's inode ID for statisfy the policy.
Default value is 1000.
</description>
</property>
<property>
<name>dfs.storage.policy.satisfier.work.multiplier.per.iteration</name>
<value>1</value>
<description>
*Note*: Advanced property. Change with caution.
This determines the total amount of block transfers to begin in
one iteration, for satisfy the policy. The actual number is obtained by
multiplying this multiplier with the total number of live nodes in the
cluster. The result number is the number of blocks to begin transfers
immediately. This number can be any positive, non-zero integer.
</description>
</property>
<property> <property>
<name>dfs.storage.policy.satisfier.recheck.timeout.millis</name> <name>dfs.storage.policy.satisfier.recheck.timeout.millis</name>
<value>300000</value> <value>300000</value>

View File

@ -112,7 +112,7 @@ SPS can be enabled and disabled dynamically without restarting the Namenode.
Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HDFS-10285)](https://issues.apache.org/jira/browse/HDFS-10285) Detailed design documentation can be found at [Storage Policy Satisfier(SPS) (HDFS-10285)](https://issues.apache.org/jira/browse/HDFS-10285)
* **Note**: When user invokes `satisfyStoragePolicy()` API on a directory, SPS will consider the files which are immediate to that directory. Sub-directories won't be considered for satisfying the policy. Its user responsibility to call this API on directories recursively, to track all files under the sub tree. * **Note**: When user invokes `satisfyStoragePolicy()` API on a directory, SPS will scan all sub-directories and consider all the files for satisfy the policy..
* HdfsAdmin API : * HdfsAdmin API :
`public void satisfyStoragePolicy(final Path path) throws IOException` `public void satisfyStoragePolicy(final Path path) throws IOException`
@ -214,7 +214,6 @@ Get the storage policy of a file or a directory.
### Satisfy Storage Policy ### Satisfy Storage Policy
Schedule blocks to move based on file's/directory's current storage policy. Schedule blocks to move based on file's/directory's current storage policy.
Note: For directory case, it will consider immediate files under that directory and it will not consider sub directories recursively.
* Command: * Command:

View File

@ -41,7 +41,7 @@ public class TestBlockStorageMovementAttemptedItems {
public void setup() throws Exception { public void setup() throws Exception {
unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded( unsatisfiedStorageMovementFiles = new BlockStorageMovementNeeded(
Mockito.mock(Namesystem.class), Mockito.mock(Namesystem.class),
Mockito.mock(StoragePolicySatisfier.class)); Mockito.mock(StoragePolicySatisfier.class), 100);
StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class); StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100, bsmAttemptedItems = new BlockStorageMovementAttemptedItems(100,
selfRetryTimeout, unsatisfiedStorageMovementFiles, sps); selfRetryTimeout, unsatisfiedStorageMovementFiles, sps);

View File

@ -191,7 +191,7 @@ public class TestPersistentStoragePolicySatisfier {
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
parentFileName, StorageType.ARCHIVE, 3, timeout, fs); parentFileName, StorageType.ARCHIVE, 3, timeout, fs);
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
childFileName, StorageType.DEFAULT, 3, timeout, fs); childFileName, StorageType.ARCHIVE, 3, timeout, fs);
} finally { } finally {
clusterShutdown(); clusterShutdown();
@ -232,7 +232,9 @@ public class TestPersistentStoragePolicySatisfier {
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
parentFileName, StorageType.ARCHIVE, 2, timeout, fs); parentFileName, StorageType.ARCHIVE, 2, timeout, fs);
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
childFileName, StorageType.DEFAULT, 3, timeout, fs); childFileName, StorageType.DISK, 1, timeout, fs);
DFSTestUtil.waitExpectedStorageType(
childFileName, StorageType.ARCHIVE, 2, timeout, fs);
} finally { } finally {
clusterShutdown(); clusterShutdown();
} }
@ -269,7 +271,7 @@ public class TestPersistentStoragePolicySatisfier {
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
parentFileName, StorageType.ARCHIVE, 3, timeout, fs); parentFileName, StorageType.ARCHIVE, 3, timeout, fs);
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
childFileName, StorageType.DEFAULT, 3, timeout, fs); childFileName, StorageType.ARCHIVE, 3, timeout, fs);
} finally { } finally {
clusterShutdown(); clusterShutdown();
} }

View File

@ -21,6 +21,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KE
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.slf4j.LoggerFactory.getLogger;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
@ -61,8 +64,10 @@ import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer; import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
@ -71,6 +76,12 @@ import com.google.common.base.Supplier;
* moved and finding its suggested target locations to move. * moved and finding its suggested target locations to move.
*/ */
public class TestStoragePolicySatisfier { public class TestStoragePolicySatisfier {
{
GenericTestUtils.setLogLevel(
getLogger(FSTreeTraverser.class), Level.DEBUG);
}
private static final String ONE_SSD = "ONE_SSD"; private static final String ONE_SSD = "ONE_SSD";
private static final String COLD = "COLD"; private static final String COLD = "COLD";
private static final Logger LOG = private static final Logger LOG =
@ -341,7 +352,9 @@ public class TestStoragePolicySatisfier {
// take no effect for the sub-dir's file in the directory. // take no effect for the sub-dir's file in the directory.
DFSTestUtil.waitExpectedStorageType( DFSTestUtil.waitExpectedStorageType(
subFile2, StorageType.DEFAULT, 3, 30000, dfs); subFile2, StorageType.SSD, 1, 30000, dfs);
DFSTestUtil.waitExpectedStorageType(
subFile2, StorageType.DISK, 2, 30000, dfs);
} finally { } finally {
shutdownCluster(); shutdownCluster();
} }
@ -1083,6 +1096,368 @@ public class TestStoragePolicySatisfier {
} }
} }
/**
* Test SPS for empty directory, xAttr should be removed.
*/
@Test(timeout = 300000)
public void testSPSForEmptyDirectory() throws IOException, TimeoutException,
InterruptedException {
MiniDFSCluster cluster = null;
try {
Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
true);
cluster = new MiniDFSCluster.Builder(conf).build();
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
Path emptyDir = new Path("/emptyDir");
fs.mkdirs(emptyDir);
fs.satisfyStoragePolicy(emptyDir);
// Make sure satisfy xattr has been removed.
DFSTestUtil.waitForXattrRemoved("/emptyDir",
XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
/**
* Test SPS for not exist directory.
*/
@Test(timeout = 300000)
public void testSPSForNonExistDirectory() throws Exception {
MiniDFSCluster cluster = null;
try {
Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
true);
cluster = new MiniDFSCluster.Builder(conf).build();
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
Path emptyDir = new Path("/emptyDir");
try {
fs.satisfyStoragePolicy(emptyDir);
fail("FileNotFoundException should throw");
} catch (FileNotFoundException e) {
// nothing to do
}
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
/**
* Test SPS for directory tree which doesn't have files.
*/
@Test(timeout = 300000)
public void testSPSWithDirectoryTreeWithoutFile() throws Exception {
MiniDFSCluster cluster = null;
try {
Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
true);
cluster = new MiniDFSCluster.Builder(conf).build();
cluster.waitActive();
// Create directories
/*
* root
* |
* A--------C--------D
* |
* G----H----I
* |
* O
*/
DistributedFileSystem fs = cluster.getFileSystem();
fs.mkdirs(new Path("/root/C/H/O"));
fs.mkdirs(new Path("/root/A"));
fs.mkdirs(new Path("/root/D"));
fs.mkdirs(new Path("/root/C/G"));
fs.mkdirs(new Path("/root/C/I"));
fs.satisfyStoragePolicy(new Path("/root"));
// Make sure satisfy xattr has been removed.
DFSTestUtil.waitForXattrRemoved("/root",
XATTR_SATISFY_STORAGE_POLICY, cluster.getNamesystem(), 30000);
} finally {
if (cluster != null) {
cluster.shutdown();
}
}
}
/**
* Test SPS for directory which has multilevel directories.
*/
@Test(timeout = 300000)
public void testMultipleLevelDirectoryForSatisfyStoragePolicy()
throws Exception {
try {
StorageType[][] diskTypes = new StorageType[][] {
{StorageType.DISK, StorageType.ARCHIVE},
{StorageType.ARCHIVE, StorageType.SSD},
{StorageType.DISK, StorageType.DISK}};
config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
true);
hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
storagesPerDatanode, capacity);
dfs = hdfsCluster.getFileSystem();
createDirectoryTree(dfs);
List<String> files = getDFSListOfTree();
dfs.setStoragePolicy(new Path("/root"), COLD);
dfs.satisfyStoragePolicy(new Path("/root"));
for (String fileName : files) {
// Wait till the block is moved to ARCHIVE
DFSTestUtil.waitExpectedStorageType(fileName, StorageType.ARCHIVE, 2,
30000, dfs);
}
} finally {
shutdownCluster();
}
}
/**
* Test SPS for batch processing.
*/
@Test(timeout = 300000)
public void testBatchProcessingForSPSDirectory() throws Exception {
try {
StorageType[][] diskTypes = new StorageType[][] {
{StorageType.DISK, StorageType.ARCHIVE},
{StorageType.ARCHIVE, StorageType.SSD},
{StorageType.DISK, StorageType.DISK}};
config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
config.setBoolean(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_ENABLED_KEY,
true);
// Set queue max capacity
config.setInt(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_QUEUE_LIMIT_KEY,
5);
hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
storagesPerDatanode, capacity);
dfs = hdfsCluster.getFileSystem();
createDirectoryTree(dfs);
List<String> files = getDFSListOfTree();
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(LogFactory
.getLog(FSTreeTraverser.class));
dfs.setStoragePolicy(new Path("/root"), COLD);
dfs.satisfyStoragePolicy(new Path("/root"));
for (String fileName : files) {
// Wait till the block is moved to ARCHIVE
DFSTestUtil.waitExpectedStorageType(fileName, StorageType.ARCHIVE, 2,
30000, dfs);
}
waitForBlocksMovementResult(files.size(), 30000);
String expectedLogMessage = "StorageMovementNeeded queue remaining"
+ " capacity is zero";
assertTrue("Log output does not contain expected log message: "
+ expectedLogMessage, logs.getOutput().contains(expectedLogMessage));
} finally {
shutdownCluster();
}
}
/**
* Test traverse when parent got deleted.
* 1. Delete /root when traversing Q
* 2. U, R, S should not be in queued.
*/
@Test
public void testTraverseWhenParentDeleted() throws Exception {
StorageType[][] diskTypes = new StorageType[][] {
{StorageType.DISK, StorageType.ARCHIVE},
{StorageType.ARCHIVE, StorageType.SSD},
{StorageType.DISK, StorageType.DISK}};
config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
storagesPerDatanode, capacity);
dfs = hdfsCluster.getFileSystem();
createDirectoryTree(dfs);
List<String> expectedTraverseOrder = getDFSListOfTree();
//Remove files which will not be traverse when parent is deleted
expectedTraverseOrder.remove("/root/D/L/R");
expectedTraverseOrder.remove("/root/D/L/S");
expectedTraverseOrder.remove("/root/D/L/Q/U");
FSDirectory fsDir = hdfsCluster.getNamesystem().getFSDirectory();
//Queue limit can control the traverse logic to wait for some free
//entry in queue. After 10 files, traverse control will be on U.
StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
Mockito.when(sps.isRunning()).thenReturn(true);
BlockStorageMovementNeeded movmentNeededQueue =
new BlockStorageMovementNeeded(fsDir.getFSNamesystem(), sps, 10);
INode rootINode = fsDir.getINode("/root");
movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
movmentNeededQueue.init();
//Wait for thread to reach U.
Thread.sleep(1000);
dfs.delete(new Path("/root/D/L"), true);
// Remove 10 element and make queue free, So other traversing will start.
for (int i = 0; i < 10; i++) {
String path = expectedTraverseOrder.remove(0);
long trackId = movmentNeededQueue.get().getTrackId();
INode inode = fsDir.getInode(trackId);
assertTrue("Failed to traverse tree, expected " + path + " but got "
+ inode.getFullPathName(), path.equals(inode.getFullPathName()));
}
//Wait to finish tree traverse
Thread.sleep(5000);
// Check other element traversed in order and R,S should not be added in
// queue which we already removed from expected list
for (String path : expectedTraverseOrder) {
long trackId = movmentNeededQueue.get().getTrackId();
INode inode = fsDir.getInode(trackId);
assertTrue("Failed to traverse tree, expected " + path + " but got "
+ inode.getFullPathName(), path.equals(inode.getFullPathName()));
}
dfs.delete(new Path("/root"), true);
}
/**
* Test traverse when root parent got deleted.
* 1. Delete L when traversing Q
* 2. E, M, U, R, S should not be in queued.
*/
@Test
public void testTraverseWhenRootParentDeleted() throws Exception {
StorageType[][] diskTypes = new StorageType[][] {
{StorageType.DISK, StorageType.ARCHIVE},
{StorageType.ARCHIVE, StorageType.SSD},
{StorageType.DISK, StorageType.DISK}};
config.setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
hdfsCluster = startCluster(config, diskTypes, diskTypes.length,
storagesPerDatanode, capacity);
dfs = hdfsCluster.getFileSystem();
createDirectoryTree(dfs);
List<String> expectedTraverseOrder = getDFSListOfTree();
// Remove files which will not be traverse when parent is deleted
expectedTraverseOrder.remove("/root/D/L/R");
expectedTraverseOrder.remove("/root/D/L/S");
expectedTraverseOrder.remove("/root/D/L/Q/U");
expectedTraverseOrder.remove("/root/D/M");
expectedTraverseOrder.remove("/root/E");
FSDirectory fsDir = hdfsCluster.getNamesystem().getFSDirectory();
StoragePolicySatisfier sps = Mockito.mock(StoragePolicySatisfier.class);
Mockito.when(sps.isRunning()).thenReturn(true);
// Queue limit can control the traverse logic to wait for some free
// entry in queue. After 10 files, traverse control will be on U.
BlockStorageMovementNeeded movmentNeededQueue =
new BlockStorageMovementNeeded(fsDir.getFSNamesystem(), sps, 10);
movmentNeededQueue.init();
INode rootINode = fsDir.getINode("/root");
movmentNeededQueue.addToPendingDirQueue(rootINode.getId());
// Wait for thread to reach U.
Thread.sleep(1000);
dfs.delete(new Path("/root/D/L"), true);
// Remove 10 element and make queue free, So other traversing will start.
for (int i = 0; i < 10; i++) {
String path = expectedTraverseOrder.remove(0);
long trackId = movmentNeededQueue.get().getTrackId();
INode inode = fsDir.getInode(trackId);
assertTrue("Failed to traverse tree, expected " + path + " but got "
+ inode.getFullPathName(), path.equals(inode.getFullPathName()));
}
// Wait to finish tree traverse
Thread.sleep(5000);
// Check other element traversed in order and E, M, U, R, S should not be
// added in queue which we already removed from expected list
for (String path : expectedTraverseOrder) {
long trackId = movmentNeededQueue.get().getTrackId();
INode inode = fsDir.getInode(trackId);
assertTrue("Failed to traverse tree, expected " + path + " but got "
+ inode.getFullPathName(), path.equals(inode.getFullPathName()));
}
dfs.delete(new Path("/root"), true);
}
private static void createDirectoryTree(DistributedFileSystem dfs)
throws Exception {
// tree structure
/*
* root
* |
* A--------B--------C--------D--------E
* | |
* F----G----H----I J----K----L----M
* | |
* N----O----P Q----R----S
* | |
* T U
*/
// create root Node and child
dfs.mkdirs(new Path("/root"));
DFSTestUtil.createFile(dfs, new Path("/root/A"), 1024, (short) 3, 0);
dfs.mkdirs(new Path("/root/B"));
DFSTestUtil.createFile(dfs, new Path("/root/C"), 1024, (short) 3, 0);
dfs.mkdirs(new Path("/root/D"));
DFSTestUtil.createFile(dfs, new Path("/root/E"), 1024, (short) 3, 0);
// Create /root/B child
DFSTestUtil.createFile(dfs, new Path("/root/B/F"), 1024, (short) 3, 0);
dfs.mkdirs(new Path("/root/B/G"));
DFSTestUtil.createFile(dfs, new Path("/root/B/H"), 1024, (short) 3, 0);
DFSTestUtil.createFile(dfs, new Path("/root/B/I"), 1024, (short) 3, 0);
// Create /root/D child
DFSTestUtil.createFile(dfs, new Path("/root/D/J"), 1024, (short) 3, 0);
DFSTestUtil.createFile(dfs, new Path("/root/D/K"), 1024, (short) 3, 0);
dfs.mkdirs(new Path("/root/D/L"));
DFSTestUtil.createFile(dfs, new Path("/root/D/M"), 1024, (short) 3, 0);
// Create /root/B/G child
DFSTestUtil.createFile(dfs, new Path("/root/B/G/N"), 1024, (short) 3, 0);
DFSTestUtil.createFile(dfs, new Path("/root/B/G/O"), 1024, (short) 3, 0);
dfs.mkdirs(new Path("/root/B/G/P"));
// Create /root/D/L child
dfs.mkdirs(new Path("/root/D/L/Q"));
DFSTestUtil.createFile(dfs, new Path("/root/D/L/R"), 1024, (short) 3, 0);
DFSTestUtil.createFile(dfs, new Path("/root/D/L/S"), 1024, (short) 3, 0);
// Create /root/B/G/P child
DFSTestUtil.createFile(dfs, new Path("/root/B/G/P/T"), 1024, (short) 3, 0);
// Create /root/D/L/Q child
DFSTestUtil.createFile(dfs, new Path("/root/D/L/Q/U"), 1024, (short) 3, 0);
}
private List<String> getDFSListOfTree() {
List<String> dfsList = new ArrayList<>();
dfsList.add("/root/A");
dfsList.add("/root/B/F");
dfsList.add("/root/B/G/N");
dfsList.add("/root/B/G/O");
dfsList.add("/root/B/G/P/T");
dfsList.add("/root/B/H");
dfsList.add("/root/B/I");
dfsList.add("/root/C");
dfsList.add("/root/D/J");
dfsList.add("/root/D/K");
dfsList.add("/root/D/L/Q/U");
dfsList.add("/root/D/L/R");
dfsList.add("/root/D/L/S");
dfsList.add("/root/D/M");
dfsList.add("/root/E");
return dfsList;
}
private String createFileAndSimulateFavoredNodes(int favoredNodesCount) private String createFileAndSimulateFavoredNodes(int favoredNodesCount)
throws IOException { throws IOException {
ArrayList<DataNode> dns = hdfsCluster.getDataNodes(); ArrayList<DataNode> dns = hdfsCluster.getDataNodes();