|
@@ -17,7 +17,6 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
|
package org.apache.hadoop.hdfs.server.namenode.snapshot;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
|
import java.io.PrintWriter;
|
|
import java.io.PrintWriter;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
@@ -33,74 +32,50 @@ import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
|
import org.apache.hadoop.hdfs.server.namenode.Content;
|
|
import org.apache.hadoop.hdfs.server.namenode.Content;
|
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
|
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.INodeDirectory.SnapshotAndINode;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
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.INodeReference;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
|
|
import org.apache.hadoop.hdfs.server.namenode.Quota;
|
|
import org.apache.hadoop.hdfs.server.namenode.Quota;
|
|
-import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.ChildrenDiff;
|
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
|
|
|
|
import org.apache.hadoop.hdfs.util.Diff.ListType;
|
|
import org.apache.hadoop.hdfs.util.Diff.ListType;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.collect.Lists;
|
|
import com.google.common.collect.Lists;
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Directories where taking snapshots is allowed.
|
|
|
|
- *
|
|
|
|
- * Like other {@link INode} subclasses, this class is synchronized externally
|
|
|
|
- * by the namesystem and FSDirectory locks.
|
|
|
|
|
|
+ * A directory with this feature is a snapshottable directory, where snapshots
|
|
|
|
+ * can be taken. This feature extends {@link DirectoryWithSnapshotFeature}, and
|
|
|
|
+ * maintains extra information about all the snapshots taken on this directory.
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
-public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
|
|
|
|
+public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature {
|
|
/** Limit the number of snapshot per snapshottable directory. */
|
|
/** Limit the number of snapshot per snapshottable directory. */
|
|
static final int SNAPSHOT_LIMIT = 1 << 16;
|
|
static final int SNAPSHOT_LIMIT = 1 << 16;
|
|
|
|
|
|
- /** Cast INode to INodeDirectorySnapshottable. */
|
|
|
|
- static public INodeDirectorySnapshottable valueOf(
|
|
|
|
- INode inode, String src) throws IOException {
|
|
|
|
- final INodeDirectory dir = INodeDirectory.valueOf(inode, src);
|
|
|
|
- if (!dir.isSnapshottable()) {
|
|
|
|
- throw new SnapshotException(
|
|
|
|
- "Directory is not a snapshottable directory: " + src);
|
|
|
|
- }
|
|
|
|
- return (INodeDirectorySnapshottable)dir;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Snapshots of this directory in ascending order of snapshot names.
|
|
* Snapshots of this directory in ascending order of snapshot names.
|
|
* Note that snapshots in ascending order of snapshot id are stored in
|
|
* Note that snapshots in ascending order of snapshot id are stored in
|
|
* {@link INodeDirectoryWithSnapshot}.diffs (a private field).
|
|
* {@link INodeDirectoryWithSnapshot}.diffs (a private field).
|
|
*/
|
|
*/
|
|
private final List<Snapshot> snapshotsByNames = new ArrayList<Snapshot>();
|
|
private final List<Snapshot> snapshotsByNames = new ArrayList<Snapshot>();
|
|
-
|
|
|
|
- /**
|
|
|
|
- * @return {@link #snapshotsByNames}
|
|
|
|
- */
|
|
|
|
- ReadOnlyList<Snapshot> getSnapshotsByNames() {
|
|
|
|
- return ReadOnlyList.Util.asReadOnlyList(this.snapshotsByNames);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/** Number of snapshots allowed. */
|
|
/** Number of snapshots allowed. */
|
|
private int snapshotQuota = SNAPSHOT_LIMIT;
|
|
private int snapshotQuota = SNAPSHOT_LIMIT;
|
|
|
|
|
|
- public INodeDirectorySnapshottable(INodeDirectory dir) {
|
|
|
|
- super(dir, true, dir.getFeatures());
|
|
|
|
- // add snapshot feature if the original directory does not have it
|
|
|
|
- if (!isWithSnapshot()) {
|
|
|
|
- addSnapshotFeature(null);
|
|
|
|
- }
|
|
|
|
|
|
+ public DirectorySnapshottableFeature(DirectoryWithSnapshotFeature feature) {
|
|
|
|
+ super(feature == null ? null : feature.getDiffs());
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/** @return the number of existing snapshots. */
|
|
/** @return the number of existing snapshots. */
|
|
public int getNumSnapshots() {
|
|
public int getNumSnapshots() {
|
|
return snapshotsByNames.size();
|
|
return snapshotsByNames.size();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private int searchSnapshot(byte[] snapshotName) {
|
|
private int searchSnapshot(byte[] snapshotName) {
|
|
return Collections.binarySearch(snapshotsByNames, snapshotName);
|
|
return Collections.binarySearch(snapshotsByNames, snapshotName);
|
|
}
|
|
}
|
|
@@ -110,7 +85,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
final int i = searchSnapshot(snapshotName);
|
|
final int i = searchSnapshot(snapshotName);
|
|
return i < 0? null: snapshotsByNames.get(i);
|
|
return i < 0? null: snapshotsByNames.get(i);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
public Snapshot getSnapshotById(int sid) {
|
|
public Snapshot getSnapshotById(int sid) {
|
|
for (Snapshot s : snapshotsByNames) {
|
|
for (Snapshot s : snapshotsByNames) {
|
|
if (s.getId() == sid) {
|
|
if (s.getId() == sid) {
|
|
@@ -119,12 +94,12 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
}
|
|
}
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/** @return {@link #snapshotsByNames} as a {@link ReadOnlyList} */
|
|
/** @return {@link #snapshotsByNames} as a {@link ReadOnlyList} */
|
|
public ReadOnlyList<Snapshot> getSnapshotList() {
|
|
public ReadOnlyList<Snapshot> getSnapshotList() {
|
|
return ReadOnlyList.Util.asReadOnlyList(snapshotsByNames);
|
|
return ReadOnlyList.Util.asReadOnlyList(snapshotsByNames);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Rename a snapshot
|
|
* Rename a snapshot
|
|
* @param path
|
|
* @param path
|
|
@@ -139,7 +114,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
* name does not exist or a snapshot with the new name already
|
|
* name does not exist or a snapshot with the new name already
|
|
* exists
|
|
* exists
|
|
*/
|
|
*/
|
|
- void renameSnapshot(String path, String oldName, String newName)
|
|
|
|
|
|
+ public void renameSnapshot(String path, String oldName, String newName)
|
|
throws SnapshotException {
|
|
throws SnapshotException {
|
|
if (newName.equals(oldName)) {
|
|
if (newName.equals(oldName)) {
|
|
return;
|
|
return;
|
|
@@ -180,22 +155,17 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
this.snapshotQuota = snapshotQuota;
|
|
this.snapshotQuota = snapshotQuota;
|
|
}
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
|
- public boolean isSnapshottable() {
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
- * Simply add a snapshot into the {@link #snapshotsByNames}. Used by FSImage
|
|
|
|
- * loading.
|
|
|
|
|
|
+ * Simply add a snapshot into the {@link #snapshotsByNames}. Used when loading
|
|
|
|
+ * fsimage.
|
|
*/
|
|
*/
|
|
void addSnapshot(Snapshot snapshot) {
|
|
void addSnapshot(Snapshot snapshot) {
|
|
this.snapshotsByNames.add(snapshot);
|
|
this.snapshotsByNames.add(snapshot);
|
|
}
|
|
}
|
|
|
|
|
|
/** Add a snapshot. */
|
|
/** Add a snapshot. */
|
|
- Snapshot addSnapshot(int id, String name) throws SnapshotException,
|
|
|
|
- QuotaExceededException {
|
|
|
|
|
|
+ public Snapshot addSnapshot(INodeDirectory snapshotRoot, int id, String name)
|
|
|
|
+ throws SnapshotException, QuotaExceededException {
|
|
//check snapshot quota
|
|
//check snapshot quota
|
|
final int n = getNumSnapshots();
|
|
final int n = getNumSnapshots();
|
|
if (n + 1 > snapshotQuota) {
|
|
if (n + 1 > snapshotQuota) {
|
|
@@ -203,7 +173,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
+ n + " snapshot(s) and the snapshot quota is "
|
|
+ n + " snapshot(s) and the snapshot quota is "
|
|
+ snapshotQuota);
|
|
+ snapshotQuota);
|
|
}
|
|
}
|
|
- final Snapshot s = new Snapshot(id, name, this);
|
|
|
|
|
|
+ final Snapshot s = new Snapshot(id, name, snapshotRoot);
|
|
final byte[] nameBytes = s.getRoot().getLocalNameBytes();
|
|
final byte[] nameBytes = s.getRoot().getLocalNameBytes();
|
|
final int i = searchSnapshot(nameBytes);
|
|
final int i = searchSnapshot(nameBytes);
|
|
if (i >= 0) {
|
|
if (i >= 0) {
|
|
@@ -211,60 +181,61 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
+ "snapshot with the same name \"" + Snapshot.getSnapshotName(s) + "\".");
|
|
+ "snapshot with the same name \"" + Snapshot.getSnapshotName(s) + "\".");
|
|
}
|
|
}
|
|
|
|
|
|
- final DirectoryDiff d = getDiffs().addDiff(id, this);
|
|
|
|
|
|
+ final DirectoryDiff d = getDiffs().addDiff(id, snapshotRoot);
|
|
d.setSnapshotRoot(s.getRoot());
|
|
d.setSnapshotRoot(s.getRoot());
|
|
snapshotsByNames.add(-i - 1, s);
|
|
snapshotsByNames.add(-i - 1, s);
|
|
|
|
|
|
- //set modification time
|
|
|
|
- updateModificationTime(Time.now(), Snapshot.CURRENT_STATE_ID);
|
|
|
|
- s.getRoot().setModificationTime(getModificationTime(),
|
|
|
|
- Snapshot.CURRENT_STATE_ID);
|
|
|
|
|
|
+ // set modification time
|
|
|
|
+ final long now = Time.now();
|
|
|
|
+ snapshotRoot.updateModificationTime(now, Snapshot.CURRENT_STATE_ID);
|
|
|
|
+ s.getRoot().setModificationTime(now, Snapshot.CURRENT_STATE_ID);
|
|
return s;
|
|
return s;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Remove the snapshot with the given name from {@link #snapshotsByNames},
|
|
* Remove the snapshot with the given name from {@link #snapshotsByNames},
|
|
* and delete all the corresponding DirectoryDiff.
|
|
* and delete all the corresponding DirectoryDiff.
|
|
- *
|
|
|
|
|
|
+ *
|
|
|
|
+ * @param snapshotRoot The directory where we take snapshots
|
|
* @param snapshotName The name of the snapshot to be removed
|
|
* @param snapshotName The name of the snapshot to be removed
|
|
* @param collectedBlocks Used to collect information to update blocksMap
|
|
* @param collectedBlocks Used to collect information to update blocksMap
|
|
- * @return The removed snapshot. Null if no snapshot with the given name
|
|
|
|
|
|
+ * @return The removed snapshot. Null if no snapshot with the given name
|
|
* exists.
|
|
* exists.
|
|
*/
|
|
*/
|
|
- Snapshot removeSnapshot(String snapshotName,
|
|
|
|
- BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes)
|
|
|
|
- throws SnapshotException {
|
|
|
|
|
|
+ public Snapshot removeSnapshot(INodeDirectory snapshotRoot,
|
|
|
|
+ String snapshotName, BlocksMapUpdateInfo collectedBlocks,
|
|
|
|
+ final List<INode> removedINodes) throws SnapshotException {
|
|
final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
|
|
final int i = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
|
|
if (i < 0) {
|
|
if (i < 0) {
|
|
throw new SnapshotException("Cannot delete snapshot " + snapshotName
|
|
throw new SnapshotException("Cannot delete snapshot " + snapshotName
|
|
- + " from path " + this.getFullPathName()
|
|
|
|
|
|
+ + " from path " + snapshotRoot.getFullPathName()
|
|
+ ": the snapshot does not exist.");
|
|
+ ": the snapshot does not exist.");
|
|
} else {
|
|
} else {
|
|
final Snapshot snapshot = snapshotsByNames.get(i);
|
|
final Snapshot snapshot = snapshotsByNames.get(i);
|
|
- int prior = Snapshot.findLatestSnapshot(this, snapshot.getId());
|
|
|
|
|
|
+ int prior = Snapshot.findLatestSnapshot(snapshotRoot, snapshot.getId());
|
|
try {
|
|
try {
|
|
- Quota.Counts counts = cleanSubtree(snapshot.getId(), prior,
|
|
|
|
- collectedBlocks, removedINodes, true);
|
|
|
|
- INodeDirectory parent = getParent();
|
|
|
|
|
|
+ Quota.Counts counts = snapshotRoot.cleanSubtree(snapshot.getId(),
|
|
|
|
+ prior, collectedBlocks, removedINodes, true);
|
|
|
|
+ INodeDirectory parent = snapshotRoot.getParent();
|
|
if (parent != null) {
|
|
if (parent != null) {
|
|
- // there will not be any WithName node corresponding to the deleted
|
|
|
|
|
|
+ // there will not be any WithName node corresponding to the deleted
|
|
// snapshot, thus only update the quota usage in the current tree
|
|
// snapshot, thus only update the quota usage in the current tree
|
|
parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
|
|
parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
|
|
-counts.get(Quota.DISKSPACE), true);
|
|
-counts.get(Quota.DISKSPACE), true);
|
|
}
|
|
}
|
|
} catch(QuotaExceededException e) {
|
|
} catch(QuotaExceededException e) {
|
|
- LOG.error("BUG: removeSnapshot increases namespace usage.", e);
|
|
|
|
|
|
+ INode.LOG.error("BUG: removeSnapshot increases namespace usage.", e);
|
|
}
|
|
}
|
|
// remove from snapshotsByNames after successfully cleaning the subtree
|
|
// remove from snapshotsByNames after successfully cleaning the subtree
|
|
snapshotsByNames.remove(i);
|
|
snapshotsByNames.remove(i);
|
|
return snapshot;
|
|
return snapshot;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- @Override
|
|
|
|
|
|
+
|
|
public ContentSummaryComputationContext computeContentSummary(
|
|
public ContentSummaryComputationContext computeContentSummary(
|
|
|
|
+ final INodeDirectory snapshotRoot,
|
|
final ContentSummaryComputationContext summary) {
|
|
final ContentSummaryComputationContext summary) {
|
|
- super.computeContentSummary(summary);
|
|
|
|
|
|
+ snapshotRoot.computeContentSummary(summary);
|
|
summary.getCounts().add(Content.SNAPSHOT, snapshotsByNames.size());
|
|
summary.getCounts().add(Content.SNAPSHOT, snapshotsByNames.size());
|
|
summary.getCounts().add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
|
|
summary.getCounts().add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
|
|
return summary;
|
|
return summary;
|
|
@@ -273,7 +244,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
/**
|
|
/**
|
|
* Compute the difference between two snapshots (or a snapshot and the current
|
|
* Compute the difference between two snapshots (or a snapshot and the current
|
|
* directory) of the directory.
|
|
* directory) of the directory.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param from The name of the start point of the comparison. Null indicating
|
|
* @param from The name of the start point of the comparison. Null indicating
|
|
* the current tree.
|
|
* the current tree.
|
|
* @param to The name of the end point. Null indicating the current tree.
|
|
* @param to The name of the end point. Null indicating the current tree.
|
|
@@ -282,52 +253,55 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
* point, or if endSnapshotName is not null but cannot be identified
|
|
* point, or if endSnapshotName is not null but cannot be identified
|
|
* as a previous snapshot.
|
|
* as a previous snapshot.
|
|
*/
|
|
*/
|
|
- SnapshotDiffInfo computeDiff(final String from, final String to)
|
|
|
|
- throws SnapshotException {
|
|
|
|
- Snapshot fromSnapshot = getSnapshotByName(from);
|
|
|
|
- Snapshot toSnapshot = getSnapshotByName(to);
|
|
|
|
|
|
+ SnapshotDiffInfo computeDiff(final INodeDirectory snapshotRoot,
|
|
|
|
+ final String from, final String to) throws SnapshotException {
|
|
|
|
+ Snapshot fromSnapshot = getSnapshotByName(snapshotRoot, from);
|
|
|
|
+ Snapshot toSnapshot = getSnapshotByName(snapshotRoot, to);
|
|
// if the start point is equal to the end point, return null
|
|
// if the start point is equal to the end point, return null
|
|
if (from.equals(to)) {
|
|
if (from.equals(to)) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
- SnapshotDiffInfo diffs = new SnapshotDiffInfo(this, fromSnapshot,
|
|
|
|
|
|
+ SnapshotDiffInfo diffs = new SnapshotDiffInfo(snapshotRoot, fromSnapshot,
|
|
toSnapshot);
|
|
toSnapshot);
|
|
- computeDiffRecursively(this, new ArrayList<byte[]>(), diffs);
|
|
|
|
|
|
+ computeDiffRecursively(snapshotRoot, snapshotRoot, new ArrayList<byte[]>(),
|
|
|
|
+ diffs);
|
|
return diffs;
|
|
return diffs;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Find the snapshot matching the given name.
|
|
* Find the snapshot matching the given name.
|
|
- *
|
|
|
|
|
|
+ *
|
|
|
|
+ * @param snapshotRoot The directory where snapshots were taken.
|
|
* @param snapshotName The name of the snapshot.
|
|
* @param snapshotName The name of the snapshot.
|
|
* @return The corresponding snapshot. Null if snapshotName is null or empty.
|
|
* @return The corresponding snapshot. Null if snapshotName is null or empty.
|
|
* @throws SnapshotException If snapshotName is not null or empty, but there
|
|
* @throws SnapshotException If snapshotName is not null or empty, but there
|
|
* is no snapshot matching the name.
|
|
* is no snapshot matching the name.
|
|
*/
|
|
*/
|
|
- private Snapshot getSnapshotByName(String snapshotName)
|
|
|
|
- throws SnapshotException {
|
|
|
|
|
|
+ private Snapshot getSnapshotByName(INodeDirectory snapshotRoot,
|
|
|
|
+ String snapshotName) throws SnapshotException {
|
|
Snapshot s = null;
|
|
Snapshot s = null;
|
|
if (snapshotName != null && !snapshotName.isEmpty()) {
|
|
if (snapshotName != null && !snapshotName.isEmpty()) {
|
|
final int index = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
|
|
final int index = searchSnapshot(DFSUtil.string2Bytes(snapshotName));
|
|
if (index < 0) {
|
|
if (index < 0) {
|
|
throw new SnapshotException("Cannot find the snapshot of directory "
|
|
throw new SnapshotException("Cannot find the snapshot of directory "
|
|
- + this.getFullPathName() + " with name " + snapshotName);
|
|
|
|
|
|
+ + snapshotRoot.getFullPathName() + " with name " + snapshotName);
|
|
}
|
|
}
|
|
s = snapshotsByNames.get(index);
|
|
s = snapshotsByNames.get(index);
|
|
}
|
|
}
|
|
return s;
|
|
return s;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Recursively compute the difference between snapshots under a given
|
|
* Recursively compute the difference between snapshots under a given
|
|
* directory/file.
|
|
* directory/file.
|
|
- * @param node The directory/file under which the diff is computed.
|
|
|
|
- * @param parentPath Relative path (corresponding to the snapshot root) of
|
|
|
|
|
|
+ * @param snapshotRoot The directory where snapshots were taken.
|
|
|
|
+ * @param node The directory/file under which the diff is computed.
|
|
|
|
+ * @param parentPath Relative path (corresponding to the snapshot root) of
|
|
* the node's parent.
|
|
* the node's parent.
|
|
* @param diffReport data structure used to store the diff.
|
|
* @param diffReport data structure used to store the diff.
|
|
*/
|
|
*/
|
|
- private void computeDiffRecursively(INode node, List<byte[]> parentPath,
|
|
|
|
- SnapshotDiffInfo diffReport) {
|
|
|
|
|
|
+ private void computeDiffRecursively(final INodeDirectory snapshotRoot,
|
|
|
|
+ INode node, List<byte[]> parentPath, SnapshotDiffInfo diffReport) {
|
|
final Snapshot earlierSnapshot = diffReport.isFromEarlier() ?
|
|
final Snapshot earlierSnapshot = diffReport.isFromEarlier() ?
|
|
diffReport.getFrom() : diffReport.getTo();
|
|
diffReport.getFrom() : diffReport.getTo();
|
|
final Snapshot laterSnapshot = diffReport.isFromEarlier() ?
|
|
final Snapshot laterSnapshot = diffReport.isFromEarlier() ?
|
|
@@ -350,9 +324,10 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
final byte[] name = child.getLocalNameBytes();
|
|
final byte[] name = child.getLocalNameBytes();
|
|
boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
|
|
boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
|
|
if (!toProcess && child instanceof INodeReference.WithName) {
|
|
if (!toProcess && child instanceof INodeReference.WithName) {
|
|
- byte[][] renameTargetPath = findRenameTargetPath((WithName) child,
|
|
|
|
- laterSnapshot == null ? Snapshot.CURRENT_STATE_ID :
|
|
|
|
- laterSnapshot.getId());
|
|
|
|
|
|
+ byte[][] renameTargetPath = findRenameTargetPath(
|
|
|
|
+ snapshotRoot, (WithName) child,
|
|
|
|
+ laterSnapshot == null ? Snapshot.CURRENT_STATE_ID :
|
|
|
|
+ laterSnapshot.getId());
|
|
if (renameTargetPath != null) {
|
|
if (renameTargetPath != null) {
|
|
toProcess = true;
|
|
toProcess = true;
|
|
diffReport.setRenameTarget(child.getId(), renameTargetPath);
|
|
diffReport.setRenameTarget(child.getId(), renameTargetPath);
|
|
@@ -360,7 +335,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
}
|
|
}
|
|
if (toProcess) {
|
|
if (toProcess) {
|
|
parentPath.add(name);
|
|
parentPath.add(name);
|
|
- computeDiffRecursively(child, parentPath, diffReport);
|
|
|
|
|
|
+ computeDiffRecursively(snapshotRoot, child, parentPath, diffReport);
|
|
parentPath.remove(parentPath.size() - 1);
|
|
parentPath.remove(parentPath.size() - 1);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -379,12 +354,12 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
* However, we should include it in our snapshot diff report as rename only
|
|
* However, we should include it in our snapshot diff report as rename only
|
|
* if the rename target is also under the same snapshottable directory.
|
|
* if the rename target is also under the same snapshottable directory.
|
|
*/
|
|
*/
|
|
- private byte[][] findRenameTargetPath(INodeReference.WithName wn,
|
|
|
|
- final int snapshotId) {
|
|
|
|
|
|
+ private byte[][] findRenameTargetPath(final INodeDirectory snapshotRoot,
|
|
|
|
+ INodeReference.WithName wn, final int snapshotId) {
|
|
INode inode = wn.getReferredINode();
|
|
INode inode = wn.getReferredINode();
|
|
final LinkedList<byte[]> ancestors = Lists.newLinkedList();
|
|
final LinkedList<byte[]> ancestors = Lists.newLinkedList();
|
|
while (inode != null) {
|
|
while (inode != null) {
|
|
- if (inode == this) {
|
|
|
|
|
|
+ if (inode == snapshotRoot) {
|
|
return ancestors.toArray(new byte[ancestors.size()][]);
|
|
return ancestors.toArray(new byte[ancestors.size()][]);
|
|
}
|
|
}
|
|
if (inode instanceof INodeReference.WithCount) {
|
|
if (inode instanceof INodeReference.WithCount) {
|
|
@@ -407,39 +382,20 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Replace itself with {@link INodeDirectoryWithSnapshot} or
|
|
|
|
- * {@link INodeDirectory} depending on the latest snapshot.
|
|
|
|
- */
|
|
|
|
- INodeDirectory replaceSelf(final int latestSnapshotId, final INodeMap inodeMap)
|
|
|
|
- throws QuotaExceededException {
|
|
|
|
- if (latestSnapshotId == Snapshot.CURRENT_STATE_ID) {
|
|
|
|
- Preconditions.checkState(getDirectoryWithSnapshotFeature()
|
|
|
|
- .getLastSnapshotId() == Snapshot.CURRENT_STATE_ID, "this=%s", this);
|
|
|
|
- }
|
|
|
|
- INodeDirectory dir = replaceSelf4INodeDirectory(inodeMap);
|
|
|
|
- if (latestSnapshotId != Snapshot.CURRENT_STATE_ID) {
|
|
|
|
- dir.recordModification(latestSnapshotId);
|
|
|
|
- }
|
|
|
|
- return dir;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
@Override
|
|
@Override
|
|
- public String toDetailString() {
|
|
|
|
- return super.toDetailString() + ", snapshotsByNames=" + snapshotsByNames;
|
|
|
|
|
|
+ public String toString() {
|
|
|
|
+ return "snapshotsByNames=" + snapshotsByNames;
|
|
}
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
|
- public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
|
|
|
|
- int snapshot) {
|
|
|
|
- super.dumpTreeRecursively(out, prefix, snapshot);
|
|
|
|
-
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ public void dumpTreeRecursively(INodeDirectory snapshotRoot, PrintWriter out,
|
|
|
|
+ StringBuilder prefix, int snapshot) {
|
|
if (snapshot == Snapshot.CURRENT_STATE_ID) {
|
|
if (snapshot == Snapshot.CURRENT_STATE_ID) {
|
|
out.println();
|
|
out.println();
|
|
out.print(prefix);
|
|
out.print(prefix);
|
|
|
|
|
|
out.print("Snapshot of ");
|
|
out.print("Snapshot of ");
|
|
- final String name = getLocalName();
|
|
|
|
|
|
+ final String name = snapshotRoot.getLocalName();
|
|
out.print(name.isEmpty()? "/": name);
|
|
out.print(name.isEmpty()? "/": name);
|
|
out.print(": quota=");
|
|
out.print(": quota=");
|
|
out.print(getSnapshotQuota());
|
|
out.print(getSnapshotQuota());
|
|
@@ -455,13 +411,14 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
out.print(", #snapshot=");
|
|
out.print(", #snapshot=");
|
|
out.println(n);
|
|
out.println(n);
|
|
|
|
|
|
- dumpTreeRecursively(out, prefix, new Iterable<SnapshotAndINode>() {
|
|
|
|
|
|
+ INodeDirectory.dumpTreeRecursively(out, prefix,
|
|
|
|
+ new Iterable<SnapshotAndINode>() {
|
|
@Override
|
|
@Override
|
|
public Iterator<SnapshotAndINode> iterator() {
|
|
public Iterator<SnapshotAndINode> iterator() {
|
|
return new Iterator<SnapshotAndINode>() {
|
|
return new Iterator<SnapshotAndINode>() {
|
|
final Iterator<DirectoryDiff> i = getDiffs().iterator();
|
|
final Iterator<DirectoryDiff> i = getDiffs().iterator();
|
|
private DirectoryDiff next = findNext();
|
|
private DirectoryDiff next = findNext();
|
|
-
|
|
|
|
|
|
+
|
|
private DirectoryDiff findNext() {
|
|
private DirectoryDiff findNext() {
|
|
for(; i.hasNext(); ) {
|
|
for(; i.hasNext(); ) {
|
|
final DirectoryDiff diff = i.next();
|
|
final DirectoryDiff diff = i.next();
|
|
@@ -476,7 +433,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
public boolean hasNext() {
|
|
public boolean hasNext() {
|
|
return next != null;
|
|
return next != null;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public SnapshotAndINode next() {
|
|
public SnapshotAndINode next() {
|
|
final SnapshotAndINode pair = new SnapshotAndINode(next
|
|
final SnapshotAndINode pair = new SnapshotAndINode(next
|
|
@@ -485,7 +442,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
next = findNext();
|
|
next = findNext();
|
|
return pair;
|
|
return pair;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void remove() {
|
|
public void remove() {
|
|
throw new UnsupportedOperationException();
|
|
throw new UnsupportedOperationException();
|
|
@@ -495,4 +452,4 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
});
|
|
});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-}
|
|
|
|
|
|
+}
|