|
@@ -28,6 +28,7 @@ import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
|
|
@@ -35,10 +36,10 @@ import org.apache.hadoop.hdfs.server.namenode.Content;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
|
|
|
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.INodeDirectoryAttributes;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.INodeMap;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeReference;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.Quota;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
|
|
@@ -51,18 +52,17 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
/**
|
|
|
- * The directory with snapshots. It maintains a list of snapshot diffs for
|
|
|
- * storing snapshot data. When there are modifications to the directory, the old
|
|
|
- * data is stored in the latest snapshot, if there is any.
|
|
|
+ * Feature for directory with snapshot-related information.
|
|
|
*/
|
|
|
-public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
+@InterfaceAudience.Private
|
|
|
+public class DirectoryWithSnapshotFeature implements INode.Feature {
|
|
|
/**
|
|
|
* The difference between the current state and a previous snapshot
|
|
|
* of the children list of an INodeDirectory.
|
|
|
*/
|
|
|
static class ChildrenDiff extends Diff<byte[], INode> {
|
|
|
ChildrenDiff() {}
|
|
|
-
|
|
|
+
|
|
|
private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
|
|
|
super(created, deleted);
|
|
|
}
|
|
@@ -73,7 +73,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
*/
|
|
|
private final boolean replace(final ListType type,
|
|
|
final INode oldChild, final INode newChild) {
|
|
|
- final List<INode> list = getList(type);
|
|
|
+ final List<INode> list = getList(type);
|
|
|
final int i = search(list, oldChild.getLocalNameBytes());
|
|
|
if (i < 0 || list.get(i).getId() != oldChild.getId()) {
|
|
|
return false;
|
|
@@ -93,10 +93,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
return false;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** clear the created list */
|
|
|
- private Quota.Counts destroyCreatedList(
|
|
|
- final INodeDirectoryWithSnapshot currentINode,
|
|
|
+ private Quota.Counts destroyCreatedList(final INodeDirectory currentINode,
|
|
|
final BlocksMapUpdateInfo collectedBlocks,
|
|
|
final List<INode> removedINodes) {
|
|
|
Quota.Counts counts = Quota.Counts.newInstance();
|
|
@@ -110,7 +109,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
createdList.clear();
|
|
|
return counts;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** clear the deleted list */
|
|
|
private Quota.Counts destroyDeletedList(
|
|
|
final BlocksMapUpdateInfo collectedBlocks,
|
|
@@ -124,19 +123,19 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
deletedList.clear();
|
|
|
return counts;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** Serialize {@link #created} */
|
|
|
private void writeCreated(DataOutput out) throws IOException {
|
|
|
final List<INode> created = getList(ListType.CREATED);
|
|
|
out.writeInt(created.size());
|
|
|
for (INode node : created) {
|
|
|
- // For INode in created list, we only need to record its local name
|
|
|
+ // For INode in created list, we only need to record its local name
|
|
|
byte[] name = node.getLocalNameBytes();
|
|
|
out.writeShort(name.length);
|
|
|
out.write(name);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** Serialize {@link #deleted} */
|
|
|
private void writeDeleted(DataOutput out,
|
|
|
ReferenceMap referenceMap) throws IOException {
|
|
@@ -146,12 +145,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
FSImageSerialization.saveINode2Image(node, out, true, referenceMap);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** Serialize to out */
|
|
|
private void write(DataOutput out, ReferenceMap referenceMap
|
|
|
) throws IOException {
|
|
|
writeCreated(out);
|
|
|
- writeDeleted(out, referenceMap);
|
|
|
+ writeDeleted(out, referenceMap);
|
|
|
}
|
|
|
|
|
|
/** Get the list of INodeDirectory contained in the deleted list */
|
|
@@ -162,17 +161,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Interpret the diff and generate a list of {@link DiffReportEntry}.
|
|
|
* @param parentPath The relative path of the parent.
|
|
|
- * @param parent The directory that the diff belongs to.
|
|
|
- * @param fromEarlier True indicates {@code diff=later-earlier},
|
|
|
+ * @param fromEarlier True indicates {@code diff=later-earlier},
|
|
|
* False indicates {@code diff=earlier-later}
|
|
|
* @return A list of {@link DiffReportEntry} as the diff report.
|
|
|
*/
|
|
|
public List<DiffReportEntry> generateReport(byte[][] parentPath,
|
|
|
- INodeDirectoryWithSnapshot parent, boolean fromEarlier) {
|
|
|
+ boolean fromEarlier) {
|
|
|
List<DiffReportEntry> cList = new ArrayList<DiffReportEntry>();
|
|
|
List<DiffReportEntry> dList = new ArrayList<DiffReportEntry>();
|
|
|
int c = 0, d = 0;
|
|
@@ -217,7 +215,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
return dList;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* The difference of an {@link INodeDirectory} between two snapshots.
|
|
|
*/
|
|
@@ -243,16 +241,16 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
this.childrenSize = childrenSize;
|
|
|
this.diff = new ChildrenDiff(createdList, deletedList);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
ChildrenDiff getChildrenDiff() {
|
|
|
return diff;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** Is the inode the root of the snapshot? */
|
|
|
boolean isSnapshotRoot() {
|
|
|
return snapshotINode == snapshot.getRoot();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
Quota.Counts combinePosteriorAndCollectBlocks(
|
|
|
final INodeDirectory currentDir, final DirectoryDiff posterior,
|
|
@@ -277,14 +275,15 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
* Since the snapshot is read-only, the logical view of the list is
|
|
|
* never changed although the internal data structure may mutate.
|
|
|
*/
|
|
|
- ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
|
|
|
+ private ReadOnlyList<INode> getChildrenList(final INodeDirectory currentDir) {
|
|
|
return new ReadOnlyList<INode>() {
|
|
|
private List<INode> children = null;
|
|
|
|
|
|
private List<INode> initChildren() {
|
|
|
if (children == null) {
|
|
|
final ChildrenDiff combined = new ChildrenDiff();
|
|
|
- for(DirectoryDiff d = DirectoryDiff.this; d != null; d = d.getPosterior()) {
|
|
|
+ for (DirectoryDiff d = DirectoryDiff.this; d != null;
|
|
|
+ d = d.getPosterior()) {
|
|
|
combined.combinePosterior(d.diff, null);
|
|
|
}
|
|
|
children = combined.apply2Current(ReadOnlyList.Util.asList(
|
|
@@ -297,17 +296,17 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
public Iterator<INode> iterator() {
|
|
|
return initChildren().iterator();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean isEmpty() {
|
|
|
return childrenSize == 0;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public int size() {
|
|
|
return childrenSize;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public INode get(int i) {
|
|
|
return initChildren().get(i);
|
|
@@ -322,9 +321,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
final Container<INode> returned = d.diff.accessPrevious(name);
|
|
|
if (returned != null) {
|
|
|
// the diff is able to determine the inode
|
|
|
- return returned.getElement();
|
|
|
+ return returned.getElement();
|
|
|
} else if (!checkPosterior) {
|
|
|
- // Since checkPosterior is false, return null, i.e. not found.
|
|
|
+ // Since checkPosterior is false, return null, i.e. not found.
|
|
|
return null;
|
|
|
} else if (d.getPosterior() == null) {
|
|
|
// no more posterior diff, get from current inode.
|
|
@@ -332,12 +331,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
return super.toString() + " childrenSize=" + childrenSize + ", " + diff;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
void write(DataOutput out, ReferenceMap referenceMap) throws IOException {
|
|
|
writeSnapshot(out);
|
|
@@ -386,7 +385,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
|
|
|
/** Replace the given child in the created/deleted list, if there is any. */
|
|
|
- private boolean replaceChild(final ListType type, final INode oldChild,
|
|
|
+ public boolean replaceChild(final ListType type, final INode oldChild,
|
|
|
final INode newChild) {
|
|
|
final List<DirectoryDiff> diffList = asList();
|
|
|
for(int i = diffList.size() - 1; i >= 0; i--) {
|
|
@@ -397,9 +396,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
return false;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** Remove the given child in the created/deleted list, if there is any. */
|
|
|
- private boolean removeChild(final ListType type, final INode child) {
|
|
|
+ public boolean removeChild(final ListType type, final INode child) {
|
|
|
final List<DirectoryDiff> diffList = asList();
|
|
|
for(int i = diffList.size() - 1; i >= 0; i--) {
|
|
|
final ChildrenDiff diff = diffList.get(i).diff;
|
|
@@ -410,84 +409,134 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Compute the difference between Snapshots.
|
|
|
- *
|
|
|
- * @param fromSnapshot Start point of the diff computation. Null indicates
|
|
|
- * current tree.
|
|
|
- * @param toSnapshot End point of the diff computation. Null indicates current
|
|
|
- * tree.
|
|
|
- * @param diff Used to capture the changes happening to the children. Note
|
|
|
- * that the diff still represents (later_snapshot - earlier_snapshot)
|
|
|
- * although toSnapshot can be before fromSnapshot.
|
|
|
- * @return Whether changes happened between the startSnapshot and endSnaphsot.
|
|
|
- */
|
|
|
- boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
|
|
|
- Snapshot toSnapshot, ChildrenDiff diff) {
|
|
|
- Snapshot earlier = fromSnapshot;
|
|
|
- Snapshot later = toSnapshot;
|
|
|
- if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
|
|
|
- earlier = toSnapshot;
|
|
|
- later = fromSnapshot;
|
|
|
+
|
|
|
+ private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
|
|
|
+ if (diffList == null || diffList.size() == 0) {
|
|
|
+ return null;
|
|
|
}
|
|
|
-
|
|
|
- boolean modified = diffs.changedBetweenSnapshots(earlier,
|
|
|
- later);
|
|
|
- if (!modified) {
|
|
|
- return false;
|
|
|
+ Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
|
|
|
+ for (INode node : diffList) {
|
|
|
+ map.put(node, node);
|
|
|
}
|
|
|
-
|
|
|
- final List<DirectoryDiff> difflist = diffs.asList();
|
|
|
- final int size = difflist.size();
|
|
|
- int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
|
|
|
- int laterDiffIndex = later == null ? size : Collections
|
|
|
- .binarySearch(difflist, later.getId());
|
|
|
- earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
|
|
|
- : earlierDiffIndex;
|
|
|
- laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
|
|
|
- : laterDiffIndex;
|
|
|
-
|
|
|
- boolean dirMetadataChanged = false;
|
|
|
- INodeDirectoryAttributes dirCopy = null;
|
|
|
- for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
|
|
|
- DirectoryDiff sdiff = difflist.get(i);
|
|
|
- diff.combinePosterior(sdiff.diff, null);
|
|
|
- if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
|
|
|
- if (dirCopy == null) {
|
|
|
- dirCopy = sdiff.snapshotINode;
|
|
|
- } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
|
|
|
- dirMetadataChanged = true;
|
|
|
+ return map;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Destroy a subtree under a DstReference node.
|
|
|
+ */
|
|
|
+ public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
|
|
|
+ final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
|
|
|
+ final List<INode> removedINodes) throws QuotaExceededException {
|
|
|
+ Preconditions.checkArgument(prior != null);
|
|
|
+ if (inode.isReference()) {
|
|
|
+ if (inode instanceof INodeReference.WithName && snapshot != null) {
|
|
|
+ // this inode has been renamed before the deletion of the DstReference
|
|
|
+ // subtree
|
|
|
+ inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
|
|
|
+ true);
|
|
|
+ } else {
|
|
|
+ // for DstReference node, continue this process to its subtree
|
|
|
+ destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
|
|
|
+ prior, collectedBlocks, removedINodes);
|
|
|
+ }
|
|
|
+ } else if (inode.isFile()) {
|
|
|
+ inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
|
|
|
+ } else if (inode.isDirectory()) {
|
|
|
+ Map<INode, INode> excludedNodes = null;
|
|
|
+ INodeDirectory dir = inode.asDirectory();
|
|
|
+ DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
|
|
|
+ if (sf != null) {
|
|
|
+ DirectoryDiffList diffList = sf.getDiffs();
|
|
|
+ DirectoryDiff priorDiff = diffList.getDiff(prior);
|
|
|
+ if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
|
|
|
+ List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
|
|
|
+ excludedNodes = cloneDiffList(dList);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (snapshot != null) {
|
|
|
+ diffList.deleteSnapshotDiff(snapshot, prior, dir, collectedBlocks,
|
|
|
+ removedINodes, true);
|
|
|
+ }
|
|
|
+ priorDiff = diffList.getDiff(prior);
|
|
|
+ if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
|
|
|
+ priorDiff.diff.destroyCreatedList(dir, collectedBlocks,
|
|
|
+ removedINodes);
|
|
|
}
|
|
|
}
|
|
|
+ for (INode child : inode.asDirectory().getChildrenList(prior)) {
|
|
|
+ if (excludedNodes != null && excludedNodes.containsKey(child)) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ destroyDstSubtree(child, snapshot, prior, collectedBlocks,
|
|
|
+ removedINodes);
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
- if (!diff.isEmpty() || dirMetadataChanged) {
|
|
|
- return true;
|
|
|
- } else if (dirCopy != null) {
|
|
|
- for (int i = laterDiffIndex; i < size; i++) {
|
|
|
- if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
|
|
|
- return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Clean an inode while we move it from the deleted list of post to the
|
|
|
+ * deleted list of prior.
|
|
|
+ * @param inode The inode to clean.
|
|
|
+ * @param post The post snapshot.
|
|
|
+ * @param prior The prior snapshot.
|
|
|
+ * @param collectedBlocks Used to collect blocks for later deletion.
|
|
|
+ * @return Quota usage update.
|
|
|
+ */
|
|
|
+ private static Quota.Counts cleanDeletedINode(INode inode,
|
|
|
+ final Snapshot post, final Snapshot prior,
|
|
|
+ final BlocksMapUpdateInfo collectedBlocks,
|
|
|
+ final List<INode> removedINodes, final boolean countDiffChange)
|
|
|
+ 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 INodeReference.WithName) {
|
|
|
+ INodeReference.WithName wn = (INodeReference.WithName) topNode;
|
|
|
+ if (wn.getLastSnapshotId() >= post.getId()) {
|
|
|
+ wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
|
|
|
+ countDiffChange);
|
|
|
+ }
|
|
|
+ // For DstReference node, since the node is not in the created list of
|
|
|
+ // prior, we should treat it as regular file/dir
|
|
|
+ } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
|
|
|
+ INodeFile file = topNode.asFile();
|
|
|
+ counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
|
|
|
+ collectedBlocks, removedINodes, countDiffChange));
|
|
|
+ } else if (topNode.isDirectory()) {
|
|
|
+ INodeDirectory dir = topNode.asDirectory();
|
|
|
+ ChildrenDiff priorChildrenDiff = null;
|
|
|
+ DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
|
|
|
+ if (sf != null) {
|
|
|
+ // delete files/dirs created after prior. Note that these
|
|
|
+ // files/dirs, along with inode, were deleted right after post.
|
|
|
+ DirectoryDiff priorDiff = sf.getDiffs().getDiff(prior);
|
|
|
+ if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
|
|
|
+ priorChildrenDiff = priorDiff.getChildrenDiff();
|
|
|
+ counts.add(priorChildrenDiff.destroyCreatedList(dir,
|
|
|
+ collectedBlocks, removedINodes));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ for (INode child : dir.getChildrenList(prior)) {
|
|
|
+ if (priorChildrenDiff != null
|
|
|
+ && priorChildrenDiff.search(ListType.DELETED,
|
|
|
+ child.getLocalNameBytes()) != null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ queue.addLast(child);
|
|
|
}
|
|
|
}
|
|
|
- return !dirCopy.metadataEquals(this);
|
|
|
- } else {
|
|
|
- return false;
|
|
|
}
|
|
|
+ return counts;
|
|
|
}
|
|
|
|
|
|
/** Diff list sorted by snapshot IDs, i.e. in chronological order. */
|
|
|
private final DirectoryDiffList diffs;
|
|
|
|
|
|
- public INodeDirectoryWithSnapshot(INodeDirectory that) {
|
|
|
- this(that, true, that instanceof INodeDirectoryWithSnapshot?
|
|
|
- ((INodeDirectoryWithSnapshot)that).getDiffs(): null);
|
|
|
- }
|
|
|
-
|
|
|
- INodeDirectoryWithSnapshot(INodeDirectory that, boolean adopt,
|
|
|
- DirectoryDiffList diffs) {
|
|
|
- super(that, adopt, true);
|
|
|
- this.diffs = diffs != null? diffs: new DirectoryDiffList();
|
|
|
+ public DirectoryWithSnapshotFeature(DirectoryDiffList diffs) {
|
|
|
+ this.diffs = diffs != null ? diffs : new DirectoryDiffList();
|
|
|
}
|
|
|
|
|
|
/** @return the last snapshot. */
|
|
@@ -499,204 +548,203 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
public DirectoryDiffList getDiffs() {
|
|
|
return diffs;
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public INodeDirectoryAttributes getSnapshotINode(Snapshot snapshot) {
|
|
|
- return diffs.getSnapshotINode(snapshot, this);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public INodeDirectoryWithSnapshot recordModification(final Snapshot latest,
|
|
|
- final INodeMap inodeMap) throws QuotaExceededException {
|
|
|
- if (isInLatestSnapshot(latest) && !shouldRecordInSrcSnapshot(latest)) {
|
|
|
- return saveSelf2Snapshot(latest, null);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get all the directories that are stored in some snapshot but not in the
|
|
|
+ * current children list. These directories are equivalent to the directories
|
|
|
+ * stored in the deletes lists.
|
|
|
+ */
|
|
|
+ public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
|
|
|
+ for (DirectoryDiff sdiff : diffs) {
|
|
|
+ sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
|
|
|
}
|
|
|
- return this;
|
|
|
- }
|
|
|
-
|
|
|
- /** Save the snapshot copy to the latest snapshot. */
|
|
|
- public INodeDirectoryWithSnapshot saveSelf2Snapshot(
|
|
|
- final Snapshot latest, final INodeDirectory snapshotCopy)
|
|
|
- throws QuotaExceededException {
|
|
|
- diffs.saveSelf2Snapshot(latest, this, snapshotCopy);
|
|
|
- return this;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public INode saveChild2Snapshot(final INode child, final Snapshot latest,
|
|
|
- final INode snapshotCopy, final INodeMap inodeMap)
|
|
|
- throws QuotaExceededException {
|
|
|
- Preconditions.checkArgument(!child.isDirectory(),
|
|
|
- "child is a directory, child=%s", child);
|
|
|
- if (latest == null) {
|
|
|
- return child;
|
|
|
- }
|
|
|
-
|
|
|
- final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, this);
|
|
|
- if (diff.getChild(child.getLocalNameBytes(), false, this) != null) {
|
|
|
- // it was already saved in the latest snapshot earlier.
|
|
|
- return child;
|
|
|
- }
|
|
|
-
|
|
|
- diff.diff.modify(snapshotCopy, child);
|
|
|
- return child;
|
|
|
- }
|
|
|
+ /**
|
|
|
+ * Add an inode into parent's children list. The caller of this method needs
|
|
|
+ * to make sure that parent is in the given snapshot "latest".
|
|
|
+ */
|
|
|
+ public boolean addChild(INodeDirectory parent, INode inode,
|
|
|
+ boolean setModTime, Snapshot latest) throws QuotaExceededException {
|
|
|
+ ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, parent).diff;
|
|
|
+ int undoInfo = diff.create(inode);
|
|
|
|
|
|
- @Override
|
|
|
- public boolean addChild(INode inode, boolean setModTime, Snapshot latest,
|
|
|
- final INodeMap inodeMap) throws QuotaExceededException {
|
|
|
- ChildrenDiff diff = null;
|
|
|
- Integer undoInfo = null;
|
|
|
- if (isInLatestSnapshot(latest)) {
|
|
|
- diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
|
|
|
- undoInfo = diff.create(inode);
|
|
|
- }
|
|
|
- final boolean added = super.addChild(inode, setModTime, null, inodeMap);
|
|
|
- if (!added && undoInfo != null) {
|
|
|
+ final boolean added = parent.addChild(inode, setModTime, null);
|
|
|
+ if (!added) {
|
|
|
diff.undoCreate(inode, undoInfo);
|
|
|
}
|
|
|
- return added;
|
|
|
+ return added;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public boolean removeChild(INode child, Snapshot latest,
|
|
|
- final INodeMap inodeMap) throws QuotaExceededException {
|
|
|
- ChildrenDiff diff = null;
|
|
|
- UndoInfo<INode> undoInfo = null;
|
|
|
+ /**
|
|
|
+ * Remove an inode from parent's children list. The caller of this method
|
|
|
+ * needs to make sure that parent is in the given snapshot "latest".
|
|
|
+ */
|
|
|
+ public boolean removeChild(INodeDirectory parent, INode child,
|
|
|
+ Snapshot latest) throws QuotaExceededException {
|
|
|
// For a directory that is not a renamed node, if isInLatestSnapshot returns
|
|
|
// false, the directory is not in the latest snapshot, thus we do not need
|
|
|
// to record the removed child in any snapshot.
|
|
|
// For a directory that was moved/renamed, note that if the directory is in
|
|
|
- // any of the previous snapshots, we will create a reference node for the
|
|
|
+ // any of the previous snapshots, we will create a reference node for the
|
|
|
// directory while rename, and isInLatestSnapshot will return true in that
|
|
|
// scenario (if all previous snapshots have been deleted, isInLatestSnapshot
|
|
|
- // still returns false). Thus if isInLatestSnapshot returns false, the
|
|
|
- // directory node cannot be in any snapshot (not in current tree, nor in
|
|
|
- // previous src tree). Thus we do not need to record the removed child in
|
|
|
+ // still returns false). Thus if isInLatestSnapshot returns false, the
|
|
|
+ // directory node cannot be in any snapshot (not in current tree, nor in
|
|
|
+ // previous src tree). Thus we do not need to record the removed child in
|
|
|
// any snapshot.
|
|
|
- if (isInLatestSnapshot(latest)) {
|
|
|
- diff = diffs.checkAndAddLatestSnapshotDiff(latest, this).diff;
|
|
|
- undoInfo = diff.delete(child);
|
|
|
- }
|
|
|
- final boolean removed = removeChild(child);
|
|
|
- if (undoInfo != null) {
|
|
|
- if (!removed) {
|
|
|
- //remove failed, undo
|
|
|
- diff.undoDelete(child, undoInfo);
|
|
|
- }
|
|
|
+ ChildrenDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest, parent).diff;
|
|
|
+ UndoInfo<INode> undoInfo = diff.delete(child);
|
|
|
+
|
|
|
+ final boolean removed = parent.removeChild(child);
|
|
|
+ if (!removed && undoInfo != null) {
|
|
|
+ // remove failed, undo
|
|
|
+ diff.undoDelete(child, undoInfo);
|
|
|
}
|
|
|
return removed;
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public void replaceChild(final INode oldChild, final INode newChild,
|
|
|
- final INodeMap inodeMap) {
|
|
|
- super.replaceChild(oldChild, newChild, inodeMap);
|
|
|
- if (oldChild.getParentReference() != null && !newChild.isReference()) {
|
|
|
- // oldChild is referred by a Reference node. Thus we are replacing the
|
|
|
- // referred inode, e.g.,
|
|
|
- // INodeFileWithSnapshot -> INodeFileUnderConstructionWithSnapshot
|
|
|
- // in this case, we do not need to update the diff list
|
|
|
- return;
|
|
|
- } else {
|
|
|
- diffs.replaceChild(ListType.CREATED, oldChild, newChild);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
- * This method is usually called by the undo section of rename.
|
|
|
- *
|
|
|
- * Before calling this function, in the rename operation, we replace the
|
|
|
- * original src node (of the rename operation) with a reference node (WithName
|
|
|
- * instance) in both the children list and a created list, delete the
|
|
|
- * reference node from the children list, and add it to the corresponding
|
|
|
- * deleted list.
|
|
|
- *
|
|
|
- * To undo the above operations, we have the following steps in particular:
|
|
|
- *
|
|
|
- * <pre>
|
|
|
- * 1) remove the WithName node from the deleted list (if it exists)
|
|
|
- * 2) replace the WithName node in the created list with srcChild
|
|
|
- * 3) add srcChild back as a child of srcParent. Note that we already add
|
|
|
- * the node into the created list of a snapshot diff in step 2, we do not need
|
|
|
- * to add srcChild to the created list of the latest snapshot.
|
|
|
- * </pre>
|
|
|
- *
|
|
|
- * We do not need to update quota usage because the old child is in the
|
|
|
- * deleted list before.
|
|
|
- *
|
|
|
- * @param oldChild
|
|
|
- * The reference node to be removed/replaced
|
|
|
- * @param newChild
|
|
|
- * The node to be added back
|
|
|
- * @param latestSnapshot
|
|
|
- * The latest snapshot. Note this may not be the last snapshot in the
|
|
|
- * {@link #diffs}, since the src tree of the current rename operation
|
|
|
- * may be the dst tree of a previous rename.
|
|
|
- * @throws QuotaExceededException should not throw this exception
|
|
|
+ * @return If there is no corresponding directory diff for the given
|
|
|
+ * snapshot, this means that the current children list should be
|
|
|
+ * returned for the snapshot. Otherwise we calculate the children list
|
|
|
+ * for the snapshot and return it.
|
|
|
*/
|
|
|
- public void undoRename4ScrParent(final INodeReference oldChild,
|
|
|
- final INode newChild, Snapshot latestSnapshot)
|
|
|
- throws QuotaExceededException {
|
|
|
- diffs.removeChild(ListType.DELETED, oldChild);
|
|
|
- diffs.replaceChild(ListType.CREATED, oldChild, newChild);
|
|
|
- // pass null for inodeMap since the parent node will not get replaced when
|
|
|
- // undoing rename
|
|
|
- addChild(newChild, true, null, null);
|
|
|
+ public ReadOnlyList<INode> getChildrenList(INodeDirectory currentINode,
|
|
|
+ final Snapshot snapshot) {
|
|
|
+ final DirectoryDiff diff = diffs.getDiff(snapshot);
|
|
|
+ return diff != null ? diff.getChildrenList(currentINode) : currentINode
|
|
|
+ .getChildrenList(null);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Undo the rename operation for the dst tree, i.e., if the rename operation
|
|
|
- * (with OVERWRITE option) removes a file/dir from the dst tree, add it back
|
|
|
- * and delete possible record in the deleted list.
|
|
|
- */
|
|
|
- public void undoRename4DstParent(final INode deletedChild,
|
|
|
- Snapshot latestSnapshot) throws QuotaExceededException {
|
|
|
- boolean removeDeletedChild = diffs.removeChild(ListType.DELETED,
|
|
|
- deletedChild);
|
|
|
- // pass null for inodeMap since the parent node will not get replaced when
|
|
|
- // undoing rename
|
|
|
- final boolean added = addChild(deletedChild, true, removeDeletedChild ? null
|
|
|
- : latestSnapshot, null);
|
|
|
- // update quota usage if adding is successfully and the old child has not
|
|
|
- // been stored in deleted list before
|
|
|
- if (added && !removeDeletedChild) {
|
|
|
- final Quota.Counts counts = deletedChild.computeQuotaUsage();
|
|
|
- addSpaceConsumed(counts.get(Quota.NAMESPACE),
|
|
|
- counts.get(Quota.DISKSPACE), false);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public ReadOnlyList<INode> getChildrenList(Snapshot snapshot) {
|
|
|
+ public INode getChild(INodeDirectory currentINode, byte[] name,
|
|
|
+ Snapshot snapshot) {
|
|
|
final DirectoryDiff diff = diffs.getDiff(snapshot);
|
|
|
- return diff != null? diff.getChildrenList(this): super.getChildrenList(null);
|
|
|
+ return diff != null ? diff.getChild(name, true, currentINode)
|
|
|
+ : currentINode.getChild(name, null);
|
|
|
}
|
|
|
+
|
|
|
+ /** Used to record the modification of a symlink node */
|
|
|
+ public INode saveChild2Snapshot(INodeDirectory currentINode,
|
|
|
+ final INode child, final Snapshot latest, final INode snapshotCopy)
|
|
|
+ throws QuotaExceededException {
|
|
|
+ Preconditions.checkArgument(!child.isDirectory(),
|
|
|
+ "child is a directory, child=%s", child);
|
|
|
+ Preconditions.checkArgument(latest != null);
|
|
|
+
|
|
|
+ final DirectoryDiff diff = diffs.checkAndAddLatestSnapshotDiff(latest,
|
|
|
+ currentINode);
|
|
|
+ if (diff.getChild(child.getLocalNameBytes(), false, currentINode) != null) {
|
|
|
+ // it was already saved in the latest snapshot earlier.
|
|
|
+ return child;
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public INode getChild(byte[] name, Snapshot snapshot) {
|
|
|
- final DirectoryDiff diff = diffs.getDiff(snapshot);
|
|
|
- return diff != null? diff.getChild(name, true, this): super.getChild(name, null);
|
|
|
+ diff.diff.modify(snapshotCopy, child);
|
|
|
+ return child;
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public String toDetailString() {
|
|
|
- return super.toDetailString() + ", " + diffs;
|
|
|
+
|
|
|
+ public void clear(INodeDirectory currentINode,
|
|
|
+ final BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
|
|
|
+ // destroy its diff list
|
|
|
+ for (DirectoryDiff diff : diffs) {
|
|
|
+ diff.destroyDiffAndCollectBlocks(currentINode, collectedBlocks,
|
|
|
+ removedINodes);
|
|
|
+ }
|
|
|
+ diffs.clear();
|
|
|
+ }
|
|
|
+
|
|
|
+ public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
|
|
|
+ for(DirectoryDiff d : diffs) {
|
|
|
+ for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
|
|
|
+ deleted.computeQuotaUsage(counts, false, Snapshot.INVALID_ID);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ counts.add(Quota.NAMESPACE, diffs.asList().size());
|
|
|
+ return counts;
|
|
|
+ }
|
|
|
+
|
|
|
+ public void computeContentSummary4Snapshot(final Content.Counts counts) {
|
|
|
+ // Create a new blank summary context for blocking processing of subtree.
|
|
|
+ ContentSummaryComputationContext summary =
|
|
|
+ new ContentSummaryComputationContext();
|
|
|
+ for(DirectoryDiff d : diffs) {
|
|
|
+ for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
|
|
|
+ deleted.computeContentSummary(summary);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // Add the counts from deleted trees.
|
|
|
+ counts.add(summary.getCounts());
|
|
|
+ // Add the deleted directory count.
|
|
|
+ counts.add(Content.DIRECTORY, diffs.asList().size());
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Get all the directories that are stored in some snapshot but not in the
|
|
|
- * current children list. These directories are equivalent to the directories
|
|
|
- * stored in the deletes lists.
|
|
|
+ * Compute the difference between Snapshots.
|
|
|
+ *
|
|
|
+ * @param fromSnapshot Start point of the diff computation. Null indicates
|
|
|
+ * current tree.
|
|
|
+ * @param toSnapshot End point of the diff computation. Null indicates current
|
|
|
+ * tree.
|
|
|
+ * @param diff Used to capture the changes happening to the children. Note
|
|
|
+ * that the diff still represents (later_snapshot - earlier_snapshot)
|
|
|
+ * although toSnapshot can be before fromSnapshot.
|
|
|
+ * @param currentINode The {@link INodeDirectory} this feature belongs to.
|
|
|
+ * @return Whether changes happened between the startSnapshot and endSnaphsot.
|
|
|
*/
|
|
|
- public void getSnapshotDirectory(List<INodeDirectory> snapshotDir) {
|
|
|
- for (DirectoryDiff sdiff : diffs) {
|
|
|
- sdiff.getChildrenDiff().getDirsInDeleted(snapshotDir);
|
|
|
+ boolean computeDiffBetweenSnapshots(Snapshot fromSnapshot,
|
|
|
+ Snapshot toSnapshot, ChildrenDiff diff, INodeDirectory currentINode) {
|
|
|
+ Snapshot earlier = fromSnapshot;
|
|
|
+ Snapshot later = toSnapshot;
|
|
|
+ if (Snapshot.ID_COMPARATOR.compare(fromSnapshot, toSnapshot) > 0) {
|
|
|
+ earlier = toSnapshot;
|
|
|
+ later = fromSnapshot;
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean modified = diffs.changedBetweenSnapshots(earlier, later);
|
|
|
+ if (!modified) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ final List<DirectoryDiff> difflist = diffs.asList();
|
|
|
+ final int size = difflist.size();
|
|
|
+ int earlierDiffIndex = Collections.binarySearch(difflist, earlier.getId());
|
|
|
+ int laterDiffIndex = later == null ? size : Collections
|
|
|
+ .binarySearch(difflist, later.getId());
|
|
|
+ earlierDiffIndex = earlierDiffIndex < 0 ? (-earlierDiffIndex - 1)
|
|
|
+ : earlierDiffIndex;
|
|
|
+ laterDiffIndex = laterDiffIndex < 0 ? (-laterDiffIndex - 1)
|
|
|
+ : laterDiffIndex;
|
|
|
+
|
|
|
+ boolean dirMetadataChanged = false;
|
|
|
+ INodeDirectoryAttributes dirCopy = null;
|
|
|
+ for (int i = earlierDiffIndex; i < laterDiffIndex; i++) {
|
|
|
+ DirectoryDiff sdiff = difflist.get(i);
|
|
|
+ diff.combinePosterior(sdiff.diff, null);
|
|
|
+ if (dirMetadataChanged == false && sdiff.snapshotINode != null) {
|
|
|
+ if (dirCopy == null) {
|
|
|
+ dirCopy = sdiff.snapshotINode;
|
|
|
+ } else if (!dirCopy.metadataEquals(sdiff.snapshotINode)) {
|
|
|
+ dirMetadataChanged = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!diff.isEmpty() || dirMetadataChanged) {
|
|
|
+ return true;
|
|
|
+ } else if (dirCopy != null) {
|
|
|
+ for (int i = laterDiffIndex; i < size; i++) {
|
|
|
+ if (!dirCopy.metadataEquals(difflist.get(i).snapshotINode)) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return !dirCopy.metadataEquals(currentINode);
|
|
|
+ } else {
|
|
|
+ return false;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public Quota.Counts cleanSubtree(final Snapshot snapshot, Snapshot prior,
|
|
|
+ public Quota.Counts cleanDirectory(final INodeDirectory currentINode,
|
|
|
+ final Snapshot snapshot, Snapshot prior,
|
|
|
final BlocksMapUpdateInfo collectedBlocks,
|
|
|
final List<INode> removedINodes, final boolean countDiffChange)
|
|
|
throws QuotaExceededException {
|
|
@@ -704,12 +752,12 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
Map<INode, INode> priorCreated = null;
|
|
|
Map<INode, INode> priorDeleted = null;
|
|
|
if (snapshot == null) { // delete the current directory
|
|
|
- recordModification(prior, null);
|
|
|
+ currentINode.recordModification(prior);
|
|
|
// delete everything in created list
|
|
|
DirectoryDiff lastDiff = diffs.getLast();
|
|
|
if (lastDiff != null) {
|
|
|
- counts.add(lastDiff.diff.destroyCreatedList(this, collectedBlocks,
|
|
|
- removedINodes));
|
|
|
+ counts.add(lastDiff.diff.destroyCreatedList(currentINode,
|
|
|
+ collectedBlocks, removedINodes));
|
|
|
}
|
|
|
} else {
|
|
|
// update prior
|
|
@@ -726,7 +774,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, this,
|
|
|
+ counts.add(getDiffs().deleteSnapshotDiff(snapshot, prior, currentINode,
|
|
|
collectedBlocks, removedINodes, countDiffChange));
|
|
|
|
|
|
// check priorDiff again since it may be created during the diff deletion
|
|
@@ -767,202 +815,13 @@ public class INodeDirectoryWithSnapshot extends INodeDirectory {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- counts.add(cleanSubtreeRecursively(snapshot, prior, collectedBlocks,
|
|
|
- removedINodes, priorDeleted, countDiffChange));
|
|
|
+ counts.add(currentINode.cleanSubtreeRecursively(snapshot, prior,
|
|
|
+ collectedBlocks, removedINodes, priorDeleted, countDiffChange));
|
|
|
|
|
|
- if (isQuotaSet()) {
|
|
|
- getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
|
|
|
+ if (currentINode.isQuotaSet()) {
|
|
|
+ currentINode.getDirectoryWithQuotaFeature().addSpaceConsumed2Cache(
|
|
|
-counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
|
|
|
}
|
|
|
return counts;
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Clean an inode while we move it from the deleted list of post to the
|
|
|
- * deleted list of prior.
|
|
|
- * @param inode The inode to clean.
|
|
|
- * @param post The post snapshot.
|
|
|
- * @param prior The prior snapshot.
|
|
|
- * @param collectedBlocks Used to collect blocks for later deletion.
|
|
|
- * @return Quota usage update.
|
|
|
- */
|
|
|
- private static Quota.Counts cleanDeletedINode(INode inode,
|
|
|
- final Snapshot post, final Snapshot prior,
|
|
|
- final BlocksMapUpdateInfo collectedBlocks,
|
|
|
- final List<INode> removedINodes, final boolean countDiffChange)
|
|
|
- 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 INodeReference.WithName) {
|
|
|
- INodeReference.WithName wn = (INodeReference.WithName) topNode;
|
|
|
- if (wn.getLastSnapshotId() >= post.getId()) {
|
|
|
- wn.cleanSubtree(post, prior, collectedBlocks, removedINodes,
|
|
|
- countDiffChange);
|
|
|
- }
|
|
|
- // For DstReference node, since the node is not in the created list of
|
|
|
- // prior, we should treat it as regular file/dir
|
|
|
- } else if (topNode.isFile() && topNode.asFile().isWithSnapshot()) {
|
|
|
- INodeFile file = topNode.asFile();
|
|
|
- counts.add(file.getDiffs().deleteSnapshotDiff(post, prior, file,
|
|
|
- collectedBlocks, removedINodes, countDiffChange));
|
|
|
- } else if (topNode.isDirectory()) {
|
|
|
- INodeDirectory dir = topNode.asDirectory();
|
|
|
- ChildrenDiff priorChildrenDiff = null;
|
|
|
- if (dir instanceof INodeDirectoryWithSnapshot) {
|
|
|
- // delete files/dirs created after prior. Note that these
|
|
|
- // files/dirs, along with inode, were deleted right after post.
|
|
|
- INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) dir;
|
|
|
- DirectoryDiff priorDiff = sdir.getDiffs().getDiff(prior);
|
|
|
- if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
|
|
|
- priorChildrenDiff = priorDiff.getChildrenDiff();
|
|
|
- counts.add(priorChildrenDiff.destroyCreatedList(sdir,
|
|
|
- collectedBlocks, removedINodes));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- for (INode child : dir.getChildrenList(prior)) {
|
|
|
- if (priorChildrenDiff != null
|
|
|
- && priorChildrenDiff.search(ListType.DELETED,
|
|
|
- child.getLocalNameBytes()) != null) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- queue.addLast(child);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return counts;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void destroyAndCollectBlocks(
|
|
|
- final BlocksMapUpdateInfo collectedBlocks,
|
|
|
- final List<INode> removedINodes) {
|
|
|
- // destroy its diff list
|
|
|
- for (DirectoryDiff diff : diffs) {
|
|
|
- diff.destroyDiffAndCollectBlocks(this, collectedBlocks, removedINodes);
|
|
|
- }
|
|
|
- diffs.clear();
|
|
|
- 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);
|
|
|
- }
|
|
|
-
|
|
|
- Snapshot lastSnapshot = diffs.getSnapshotById(lastSnapshotId);
|
|
|
-
|
|
|
- ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshot);
|
|
|
- for (INode child : childrenList) {
|
|
|
- child.computeQuotaUsage(counts, useCache, lastSnapshotId);
|
|
|
- }
|
|
|
-
|
|
|
- counts.add(Quota.NAMESPACE, 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, Snapshot.INVALID_ID);
|
|
|
- }
|
|
|
- }
|
|
|
- counts.add(Quota.NAMESPACE, diffs.asList().size());
|
|
|
- return counts;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public ContentSummaryComputationContext computeContentSummary(
|
|
|
- final ContentSummaryComputationContext summary) {
|
|
|
- // Snapshot summary calc won't be relinquishing locks in the middle.
|
|
|
- // Do this first and handover to parent.
|
|
|
- computeContentSummary4Snapshot(summary.getCounts());
|
|
|
- super.computeContentSummary(summary);
|
|
|
- return summary;
|
|
|
- }
|
|
|
-
|
|
|
- private void computeContentSummary4Snapshot(final Content.Counts counts) {
|
|
|
- // Create a new blank summary context for blocking processing of subtree.
|
|
|
- ContentSummaryComputationContext summary =
|
|
|
- new ContentSummaryComputationContext();
|
|
|
- for(DirectoryDiff d : diffs) {
|
|
|
- for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
|
|
|
- deleted.computeContentSummary(summary);
|
|
|
- }
|
|
|
- }
|
|
|
- // Add the counts from deleted trees.
|
|
|
- counts.add(summary.getCounts());
|
|
|
- // Add the deleted directory count.
|
|
|
- counts.add(Content.DIRECTORY, diffs.asList().size());
|
|
|
- }
|
|
|
-
|
|
|
- private static Map<INode, INode> cloneDiffList(List<INode> diffList) {
|
|
|
- if (diffList == null || diffList.size() == 0) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- Map<INode, INode> map = new HashMap<INode, INode>(diffList.size());
|
|
|
- for (INode node : diffList) {
|
|
|
- map.put(node, node);
|
|
|
- }
|
|
|
- return map;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Destroy a subtree under a DstReference node.
|
|
|
- */
|
|
|
- public static void destroyDstSubtree(INode inode, final Snapshot snapshot,
|
|
|
- final Snapshot prior, final BlocksMapUpdateInfo collectedBlocks,
|
|
|
- final List<INode> removedINodes) throws QuotaExceededException {
|
|
|
- Preconditions.checkArgument(prior != null);
|
|
|
- if (inode.isReference()) {
|
|
|
- if (inode instanceof INodeReference.WithName && snapshot != null) {
|
|
|
- // this inode has been renamed before the deletion of the DstReference
|
|
|
- // subtree
|
|
|
- inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes,
|
|
|
- true);
|
|
|
- } else {
|
|
|
- // for DstReference node, continue this process to its subtree
|
|
|
- destroyDstSubtree(inode.asReference().getReferredINode(), snapshot,
|
|
|
- prior, collectedBlocks, removedINodes);
|
|
|
- }
|
|
|
- } else if (inode.isFile()) {
|
|
|
- inode.cleanSubtree(snapshot, prior, collectedBlocks, removedINodes, true);
|
|
|
- } else if (inode.isDirectory()) {
|
|
|
- Map<INode, INode> excludedNodes = null;
|
|
|
- if (inode instanceof INodeDirectoryWithSnapshot) {
|
|
|
- INodeDirectoryWithSnapshot sdir = (INodeDirectoryWithSnapshot) inode;
|
|
|
-
|
|
|
- DirectoryDiffList diffList = sdir.getDiffs();
|
|
|
- DirectoryDiff priorDiff = diffList.getDiff(prior);
|
|
|
- if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
|
|
|
- List<INode> dList = priorDiff.diff.getList(ListType.DELETED);
|
|
|
- excludedNodes = cloneDiffList(dList);
|
|
|
- }
|
|
|
-
|
|
|
- if (snapshot != null) {
|
|
|
- diffList.deleteSnapshotDiff(snapshot, prior, sdir, collectedBlocks,
|
|
|
- removedINodes, true);
|
|
|
- }
|
|
|
- priorDiff = diffList.getDiff(prior);
|
|
|
- if (priorDiff != null && priorDiff.getSnapshot().equals(prior)) {
|
|
|
- priorDiff.diff.destroyCreatedList(sdir, collectedBlocks,
|
|
|
- removedINodes);
|
|
|
- }
|
|
|
- }
|
|
|
- for (INode child : inode.asDirectory().getChildrenList(prior)) {
|
|
|
- if (excludedNodes != null && excludedNodes.containsKey(child)) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- destroyDstSubtree(child, snapshot, prior, collectedBlocks,
|
|
|
- removedINodes);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
}
|