|
@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
|
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
|
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
-import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
|
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryAttributes;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
@@ -97,15 +96,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
|
|
|
|
/** clear the created list */
|
|
/** clear the created list */
|
|
private QuotaCounts destroyCreatedList(
|
|
private QuotaCounts destroyCreatedList(
|
|
- final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
|
|
|
|
- final BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
- final List<INode> removedINodes, List<Long> removedUCFiles) {
|
|
|
|
|
|
+ INode.ReclaimContext reclaimContext, final INodeDirectory currentINode) {
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
final List<INode> createdList = getList(ListType.CREATED);
|
|
final List<INode> createdList = getList(ListType.CREATED);
|
|
for (INode c : createdList) {
|
|
for (INode c : createdList) {
|
|
- c.computeQuotaUsage(bsps, counts, true);
|
|
|
|
- c.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
|
|
|
|
- removedUCFiles);
|
|
|
|
|
|
+ c.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, true);
|
|
|
|
+ c.destroyAndCollectBlocks(reclaimContext);
|
|
// c should be contained in the children list, remove it
|
|
// c should be contained in the children list, remove it
|
|
currentINode.removeChild(c);
|
|
currentINode.removeChild(c);
|
|
}
|
|
}
|
|
@@ -114,16 +110,12 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
}
|
|
}
|
|
|
|
|
|
/** clear the deleted list */
|
|
/** clear the deleted list */
|
|
- private QuotaCounts destroyDeletedList(
|
|
|
|
- final BlockStoragePolicySuite bsps,
|
|
|
|
- final BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
- final List<INode> removedINodes, List<Long> removedUCFiles) {
|
|
|
|
|
|
+ private QuotaCounts destroyDeletedList(INode.ReclaimContext reclaimContext) {
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
final List<INode> deletedList = getList(ListType.DELETED);
|
|
final List<INode> deletedList = getList(ListType.DELETED);
|
|
for (INode d : deletedList) {
|
|
for (INode d : deletedList) {
|
|
- d.computeQuotaUsage(bsps, counts, false);
|
|
|
|
- d.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
|
|
|
|
- removedUCFiles);
|
|
|
|
|
|
+ d.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
|
|
|
|
+ d.destroyAndCollectBlocks(reclaimContext);
|
|
}
|
|
}
|
|
deletedList.clear();
|
|
deletedList.clear();
|
|
return counts;
|
|
return counts;
|
|
@@ -211,19 +203,17 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
QuotaCounts combinePosteriorAndCollectBlocks(
|
|
QuotaCounts combinePosteriorAndCollectBlocks(
|
|
- final BlockStoragePolicySuite bsps, final INodeDirectory currentDir,
|
|
|
|
- final DirectoryDiff posterior,
|
|
|
|
- final BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
- final List<INode> removedINodes) {
|
|
|
|
|
|
+ final INode.ReclaimContext reclaimContext,
|
|
|
|
+ final INodeDirectory currentDir,
|
|
|
|
+ final DirectoryDiff posterior) {
|
|
final QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
final QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
|
|
diff.combinePosterior(posterior.diff, new Diff.Processor<INode>() {
|
|
/** Collect blocks for deleted files. */
|
|
/** Collect blocks for deleted files. */
|
|
@Override
|
|
@Override
|
|
public void process(INode inode) {
|
|
public void process(INode inode) {
|
|
if (inode != null) {
|
|
if (inode != null) {
|
|
- inode.computeQuotaUsage(bsps, counts, false);
|
|
|
|
- inode.destroyAndCollectBlocks(bsps, collectedBlocks, removedINodes,
|
|
|
|
- null);
|
|
|
|
|
|
+ inode.computeQuotaUsage(reclaimContext.storagePolicySuite(), counts, false);
|
|
|
|
+ inode.destroyAndCollectBlocks(reclaimContext);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
});
|
|
});
|
|
@@ -322,12 +312,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
QuotaCounts destroyDiffAndCollectBlocks(
|
|
QuotaCounts destroyDiffAndCollectBlocks(
|
|
- BlockStoragePolicySuite bsps, INodeDirectory currentINode,
|
|
|
|
- BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
|
|
|
|
|
|
+ INode.ReclaimContext reclaimContext, INodeDirectory currentINode) {
|
|
// this diff has been deleted
|
|
// this diff has been deleted
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
- counts.add(diff.destroyDeletedList(bsps, collectedBlocks, removedINodes,
|
|
|
|
- null));
|
|
|
|
|
|
+ counts.add(diff.destroyDeletedList(reclaimContext));
|
|
INodeDirectoryAttributes snapshotINode = getSnapshotINode();
|
|
INodeDirectoryAttributes snapshotINode = getSnapshotINode();
|
|
if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
|
|
if (snapshotINode != null && snapshotINode.getAclFeature() != null) {
|
|
AclStorage.removeAclFeature(snapshotINode.getAclFeature());
|
|
AclStorage.removeAclFeature(snapshotINode.getAclFeature());
|
|
@@ -412,25 +400,24 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
* Destroy a subtree under a DstReference node.
|
|
* Destroy a subtree under a DstReference node.
|
|
*/
|
|
*/
|
|
public static void destroyDstSubtree(
|
|
public static void destroyDstSubtree(
|
|
- final BlockStoragePolicySuite bsps, INode inode, final int snapshot,
|
|
|
|
- final int prior, final BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
- final List<INode> removedINodes, List<Long> removedUCFiles) throws QuotaExceededException {
|
|
|
|
|
|
+ INode.ReclaimContext reclaimContext, INode inode, final int snapshot,
|
|
|
|
+ final int prior) throws QuotaExceededException {
|
|
Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID);
|
|
Preconditions.checkArgument(prior != Snapshot.NO_SNAPSHOT_ID);
|
|
if (inode.isReference()) {
|
|
if (inode.isReference()) {
|
|
if (inode instanceof INodeReference.WithName
|
|
if (inode instanceof INodeReference.WithName
|
|
&& snapshot != Snapshot.CURRENT_STATE_ID) {
|
|
&& snapshot != Snapshot.CURRENT_STATE_ID) {
|
|
// this inode has been renamed before the deletion of the DstReference
|
|
// this inode has been renamed before the deletion of the DstReference
|
|
// subtree
|
|
// subtree
|
|
- inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes,
|
|
|
|
- removedUCFiles);
|
|
|
|
|
|
+ inode.cleanSubtree(reclaimContext,
|
|
|
|
+ snapshot, prior);
|
|
} else {
|
|
} else {
|
|
// for DstReference node, continue this process to its subtree
|
|
// for DstReference node, continue this process to its subtree
|
|
- destroyDstSubtree(bsps, inode.asReference().getReferredINode(), snapshot,
|
|
|
|
- prior, collectedBlocks, removedINodes, removedUCFiles);
|
|
|
|
|
|
+ destroyDstSubtree(reclaimContext,
|
|
|
|
+ inode.asReference().getReferredINode(), snapshot,
|
|
|
|
+ prior);
|
|
}
|
|
}
|
|
} else if (inode.isFile()) {
|
|
} else if (inode.isFile()) {
|
|
- inode.cleanSubtree(bsps, snapshot, prior, collectedBlocks, removedINodes,
|
|
|
|
- removedUCFiles);
|
|
|
|
|
|
+ inode.cleanSubtree(reclaimContext, snapshot, prior);
|
|
} else if (inode.isDirectory()) {
|
|
} else if (inode.isDirectory()) {
|
|
Map<INode, INode> excludedNodes = null;
|
|
Map<INode, INode> excludedNodes = null;
|
|
INodeDirectory dir = inode.asDirectory();
|
|
INodeDirectory dir = inode.asDirectory();
|
|
@@ -444,21 +431,19 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
}
|
|
}
|
|
|
|
|
|
if (snapshot != Snapshot.CURRENT_STATE_ID) {
|
|
if (snapshot != Snapshot.CURRENT_STATE_ID) {
|
|
- diffList.deleteSnapshotDiff(bsps, snapshot, prior, dir, collectedBlocks,
|
|
|
|
- removedINodes);
|
|
|
|
|
|
+ diffList.deleteSnapshotDiff(reclaimContext,
|
|
|
|
+ snapshot, prior, dir);
|
|
}
|
|
}
|
|
priorDiff = diffList.getDiffById(prior);
|
|
priorDiff = diffList.getDiffById(prior);
|
|
if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
|
|
if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
|
|
- priorDiff.diff.destroyCreatedList(bsps, dir, collectedBlocks,
|
|
|
|
- removedINodes, removedUCFiles);
|
|
|
|
|
|
+ priorDiff.diff.destroyCreatedList(reclaimContext, dir);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
for (INode child : inode.asDirectory().getChildrenList(prior)) {
|
|
for (INode child : inode.asDirectory().getChildrenList(prior)) {
|
|
if (excludedNodes != null && excludedNodes.containsKey(child)) {
|
|
if (excludedNodes != null && excludedNodes.containsKey(child)) {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
- destroyDstSubtree(bsps, child, snapshot, prior, collectedBlocks,
|
|
|
|
- removedINodes, removedUCFiles);
|
|
|
|
|
|
+ destroyDstSubtree(reclaimContext, child, snapshot, prior);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -466,18 +451,15 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
/**
|
|
/**
|
|
* Clean an inode while we move it from the deleted list of post to the
|
|
* Clean an inode while we move it from the deleted list of post to the
|
|
* deleted list of prior.
|
|
* deleted list of prior.
|
|
- * @param bsps The block storage policy suite.
|
|
|
|
|
|
+ * @param reclaimContext blocks and inodes that need to be reclaimed
|
|
* @param inode The inode to clean.
|
|
* @param inode The inode to clean.
|
|
* @param post The post snapshot.
|
|
* @param post The post snapshot.
|
|
* @param prior The id of the prior snapshot.
|
|
* @param prior The id of the prior snapshot.
|
|
- * @param collectedBlocks Used to collect blocks for later deletion.
|
|
|
|
- * @param removedUCFiles
|
|
|
|
* @return Quota usage update.
|
|
* @return Quota usage update.
|
|
*/
|
|
*/
|
|
private static QuotaCounts cleanDeletedINode(
|
|
private static QuotaCounts cleanDeletedINode(
|
|
- final BlockStoragePolicySuite bsps, INode inode, final int post, final int prior,
|
|
|
|
- final BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
- final List<INode> removedINodes, List<Long> removedUCFiles) {
|
|
|
|
|
|
+ INode.ReclaimContext reclaimContext, INode inode, final int post,
|
|
|
|
+ final int prior) {
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
Deque<INode> queue = new ArrayDeque<INode>();
|
|
Deque<INode> queue = new ArrayDeque<INode>();
|
|
queue.addLast(inode);
|
|
queue.addLast(inode);
|
|
@@ -486,15 +468,13 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
if (topNode instanceof INodeReference.WithName) {
|
|
if (topNode instanceof INodeReference.WithName) {
|
|
INodeReference.WithName wn = (INodeReference.WithName) topNode;
|
|
INodeReference.WithName wn = (INodeReference.WithName) topNode;
|
|
if (wn.getLastSnapshotId() >= post) {
|
|
if (wn.getLastSnapshotId() >= post) {
|
|
- wn.cleanSubtree(bsps, post, prior, collectedBlocks, removedINodes,
|
|
|
|
- removedUCFiles);
|
|
|
|
|
|
+ wn.cleanSubtree(reclaimContext, post, prior);
|
|
}
|
|
}
|
|
// For DstReference node, since the node is not in the created list of
|
|
// For DstReference node, since the node is not in the created list of
|
|
// prior, we should treat it as regular file/dir
|
|
// prior, we should treat it as regular file/dir
|
|
} else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
|
|
} else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
|
|
INodeFile file = topNode.asFile();
|
|
INodeFile file = topNode.asFile();
|
|
- counts.add(file.getDiffs().deleteSnapshotDiff(bsps, post, prior, file,
|
|
|
|
- collectedBlocks, removedINodes));
|
|
|
|
|
|
+ counts.add(file.getDiffs().deleteSnapshotDiff(reclaimContext, post, prior, file));
|
|
} else if (topNode.isDirectory()) {
|
|
} else if (topNode.isDirectory()) {
|
|
INodeDirectory dir = topNode.asDirectory();
|
|
INodeDirectory dir = topNode.asDirectory();
|
|
ChildrenDiff priorChildrenDiff = null;
|
|
ChildrenDiff priorChildrenDiff = null;
|
|
@@ -505,8 +485,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
DirectoryDiff priorDiff = sf.getDiffs().getDiffById(prior);
|
|
DirectoryDiff priorDiff = sf.getDiffs().getDiffById(prior);
|
|
if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
|
|
if (priorDiff != null && priorDiff.getSnapshotId() == prior) {
|
|
priorChildrenDiff = priorDiff.getChildrenDiff();
|
|
priorChildrenDiff = priorDiff.getChildrenDiff();
|
|
- counts.add(priorChildrenDiff.destroyCreatedList(bsps, dir,
|
|
|
|
- collectedBlocks, removedINodes, removedUCFiles));
|
|
|
|
|
|
+ counts.add(priorChildrenDiff.destroyCreatedList(reclaimContext,
|
|
|
|
+ dir));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -636,13 +616,11 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
return child;
|
|
return child;
|
|
}
|
|
}
|
|
|
|
|
|
- public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode,
|
|
|
|
- final BlocksMapUpdateInfo collectedBlocks, final List<INode>
|
|
|
|
- removedINodes, final List<Long> removedUCFiles) {
|
|
|
|
|
|
+ public void clear(
|
|
|
|
+ INode.ReclaimContext reclaimContext, INodeDirectory currentINode) {
|
|
// destroy its diff list
|
|
// destroy its diff list
|
|
for (DirectoryDiff diff : diffs) {
|
|
for (DirectoryDiff diff : diffs) {
|
|
- diff.destroyDiffAndCollectBlocks(bsps, currentINode, collectedBlocks,
|
|
|
|
- removedINodes);
|
|
|
|
|
|
+ diff.destroyDiffAndCollectBlocks(reclaimContext, currentINode);
|
|
}
|
|
}
|
|
diffs.clear();
|
|
diffs.clear();
|
|
}
|
|
}
|
|
@@ -729,9 +707,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
}
|
|
}
|
|
|
|
|
|
public QuotaCounts cleanDirectory(
|
|
public QuotaCounts cleanDirectory(
|
|
- final BlockStoragePolicySuite bsps, final INodeDirectory currentINode,
|
|
|
|
- final int snapshot, int prior, final BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
- final List<INode> removedINodes, List<Long> removedUCFiles) {
|
|
|
|
|
|
+ INode.ReclaimContext reclaimContext, final INodeDirectory currentINode,
|
|
|
|
+ final int snapshot, int prior) {
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
QuotaCounts counts = new QuotaCounts.Builder().build();
|
|
Map<INode, INode> priorCreated = null;
|
|
Map<INode, INode> priorCreated = null;
|
|
Map<INode, INode> priorDeleted = null;
|
|
Map<INode, INode> priorDeleted = null;
|
|
@@ -740,11 +717,11 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
// delete everything in created list
|
|
// delete everything in created list
|
|
DirectoryDiff lastDiff = diffs.getLast();
|
|
DirectoryDiff lastDiff = diffs.getLast();
|
|
if (lastDiff != null) {
|
|
if (lastDiff != null) {
|
|
- counts.add(lastDiff.diff.destroyCreatedList(bsps, currentINode,
|
|
|
|
- collectedBlocks, removedINodes, removedUCFiles));
|
|
|
|
|
|
+ counts.add(lastDiff.diff.destroyCreatedList(reclaimContext,
|
|
|
|
+ currentINode));
|
|
}
|
|
}
|
|
- counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
|
|
|
|
- collectedBlocks, removedINodes, removedUCFiles, priorDeleted));
|
|
|
|
|
|
+ counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
|
|
|
|
+ snapshot, prior, priorDeleted));
|
|
} else {
|
|
} else {
|
|
// update prior
|
|
// update prior
|
|
prior = getDiffs().updatePrior(snapshot, prior);
|
|
prior = getDiffs().updatePrior(snapshot, prior);
|
|
@@ -760,10 +737,10 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- counts.add(getDiffs().deleteSnapshotDiff(bsps, snapshot, prior,
|
|
|
|
- currentINode, collectedBlocks, removedINodes));
|
|
|
|
- counts.add(currentINode.cleanSubtreeRecursively(bsps, snapshot, prior,
|
|
|
|
- collectedBlocks, removedINodes, removedUCFiles, priorDeleted));
|
|
|
|
|
|
+ counts.add(getDiffs().deleteSnapshotDiff(reclaimContext, snapshot, prior,
|
|
|
|
+ currentINode));
|
|
|
|
+ counts.add(currentINode.cleanSubtreeRecursively(reclaimContext,
|
|
|
|
+ snapshot, prior, priorDeleted));
|
|
|
|
|
|
// check priorDiff again since it may be created during the diff deletion
|
|
// check priorDiff again since it may be created during the diff deletion
|
|
if (prior != Snapshot.NO_SNAPSHOT_ID) {
|
|
if (prior != Snapshot.NO_SNAPSHOT_ID) {
|
|
@@ -779,8 +756,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
for (INode cNode : priorDiff.getChildrenDiff().getList(
|
|
for (INode cNode : priorDiff.getChildrenDiff().getList(
|
|
ListType.CREATED)) {
|
|
ListType.CREATED)) {
|
|
if (priorCreated.containsKey(cNode)) {
|
|
if (priorCreated.containsKey(cNode)) {
|
|
- counts.add(cNode.cleanSubtree(bsps, snapshot, Snapshot.NO_SNAPSHOT_ID,
|
|
|
|
- collectedBlocks, removedINodes, removedUCFiles));
|
|
|
|
|
|
+ counts.add(cNode.cleanSubtree(reclaimContext,
|
|
|
|
+ snapshot, Snapshot.NO_SNAPSHOT_ID));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -796,8 +773,8 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
for (INode dNode : priorDiff.getChildrenDiff().getList(
|
|
for (INode dNode : priorDiff.getChildrenDiff().getList(
|
|
ListType.DELETED)) {
|
|
ListType.DELETED)) {
|
|
if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
|
|
if (priorDeleted == null || !priorDeleted.containsKey(dNode)) {
|
|
- counts.add(cleanDeletedINode(bsps, dNode, snapshot, prior,
|
|
|
|
- collectedBlocks, removedINodes, removedUCFiles));
|
|
|
|
|
|
+ counts.add(cleanDeletedINode(reclaimContext,
|
|
|
|
+ dNode, snapshot, prior));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|