|
@@ -32,6 +32,9 @@ import org.apache.hadoop.hdfs.server.namenode.INode;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff.Container;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.diff.Diff.UndoInfo;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
@@ -44,367 +47,18 @@ import com.google.common.base.Preconditions;
|
|
public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
/**
|
|
/**
|
|
* The difference between the current state and a previous snapshot
|
|
* The difference between the current state and a previous snapshot
|
|
- * of an INodeDirectory.
|
|
|
|
- *
|
|
|
|
- * <pre>
|
|
|
|
- * Two lists are maintained in the algorithm:
|
|
|
|
- * - c-list for newly created inodes
|
|
|
|
- * - d-list for the deleted inodes
|
|
|
|
- *
|
|
|
|
- * Denote the state of an inode by the following
|
|
|
|
- * (0, 0): neither in c-list nor d-list
|
|
|
|
- * (c, 0): in c-list but not in d-list
|
|
|
|
- * (0, d): in d-list but not in c-list
|
|
|
|
- * (c, d): in both c-list and d-list
|
|
|
|
- *
|
|
|
|
- * For each case below, ( , ) at the end shows the result state of the inode.
|
|
|
|
- *
|
|
|
|
- * Case 1. Suppose the inode i is NOT in the previous snapshot. (0, 0)
|
|
|
|
- * 1.1. create i in current: add it to c-list (c, 0)
|
|
|
|
- * 1.1.1. create i in current and then create: impossible
|
|
|
|
- * 1.1.2. create i in current and then delete: remove it from c-list (0, 0)
|
|
|
|
- * 1.1.3. create i in current and then modify: replace it in c-list (c', 0)
|
|
|
|
- *
|
|
|
|
- * 1.2. delete i from current: impossible
|
|
|
|
- *
|
|
|
|
- * 1.3. modify i in current: impossible
|
|
|
|
- *
|
|
|
|
- * Case 2. Suppose the inode i is ALREADY in the previous snapshot. (0, 0)
|
|
|
|
- * 2.1. create i in current: impossible
|
|
|
|
- *
|
|
|
|
- * 2.2. delete i from current: add it to d-list (0, d)
|
|
|
|
- * 2.2.1. delete i from current and then create: add it to c-list (c, d)
|
|
|
|
- * 2.2.2. delete i from current and then delete: impossible
|
|
|
|
- * 2.2.2. delete i from current and then modify: impossible
|
|
|
|
- *
|
|
|
|
- * 2.3. modify i in current: put it in both c-list and d-list (c, d)
|
|
|
|
- * 2.3.1. modify i in current and then create: impossible
|
|
|
|
- * 2.3.2. modify i in current and then delete: remove it from c-list (0, d)
|
|
|
|
- * 2.3.3. modify i in current and then modify: replace it in c-list (c', d)
|
|
|
|
- * </pre>
|
|
|
|
|
|
+ * of the children list of an INodeDirectory.
|
|
*/
|
|
*/
|
|
- public static class Diff {
|
|
|
|
- /**
|
|
|
|
- * Search the inode from the list.
|
|
|
|
- * @return -1 if the list is null; otherwise, return the insertion point
|
|
|
|
- * defined in {@link Collections#binarySearch(List, Object)}.
|
|
|
|
- * Note that, when the list is null, -1 is the correct insertion point.
|
|
|
|
- */
|
|
|
|
- static int search(final List<INode> inodes, final INode i) {
|
|
|
|
- return search(inodes, i.getLocalNameBytes());
|
|
|
|
- }
|
|
|
|
- private static int search(final List<INode> inodes, final byte[] name) {
|
|
|
|
- return inodes == null? -1: Collections.binarySearch(inodes, name);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static void remove(final List<INode> inodes, final int i,
|
|
|
|
- final INode expected) {
|
|
|
|
- final INode removed = inodes.remove(-i - 1);
|
|
|
|
- Preconditions.checkState(removed == expected,
|
|
|
|
- "removed != expected=%s, removed=%s.", expected, removed);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** c-list: inode(s) created in current. */
|
|
|
|
- private List<INode> created;
|
|
|
|
- /** d-list: inode(s) deleted from current. */
|
|
|
|
- private List<INode> deleted;
|
|
|
|
-
|
|
|
|
- INode searchCreated(final byte[] name) {
|
|
|
|
- int cIndex = search(created, name);
|
|
|
|
- return cIndex < 0 ? null : created.get(cIndex);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- INode searchDeleted(final byte[] name) {
|
|
|
|
- int dIndex = search(deleted, name);
|
|
|
|
- return dIndex < 0 ? null : deleted.get(dIndex);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Insert the inode to created.
|
|
|
|
- * @param i the insertion point defined
|
|
|
|
- * in {@link Collections#binarySearch(List, Object)}
|
|
|
|
- */
|
|
|
|
- private void insertCreated(final INode inode, final int i) {
|
|
|
|
- if (i >= 0) {
|
|
|
|
- throw new AssertionError("Inode already exists: inode=" + inode
|
|
|
|
- + ", created=" + created);
|
|
|
|
- }
|
|
|
|
- if (created == null) {
|
|
|
|
- created = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
|
|
|
|
- }
|
|
|
|
- created.add(-i - 1, inode);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Insert the inode to deleted.
|
|
|
|
- * @param i the insertion point defined
|
|
|
|
- * in {@link Collections#binarySearch(List, Object)}
|
|
|
|
- */
|
|
|
|
- private void insertDeleted(final INode inode, final int i) {
|
|
|
|
- if (i >= 0) {
|
|
|
|
- throw new AssertionError("Inode already exists: inode=" + inode
|
|
|
|
- + ", deleted=" + deleted);
|
|
|
|
- }
|
|
|
|
- if (deleted == null) {
|
|
|
|
- deleted = new ArrayList<INode>(DEFAULT_FILES_PER_DIRECTORY);
|
|
|
|
- }
|
|
|
|
- deleted.add(-i - 1, inode);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Create an inode in current state.
|
|
|
|
- * @return the c-list insertion point for undo.
|
|
|
|
- */
|
|
|
|
- int create(final INode inode) {
|
|
|
|
- final int c = search(created, inode);
|
|
|
|
- insertCreated(inode, c);
|
|
|
|
- return c;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void undoCreate(final INode inode, final int insertionPoint) {
|
|
|
|
- remove(created, insertionPoint, inode);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Delete an inode from current state.
|
|
|
|
- * @return a triple for undo.
|
|
|
|
- */
|
|
|
|
- Triple<Integer, INode, Integer> delete(final INode inode) {
|
|
|
|
- final int c = search(created, inode);
|
|
|
|
- INode previous = null;
|
|
|
|
- Integer d = null;
|
|
|
|
- if (c >= 0) {
|
|
|
|
- // remove a newly created inode
|
|
|
|
- previous = created.remove(c);
|
|
|
|
- } else {
|
|
|
|
- // not in c-list, it must be in previous
|
|
|
|
- d = search(deleted, inode);
|
|
|
|
- insertDeleted(inode, d);
|
|
|
|
- }
|
|
|
|
- return new Triple<Integer, INode, Integer>(c, previous, d);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void undoDelete(final INode inode,
|
|
|
|
- final Triple<Integer, INode, Integer> undoInfo) {
|
|
|
|
- final int c = undoInfo.left;
|
|
|
|
- if (c >= 0) {
|
|
|
|
- created.add(c, undoInfo.middle);
|
|
|
|
- } else {
|
|
|
|
- remove(deleted, undoInfo.right, inode);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Modify an inode in current state.
|
|
|
|
- * @return a triple for undo.
|
|
|
|
- */
|
|
|
|
- Triple<Integer, INode, Integer> modify(final INode oldinode,
|
|
|
|
- final INode newinode) {
|
|
|
|
- if (!oldinode.equals(newinode)) {
|
|
|
|
- throw new AssertionError("The names do not match: oldinode="
|
|
|
|
- + oldinode + ", newinode=" + newinode);
|
|
|
|
- }
|
|
|
|
- final int c = search(created, newinode);
|
|
|
|
- INode previous = null;
|
|
|
|
- Integer d = null;
|
|
|
|
- if (c >= 0) {
|
|
|
|
- // Case 1.1.3 and 2.3.3: inode is already in c-list,
|
|
|
|
- previous = created.set(c, newinode);
|
|
|
|
-
|
|
|
|
- //TODO: fix a bug that previous != oldinode. Set it to oldinode for now
|
|
|
|
- previous = oldinode;
|
|
|
|
- } else {
|
|
|
|
- d = search(deleted, oldinode);
|
|
|
|
- if (d < 0) {
|
|
|
|
- // Case 2.3: neither in c-list nor d-list
|
|
|
|
- insertCreated(newinode, c);
|
|
|
|
- insertDeleted(oldinode, d);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return new Triple<Integer, INode, Integer>(c, previous, d);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void undoModify(final INode oldinode, final INode newinode,
|
|
|
|
- final Triple<Integer, INode, Integer> undoInfo) {
|
|
|
|
- final int c = undoInfo.left;
|
|
|
|
- if (c >= 0) {
|
|
|
|
- created.set(c, undoInfo.middle);
|
|
|
|
- } else {
|
|
|
|
- final int d = undoInfo.right;
|
|
|
|
- if (d < 0) {
|
|
|
|
- remove(created, c, newinode);
|
|
|
|
- remove(deleted, d, oldinode);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Find an inode in the previous snapshot.
|
|
|
|
- * @return null if the inode cannot be determined in the previous snapshot
|
|
|
|
- * since no change is recorded and it should be determined in the
|
|
|
|
- * current snapshot; otherwise, return an array with size one
|
|
|
|
- * containing the inode in the previous snapshot. Note that the
|
|
|
|
- * inode can possibly be null which means that the inode is not
|
|
|
|
- * found in the previous snapshot.
|
|
|
|
- */
|
|
|
|
- INode[] accessPrevious(byte[] name) {
|
|
|
|
- return accessPrevious(name, created, deleted);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static INode[] accessPrevious(byte[] name,
|
|
|
|
- final List<INode> clist, final List<INode> dlist) {
|
|
|
|
- final int d = search(dlist, name);
|
|
|
|
- if (d >= 0) {
|
|
|
|
- // the inode was in previous and was once deleted in current.
|
|
|
|
- return new INode[]{dlist.get(d)};
|
|
|
|
- } else {
|
|
|
|
- final int c = search(clist, name);
|
|
|
|
- // When c >= 0, the inode in current is a newly created inode.
|
|
|
|
- return c >= 0? new INode[]{null}: null;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Find an inode in the current snapshot.
|
|
|
|
- * @return null if the inode cannot be determined in the current snapshot
|
|
|
|
- * since no change is recorded and it should be determined in the
|
|
|
|
- * previous snapshot; otherwise, return an array with size one
|
|
|
|
- * containing the inode in the current snapshot. Note that the
|
|
|
|
- * inode can possibly be null which means that the inode is not
|
|
|
|
- * found in the current snapshot.
|
|
|
|
- */
|
|
|
|
- INode[] accessCurrent(byte[] name) {
|
|
|
|
- return accessPrevious(name, deleted, created);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Apply this diff to previous snapshot in order to obtain current state.
|
|
|
|
- * @return the current state of the list.
|
|
|
|
- */
|
|
|
|
- List<INode> apply2Previous(final List<INode> previous) {
|
|
|
|
- return apply2Previous(previous, created, deleted);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private static List<INode> apply2Previous(final List<INode> previous,
|
|
|
|
- final List<INode> clist, final List<INode> dlist) {
|
|
|
|
- final List<INode> current = new ArrayList<INode>(previous);
|
|
|
|
- if (dlist != null) {
|
|
|
|
- for(INode d : dlist) {
|
|
|
|
- current.remove(d);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (clist != null) {
|
|
|
|
- for(INode c : clist) {
|
|
|
|
- final int i = search(current, c);
|
|
|
|
- current.add(-i - 1, c);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return current;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Apply the reverse of this diff to current state in order
|
|
|
|
- * to obtain the previous snapshot.
|
|
|
|
- * @return the previous state of the list.
|
|
|
|
- */
|
|
|
|
- List<INode> apply2Current(final List<INode> current) {
|
|
|
|
- return apply2Previous(current, deleted, created);
|
|
|
|
- }
|
|
|
|
|
|
+ public static class ChildrenDiff extends Diff<byte[], INode> {
|
|
|
|
+ private ChildrenDiff() {}
|
|
|
|
|
|
- /**
|
|
|
|
- * Combine the posterior diff with this diff. This function needs to called
|
|
|
|
- * before the posterior diff is to be deleted. In general we have:
|
|
|
|
- *
|
|
|
|
- * <pre>
|
|
|
|
- * 1. For (c, 0) in the posterior diff, check the inode in this diff:
|
|
|
|
- * 1.1 (c', 0) in this diff: impossible
|
|
|
|
- * 1.2 (0, d') in this diff: put in created --> (c, d')
|
|
|
|
- * 1.3 (c', d') in this diff: impossible
|
|
|
|
- * 1.4 (0, 0) in this diff: put in created --> (c, 0)
|
|
|
|
- * This is the same logic with {@link #create(INode)}.
|
|
|
|
- *
|
|
|
|
- * 2. For (0, d) in the posterior diff,
|
|
|
|
- * 2.1 (c', 0) in this diff: remove from old created --> (0, 0)
|
|
|
|
- * 2.2 (0, d') in this diff: impossible
|
|
|
|
- * 2.3 (c', d') in this diff: remove from old created --> (0, d')
|
|
|
|
- * 2.4 (0, 0) in this diff: put in deleted --> (0, d)
|
|
|
|
- * This is the same logic with {@link #delete(INode)}.
|
|
|
|
- *
|
|
|
|
- * 3. For (c, d) in the posterior diff,
|
|
|
|
- * 3.1 (c', 0) in this diff: replace old created --> (c, 0)
|
|
|
|
- * 3.2 (0, d') in this diff: impossible
|
|
|
|
- * 3.3 (c', d') in this diff: replace old created --> (c, d')
|
|
|
|
- * 3.4 (0, 0) in this diff: put in created and deleted --> (c, d)
|
|
|
|
- * This is the same logic with {@link #modify(INode, INode)}.
|
|
|
|
- * </pre>
|
|
|
|
- *
|
|
|
|
- * Note that after this function the postDiff will be deleted.
|
|
|
|
- *
|
|
|
|
- * @param the posterior diff to combine
|
|
|
|
- * @param deletedINodeProcesser Used in case 2.1, 2.3, 3.1, and 3.3
|
|
|
|
- * to process the deleted inodes.
|
|
|
|
- */
|
|
|
|
- void combinePostDiff(Diff postDiff, Processor deletedINodeProcesser) {
|
|
|
|
- final List<INode> postCreated = postDiff.created != null?
|
|
|
|
- postDiff.created: Collections.<INode>emptyList();
|
|
|
|
- final List<INode> postDeleted = postDiff.deleted != null?
|
|
|
|
- postDiff.deleted: Collections.<INode>emptyList();
|
|
|
|
- final Iterator<INode> createdIterator = postCreated.iterator();
|
|
|
|
- final Iterator<INode> deletedIterator = postDeleted.iterator();
|
|
|
|
-
|
|
|
|
- INode c = createdIterator.hasNext()? createdIterator.next(): null;
|
|
|
|
- INode d = deletedIterator.hasNext()? deletedIterator.next(): null;
|
|
|
|
-
|
|
|
|
- for(; c != null || d != null; ) {
|
|
|
|
- final int cmp = c == null? 1
|
|
|
|
- : d == null? -1
|
|
|
|
- : c.compareTo(d.getLocalNameBytes());
|
|
|
|
- if (cmp < 0) {
|
|
|
|
- // case 1: only in c-list
|
|
|
|
- create(c);
|
|
|
|
- c = createdIterator.hasNext()? createdIterator.next(): null;
|
|
|
|
- } else if (cmp > 0) {
|
|
|
|
- // case 2: only in d-list
|
|
|
|
- Triple<Integer, INode, Integer> triple = delete(d);
|
|
|
|
- if (deletedINodeProcesser != null) {
|
|
|
|
- deletedINodeProcesser.process(triple.middle);
|
|
|
|
- }
|
|
|
|
- d = deletedIterator.hasNext()? deletedIterator.next(): null;
|
|
|
|
- } else {
|
|
|
|
- // case 3: in both c-list and d-list
|
|
|
|
- final Triple<Integer, INode, Integer> triple = modify(d, c);
|
|
|
|
- if (deletedINodeProcesser != null) {
|
|
|
|
- deletedINodeProcesser.process(triple.middle);
|
|
|
|
- }
|
|
|
|
- c = createdIterator.hasNext()? createdIterator.next(): null;
|
|
|
|
- d = deletedIterator.hasNext()? deletedIterator.next(): null;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** Convert the inode list to a compact string. */
|
|
|
|
- static String toString(List<INode> inodes) {
|
|
|
|
- if (inodes == null || inodes.isEmpty()) {
|
|
|
|
- return "<empty>";
|
|
|
|
- }
|
|
|
|
- final StringBuilder b = new StringBuilder("[")
|
|
|
|
- .append(inodes.get(0));
|
|
|
|
- for(int i = 1; i < inodes.size(); i++) {
|
|
|
|
- b.append(", ").append(inodes.get(i));
|
|
|
|
- }
|
|
|
|
- return b.append("]").toString();
|
|
|
|
|
|
+ private ChildrenDiff(final List<INode> created, final List<INode> deleted) {
|
|
|
|
+ super(created, deleted);
|
|
}
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
|
- public String toString() {
|
|
|
|
- return getClass().getSimpleName()
|
|
|
|
- + "{created=" + toString(created)
|
|
|
|
- + ", deleted=" + toString(deleted) + "}";
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/** Serialize {@link #created} */
|
|
/** Serialize {@link #created} */
|
|
private void writeCreated(DataOutput out) throws IOException {
|
|
private void writeCreated(DataOutput out) throws IOException {
|
|
- if (created != null) {
|
|
|
|
|
|
+ final List<INode> created = getCreatedList();
|
|
out.writeInt(created.size());
|
|
out.writeInt(created.size());
|
|
for (INode node : created) {
|
|
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
|
|
@@ -412,23 +66,21 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
out.writeShort(name.length);
|
|
out.writeShort(name.length);
|
|
out.write(name);
|
|
out.write(name);
|
|
}
|
|
}
|
|
- } else {
|
|
|
|
- out.writeInt(0);
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/** Serialize {@link #deleted} */
|
|
/** Serialize {@link #deleted} */
|
|
private void writeDeleted(DataOutput out) throws IOException {
|
|
private void writeDeleted(DataOutput out) throws IOException {
|
|
- if (deleted != null) {
|
|
|
|
|
|
+ final List<INode> deleted = getDeletedList();
|
|
out.writeInt(deleted.size());
|
|
out.writeInt(deleted.size());
|
|
for (INode node : deleted) {
|
|
for (INode node : deleted) {
|
|
if (node.isDirectory()) {
|
|
if (node.isDirectory()) {
|
|
FSImageSerialization.writeINodeDirectory((INodeDirectory) node, out);
|
|
FSImageSerialization.writeINodeDirectory((INodeDirectory) node, out);
|
|
} else { // INodeFile
|
|
} else { // INodeFile
|
|
|
|
+ final List<INode> created = getCreatedList();
|
|
// we write the block information only for INodeFile node when the
|
|
// we write the block information only for INodeFile node when the
|
|
// node is only stored in the deleted list or the node is not a
|
|
// node is only stored in the deleted list or the node is not a
|
|
// snapshot copy
|
|
// snapshot copy
|
|
- int createdIndex = search(created, node);
|
|
|
|
|
|
+ int createdIndex = search(created, node.getKey());
|
|
if (createdIndex < 0) {
|
|
if (createdIndex < 0) {
|
|
FSImageSerialization.writeINodeFile((INodeFile) node, out, true);
|
|
FSImageSerialization.writeINodeFile((INodeFile) node, out, true);
|
|
} else {
|
|
} else {
|
|
@@ -446,9 +98,6 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- } else {
|
|
|
|
- out.writeInt(0);
|
|
|
|
- }
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/** Serialize to out */
|
|
/** Serialize to out */
|
|
@@ -460,11 +109,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
/** @return The list of INodeDirectory contained in the deleted list */
|
|
/** @return The list of INodeDirectory contained in the deleted list */
|
|
private List<INodeDirectory> getDirsInDeleted() {
|
|
private List<INodeDirectory> getDirsInDeleted() {
|
|
List<INodeDirectory> dirList = new ArrayList<INodeDirectory>();
|
|
List<INodeDirectory> dirList = new ArrayList<INodeDirectory>();
|
|
- if (deleted != null) {
|
|
|
|
- for (INode node : deleted) {
|
|
|
|
- if (node.isDirectory()) {
|
|
|
|
- dirList.add((INodeDirectory) node);
|
|
|
|
- }
|
|
|
|
|
|
+ for (INode node : getDeletedList()) {
|
|
|
|
+ if (node.isDirectory()) {
|
|
|
|
+ dirList.add((INodeDirectory) node);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return dirList;
|
|
return dirList;
|
|
@@ -502,7 +149,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
*/
|
|
*/
|
|
private SnapshotDiff posteriorDiff;
|
|
private SnapshotDiff posteriorDiff;
|
|
/** The children list diff. */
|
|
/** The children list diff. */
|
|
- private final Diff diff;
|
|
|
|
|
|
+ private final ChildrenDiff diff;
|
|
/** The snapshot inode data. It is null when there is no change. */
|
|
/** The snapshot inode data. It is null when there is no change. */
|
|
private INodeDirectory snapshotINode = null;
|
|
private INodeDirectory snapshotINode = null;
|
|
|
|
|
|
@@ -511,7 +158,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
|
|
|
this.snapshot = snapshot;
|
|
this.snapshot = snapshot;
|
|
this.childrenSize = dir.getChildrenList(null).size();
|
|
this.childrenSize = dir.getChildrenList(null).size();
|
|
- this.diff = new Diff();
|
|
|
|
|
|
+ this.diff = new ChildrenDiff();
|
|
}
|
|
}
|
|
|
|
|
|
/** Constructor used by FSImage loading */
|
|
/** Constructor used by FSImage loading */
|
|
@@ -523,12 +170,10 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
this.childrenSize = childrenSize;
|
|
this.childrenSize = childrenSize;
|
|
this.snapshotINode = snapshotINode;
|
|
this.snapshotINode = snapshotINode;
|
|
this.posteriorDiff = posteriorDiff;
|
|
this.posteriorDiff = posteriorDiff;
|
|
- this.diff = new Diff();
|
|
|
|
- diff.created = createdList;
|
|
|
|
- diff.deleted = deletedList;
|
|
|
|
|
|
+ this.diff = new ChildrenDiff(createdList, deletedList);
|
|
}
|
|
}
|
|
|
|
|
|
- Diff getDiff() {
|
|
|
|
|
|
+ ChildrenDiff getDiff() {
|
|
return diff;
|
|
return diff;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -581,9 +226,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
|
|
|
private List<INode> initChildren() {
|
|
private List<INode> initChildren() {
|
|
if (children == null) {
|
|
if (children == null) {
|
|
- final Diff combined = new Diff();
|
|
|
|
|
|
+ final ChildrenDiff combined = new ChildrenDiff();
|
|
for(SnapshotDiff d = SnapshotDiff.this; d != null; d = d.posteriorDiff) {
|
|
for(SnapshotDiff d = SnapshotDiff.this; d != null; d = d.posteriorDiff) {
|
|
- combined.combinePostDiff(d.diff, null);
|
|
|
|
|
|
+ combined.combinePosterior(d.diff, null);
|
|
}
|
|
}
|
|
children = combined.apply2Current(ReadOnlyList.Util.asList(
|
|
children = combined.apply2Current(ReadOnlyList.Util.asList(
|
|
INodeDirectoryWithSnapshot.this.getChildrenList(null)));
|
|
INodeDirectoryWithSnapshot.this.getChildrenList(null)));
|
|
@@ -616,10 +261,10 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
/** @return the child with the given name. */
|
|
/** @return the child with the given name. */
|
|
INode getChild(byte[] name, boolean checkPosterior) {
|
|
INode getChild(byte[] name, boolean checkPosterior) {
|
|
for(SnapshotDiff d = this; ; d = d.posteriorDiff) {
|
|
for(SnapshotDiff d = this; ; d = d.posteriorDiff) {
|
|
- final INode[] array = d.diff.accessPrevious(name);
|
|
|
|
- if (array != null) {
|
|
|
|
- // the diff is able to find it
|
|
|
|
- return array[0];
|
|
|
|
|
|
+ final Container<INode> returned = d.diff.accessPrevious(name);
|
|
|
|
+ if (returned != null) {
|
|
|
|
+ // the diff is able to determine the inode
|
|
|
|
+ return returned.getElement();
|
|
} else if (!checkPosterior) {
|
|
} 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;
|
|
return null;
|
|
@@ -667,12 +312,6 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
return diff.getDirsInDeleted();
|
|
return diff.getDirsInDeleted();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- /** An interface for passing a method to process inodes. */
|
|
|
|
- static interface Processor {
|
|
|
|
- /** Process the given inode. */
|
|
|
|
- void process(INode inode);
|
|
|
|
- }
|
|
|
|
|
|
|
|
/** Create an {@link INodeDirectoryWithSnapshot} with the given snapshot.*/
|
|
/** Create an {@link INodeDirectoryWithSnapshot} with the given snapshot.*/
|
|
public static INodeDirectoryWithSnapshot newInstance(INodeDirectory dir,
|
|
public static INodeDirectoryWithSnapshot newInstance(INodeDirectory dir,
|
|
@@ -718,7 +357,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
if (snapshotIndex > 0) {
|
|
if (snapshotIndex > 0) {
|
|
// combine the to-be-removed diff with its previous diff
|
|
// combine the to-be-removed diff with its previous diff
|
|
SnapshotDiff previousDiff = diffs.get(snapshotIndex - 1);
|
|
SnapshotDiff previousDiff = diffs.get(snapshotIndex - 1);
|
|
- previousDiff.diff.combinePostDiff(diffToRemove.diff, new Processor() {
|
|
|
|
|
|
+ previousDiff.diff.combinePosterior(diffToRemove.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) {
|
|
@@ -779,10 +419,10 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Check if the latest {@link Diff} exists. If not, add it.
|
|
|
|
- * @return the latest {@link Diff}, which is never null.
|
|
|
|
|
|
+ * Check if the latest {@link ChildrenDiff} exists. If not, add it.
|
|
|
|
+ * @return the latest {@link ChildrenDiff}, which is never null.
|
|
*/
|
|
*/
|
|
- Diff checkAndAddLatestDiff(Snapshot latest) {
|
|
|
|
|
|
+ ChildrenDiff checkAndAddLatestDiff(Snapshot latest) {
|
|
return checkAndAddLatestSnapshotDiff(latest).diff;
|
|
return checkAndAddLatestSnapshotDiff(latest).diff;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -845,8 +485,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
|
|
|
final Pair<? extends INode, ? extends INode> p = child.createSnapshotCopy();
|
|
final Pair<? extends INode, ? extends INode> p = child.createSnapshotCopy();
|
|
if (p.left != p.right) {
|
|
if (p.left != p.right) {
|
|
- final Triple<Integer, INode, Integer> triple = diff.diff.modify(p.right, p.left);
|
|
|
|
- if (triple.middle != null && p.left instanceof FileWithSnapshot) {
|
|
|
|
|
|
+ final UndoInfo<INode> undoIndo = diff.diff.modify(p.right, p.left);
|
|
|
|
+ if (undoIndo.getTrashedElement() != null && p.left instanceof FileWithSnapshot) {
|
|
// also should remove oldinode from the circular list
|
|
// also should remove oldinode from the circular list
|
|
FileWithSnapshot newNodeWithLink = (FileWithSnapshot) p.left;
|
|
FileWithSnapshot newNodeWithLink = (FileWithSnapshot) p.left;
|
|
FileWithSnapshot oldNodeWithLink = (FileWithSnapshot) p.right;
|
|
FileWithSnapshot oldNodeWithLink = (FileWithSnapshot) p.right;
|
|
@@ -859,7 +499,7 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public boolean addChild(INode inode, boolean setModTime, Snapshot latest) {
|
|
public boolean addChild(INode inode, boolean setModTime, Snapshot latest) {
|
|
- Diff diff = null;
|
|
|
|
|
|
+ ChildrenDiff diff = null;
|
|
Integer undoInfo = null;
|
|
Integer undoInfo = null;
|
|
if (latest != null) {
|
|
if (latest != null) {
|
|
diff = checkAndAddLatestDiff(latest);
|
|
diff = checkAndAddLatestDiff(latest);
|
|
@@ -874,8 +514,8 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public INode removeChild(INode child, Snapshot latest) {
|
|
public INode removeChild(INode child, Snapshot latest) {
|
|
- Diff diff = null;
|
|
|
|
- Triple<Integer, INode, Integer> undoInfo = null;
|
|
|
|
|
|
+ ChildrenDiff diff = null;
|
|
|
|
+ UndoInfo<INode> undoInfo = null;
|
|
if (latest != null) {
|
|
if (latest != null) {
|
|
diff = checkAndAddLatestDiff(latest);
|
|
diff = checkAndAddLatestDiff(latest);
|
|
undoInfo = diff.delete(child);
|
|
undoInfo = diff.delete(child);
|
|
@@ -887,9 +527,9 @@ public class INodeDirectoryWithSnapshot extends INodeDirectoryWithQuota {
|
|
diff.undoDelete(child, undoInfo);
|
|
diff.undoDelete(child, undoInfo);
|
|
} else {
|
|
} else {
|
|
//clean up the previously created file, if there is any.
|
|
//clean up the previously created file, if there is any.
|
|
- final INode previous = undoInfo.middle;
|
|
|
|
- if (previous != null && previous instanceof FileWithSnapshot) {
|
|
|
|
- ((FileWithSnapshot)previous).removeSelf();
|
|
|
|
|
|
+ final INode trashed = undoInfo.getTrashedElement();
|
|
|
|
+ if (trashed != null && trashed instanceof FileWithSnapshot) {
|
|
|
|
+ ((FileWithSnapshot)trashed).removeSelf();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|