|
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.Content;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.Content.CountsMap.Key;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
|
@@ -115,11 +116,14 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
Quota.Counts counts = Quota.Counts.newInstance();
|
|
|
final List<INode> deletedList = getList(ListType.DELETED);
|
|
|
for (INode d : deletedList) {
|
|
|
- if (INodeReference.tryRemoveReference(d) <= 0) {
|
|
|
- d.computeQuotaUsage(counts, false);
|
|
|
- d.destroyAndCollectBlocks(collectedBlocks, removedINodes);
|
|
|
- } else {
|
|
|
- refNodes.add(d.asReference());
|
|
|
+ d.computeQuotaUsage(counts, false);
|
|
|
+ d.destroyAndCollectBlocks(collectedBlocks, removedINodes);
|
|
|
+ if (d.isReference()) {
|
|
|
+ INodeReference.WithCount wc =
|
|
|
+ (INodeReference.WithCount) d.asReference().getReferredINode();
|
|
|
+ if (wc.getReferenceCount() > 0) {
|
|
|
+ refNodes.add(d.asReference());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
deletedList.clear();
|
|
@@ -271,25 +275,36 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
@Override
|
|
|
public void process(INode inode) {
|
|
|
if (inode != null) {
|
|
|
- if (INodeReference.tryRemoveReference(inode) <= 0) {
|
|
|
- inode.computeQuotaUsage(counts, false);
|
|
|
- inode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
|
|
|
- } else {
|
|
|
- // if the node is a reference node, we should continue the
|
|
|
- // snapshot deletion process
|
|
|
- try {
|
|
|
- // use null as prior here because we are handling a reference
|
|
|
- // node stored in the created list of a snapshot diff. This
|
|
|
- // snapshot diff must be associated with the latest snapshot of
|
|
|
- // the dst tree before the rename operation. In this scenario,
|
|
|
- // the prior snapshot should be the one created in the src tree,
|
|
|
- // and it can be identified by the cleanSubtree since we call
|
|
|
- // recordModification before the rename.
|
|
|
- counts.add(inode.cleanSubtree(posterior.snapshot, null,
|
|
|
- collectedBlocks, removedINodes));
|
|
|
- } catch (QuotaExceededException e) {
|
|
|
- String error = "should not have QuotaExceededException while deleting snapshot";
|
|
|
- LOG.error(error, e);
|
|
|
+ inode.computeQuotaUsage(counts, false);
|
|
|
+ inode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
|
|
|
+
|
|
|
+ boolean handleRef = false;
|
|
|
+ if (inode.isReference()) {
|
|
|
+ INodeReference.WithCount wc = (INodeReference.WithCount) inode
|
|
|
+ .asReference().getReferredINode();
|
|
|
+ if (wc.getReferenceCount() > 0) {
|
|
|
+ handleRef = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (handleRef) {
|
|
|
+ final Snapshot postSnapshot = posterior.snapshot;
|
|
|
+ if (inode instanceof INodeReference.DstReference) {
|
|
|
+ // we are handling a reference node and its subtree stored in
|
|
|
+ // the created list of a snapshot diff, which must be associated
|
|
|
+ // with the latest snapshot of the dst tree before the rename
|
|
|
+ // operation.
|
|
|
+ destroyDstSnapshot(inode, postSnapshot, null, collectedBlocks,
|
|
|
+ removedINodes);
|
|
|
+ } else if (inode instanceof INodeReference.WithName) {
|
|
|
+ // the inode should be renamed again. We only need to delete
|
|
|
+ // postSnapshot in its subtree.
|
|
|
+ try {
|
|
|
+ inode.cleanSubtree(postSnapshot, null, collectedBlocks,
|
|
|
+ removedINodes);
|
|
|
+ } catch (QuotaExceededException e) {
|
|
|
+ LOG.error("Error: should not throw QuotaExceededException", e);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -297,6 +312,74 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
});
|
|
|
return counts;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * For a reference node, delete its first snapshot associated with the dst
|
|
|
+ * tree of a rename operation, i.e., the snapshot diff is associated with
|
|
|
+ * the latest snapshot of the dst tree before the rename operation. The
|
|
|
+ * difference between this process and a regular snapshot deletion
|
|
|
+ * process is that we need to delete everything created after the rename,
|
|
|
+ * i.e., we should destroy the whole created list of the referred node.
|
|
|
+ */
|
|
|
+ private Quota.Counts destroyDstSnapshot(INode inode, final Snapshot snapshot,
|
|
|
+ Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
|
|
|
+ final List<INode> removedINodes) {
|
|
|
+ Quota.Counts counts = Quota.Counts.newInstance();
|
|
|
+ try {
|
|
|
+ if (inode.isReference()) {
|
|
|
+ INodeReference referenceNode = inode.asReference();
|
|
|
+ INodeReference.WithCount wc =
|
|
|
+ (WithCount) referenceNode.getReferredINode();
|
|
|
+ INode referred = wc.getReferredINode();
|
|
|
+ Quota.Counts subCounts = destroyDstSnapshot(referred, snapshot,
|
|
|
+ prior, collectedBlocks, removedINodes);
|
|
|
+ if (inode instanceof INodeReference.WithName) {
|
|
|
+ INodeReference ref = wc.getParentReference();
|
|
|
+ if (ref != null) {
|
|
|
+ ref.addSpaceConsumed(-subCounts.get(Quota.NAMESPACE),
|
|
|
+ -subCounts.get(Quota.DISKSPACE), true, Snapshot.INVALID_ID);
|
|
|
+ }
|
|
|
+ } else if (inode instanceof INodeReference.DstReference) {
|
|
|
+ wc.addSpaceConsumedToRenameSrc(-counts.get(Quota.NAMESPACE),
|
|
|
+ -counts.get(Quota.DISKSPACE), true, snapshot.getId());
|
|
|
+ }
|
|
|
+ } else if (inode.isFile()) { // file and not reference
|
|
|
+ counts.add(inode.cleanSubtree(snapshot, null, collectedBlocks,
|
|
|
+ removedINodes));
|
|
|
+ } else if (inode.isDirectory()) { // directory and not reference
|
|
|
+ if (inode.asDirectory() instanceof INodeDirectoryWithSnapshot) {
|
|
|
+ INodeDirectoryWithSnapshot dirNode =
|
|
|
+ (INodeDirectoryWithSnapshot) inode.asDirectory();
|
|
|
+ DirectoryDiffList diffList = dirNode.getDiffs();
|
|
|
+ prior = diffList.updatePrior(snapshot, prior);
|
|
|
+ counts.add(diffList.deleteSnapshotDiff(snapshot, prior, dirNode,
|
|
|
+ collectedBlocks, removedINodes));
|
|
|
+ if (prior != null) {
|
|
|
+ DirectoryDiff priorDiff = diffList.getDiff(prior);
|
|
|
+ if (priorDiff != null) {
|
|
|
+ // destroy everything in the created list!
|
|
|
+ counts.add(priorDiff.diff.destroyCreatedList(dirNode,
|
|
|
+ collectedBlocks, removedINodes));
|
|
|
+ for (INode dNode : priorDiff.getChildrenDiff().getList(
|
|
|
+ ListType.DELETED)) {
|
|
|
+ counts.add(cleanDeletedINode(dNode, snapshot, prior,
|
|
|
+ collectedBlocks, removedINodes));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ Snapshot s = snapshot != null && prior != null ? prior : snapshot;
|
|
|
+ for (INode child : inode.asDirectory().getChildrenList(s)) {
|
|
|
+ counts.add(destroyDstSnapshot(child, s, prior, collectedBlocks,
|
|
|
+ removedINodes));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } catch (QuotaExceededException e) {
|
|
|
+ String error = "should not have QuotaExceededException while deleting snapshot";
|
|
|
+ LOG.error(error, e);
|
|
|
+ }
|
|
|
+ return counts;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* @return The children list of a directory in a snapshot.
|
|
@@ -395,22 +478,19 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
for (INodeReference ref : refNodes) {
|
|
|
// if the node is a reference node, we should continue the
|
|
|
// snapshot deletion process
|
|
|
- try {
|
|
|
- // Use null as prior snapshot. We are handling a reference node stored
|
|
|
- // in the delete list of this snapshot diff. We need to destroy this
|
|
|
- // snapshot diff because it is the very first one in history.
|
|
|
- // If the ref node is a WithName instance acting as the src node of
|
|
|
- // the rename operation, there will not be any snapshot before the
|
|
|
- // snapshot to be deleted. If the ref node presents the dst node of a
|
|
|
- // rename operation, we can identify the corresponding prior snapshot
|
|
|
- // when we come into the subtree of the ref node.
|
|
|
- counts.add(ref.cleanSubtree(this.snapshot, null, collectedBlocks,
|
|
|
- removedINodes));
|
|
|
- } catch (QuotaExceededException e) {
|
|
|
- String error =
|
|
|
- "should not have QuotaExceededException while deleting snapshot "
|
|
|
- + this.snapshot;
|
|
|
- LOG.error(error, e);
|
|
|
+ if (ref instanceof INodeReference.DstReference) {
|
|
|
+ // if ref is a DstReference instance, we should delete all the files
|
|
|
+ // created after the rename
|
|
|
+ destroyDstSnapshot(ref, snapshot, null, collectedBlocks,
|
|
|
+ removedINodes);
|
|
|
+ } else if (ref instanceof INodeReference.WithName) {
|
|
|
+ // ref should have been renamed again. We only need to delete
|
|
|
+ // the snapshot in its subtree.
|
|
|
+ try {
|
|
|
+ ref.cleanSubtree(snapshot, null, collectedBlocks, removedINodes);
|
|
|
+ } catch (QuotaExceededException e) {
|
|
|
+ LOG.error("Error: should not throw QuotaExceededException", e);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
return counts;
|
|
@@ -689,7 +769,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
if (added && !removeDeletedChild) {
|
|
|
final Quota.Counts counts = deletedChild.computeQuotaUsage();
|
|
|
addSpaceConsumed(counts.get(Quota.NAMESPACE),
|
|
|
- counts.get(Quota.DISKSPACE), false);
|
|
|
+ counts.get(Quota.DISKSPACE), false, Snapshot.INVALID_ID);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -796,16 +876,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
* @param collectedBlocks Used to collect blocks for later deletion.
|
|
|
* @return Quota usage update.
|
|
|
*/
|
|
|
- private Quota.Counts cleanDeletedINode(INode inode, Snapshot post,
|
|
|
+ private static Quota.Counts cleanDeletedINode(INode inode, Snapshot post,
|
|
|
Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
|
|
|
- final List<INode> removedINodes) {
|
|
|
+ final List<INode> removedINodes) throws QuotaExceededException {
|
|
|
Quota.Counts counts = Quota.Counts.newInstance();
|
|
|
Deque<INode> queue = new ArrayDeque<INode>();
|
|
|
queue.addLast(inode);
|
|
|
while (!queue.isEmpty()) {
|
|
|
INode topNode = queue.pollFirst();
|
|
|
- if (topNode instanceof FileWithSnapshot) {
|
|
|
- FileWithSnapshot fs = (FileWithSnapshot) topNode;
|
|
|
+ if (topNode.isFile() && topNode.asFile() instanceof FileWithSnapshot) {
|
|
|
+ FileWithSnapshot fs = (FileWithSnapshot) topNode.asFile();
|
|
|
counts.add(fs.getDiffs().deleteSnapshotDiff(post, prior,
|
|
|
topNode.asFile(), collectedBlocks, removedINodes));
|
|
|
} else if (topNode.isDirectory()) {
|
|
@@ -840,13 +920,41 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
super.destroyAndCollectBlocks(collectedBlocks, removedINodes);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
|
|
|
+ boolean useCache, int lastSnapshotId) {
|
|
|
+ if ((useCache && isQuotaSet()) || lastSnapshotId == Snapshot.INVALID_ID) {
|
|
|
+ return super.computeQuotaUsage(counts, useCache, lastSnapshotId);
|
|
|
+ }
|
|
|
+
|
|
|
+ final int diffNum = 0;
|
|
|
+ Snapshot lastSnapshot = null;
|
|
|
+ Snapshot lastInDiff = diffs.getLastSnapshot();
|
|
|
+ // if lastSnapshotId > lastInDiff.getId(), the snapshot diff associated with
|
|
|
+ // lastSnapshotId must have been deleted. We should call
|
|
|
+ // getChildrenList(null) to get the children list for the continuous
|
|
|
+ // computation. In the meanwhile, there must be some snapshot diff whose
|
|
|
+ // snapshot id is no less than lastSnapshotId. Otherwise the WithName node
|
|
|
+ // itself should have been deleted.
|
|
|
+ if (lastInDiff != null && lastInDiff.getId() >= lastSnapshotId) {
|
|
|
+ lastSnapshot = diffs.searchSnapshotById(lastSnapshotId);
|
|
|
+ }
|
|
|
+
|
|
|
+ ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
|
|
|
+ for (INode child : childrenList) {
|
|
|
+ child.computeQuotaUsage(counts, useCache, lastSnapshotId);
|
|
|
+ }
|
|
|
+
|
|
|
+ counts.add(Quota.NAMESPACE, diffNum + 1);
|
|
|
+ return counts;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
|
|
|
super.computeQuotaUsage4CurrentDirectory(counts);
|
|
|
-
|
|
|
for(DirectoryDiff d : diffs) {
|
|
|
for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
|
|
|
- deleted.computeQuotaUsage(counts, false);
|
|
|
+ deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
|
|
|
}
|
|
|
}
|
|
|
counts.add(Quota.NAMESPACE, diffs.asList().size());
|