|
@@ -24,6 +24,7 @@ import java.util.Collections;
|
|
import java.util.Comparator;
|
|
import java.util.Comparator;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
|
|
+import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.SortedMap;
|
|
import java.util.SortedMap;
|
|
@@ -43,6 +44,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.INodeFile;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeMap;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeMap;
|
|
|
|
+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.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.ChildrenDiff;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
|
|
@@ -51,6 +55,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
import com.google.common.primitives.SignedBytes;
|
|
import com.google.common.primitives.SignedBytes;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -98,7 +103,43 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
};
|
|
};
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ static class RenameEntry {
|
|
|
|
+ private byte[][] sourcePath;
|
|
|
|
+ private byte[][] targetPath;
|
|
|
|
+
|
|
|
|
+ void setSource(INode source, byte[][] sourceParentPath) {
|
|
|
|
+ Preconditions.checkState(sourcePath == null);
|
|
|
|
+ sourcePath = new byte[sourceParentPath.length + 1][];
|
|
|
|
+ System.arraycopy(sourceParentPath, 0, sourcePath, 0,
|
|
|
|
+ sourceParentPath.length);
|
|
|
|
+ sourcePath[sourcePath.length - 1] = source.getLocalNameBytes();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void setTarget(INode target, byte[][] targetParentPath) {
|
|
|
|
+ targetPath = new byte[targetParentPath.length + 1][];
|
|
|
|
+ System.arraycopy(targetParentPath, 0, targetPath, 0,
|
|
|
|
+ targetParentPath.length);
|
|
|
|
+ targetPath[targetPath.length - 1] = target.getLocalNameBytes();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void setTarget(byte[][] targetPath) {
|
|
|
|
+ this.targetPath = targetPath;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ boolean isRename() {
|
|
|
|
+ return sourcePath != null && targetPath != null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ byte[][] getSourcePath() {
|
|
|
|
+ return sourcePath;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ byte[][] getTargetPath() {
|
|
|
|
+ return targetPath;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/** The root directory of the snapshots */
|
|
/** The root directory of the snapshots */
|
|
private final INodeDirectorySnapshottable snapshotRoot;
|
|
private final INodeDirectorySnapshottable snapshotRoot;
|
|
/** The starting point of the difference */
|
|
/** The starting point of the difference */
|
|
@@ -109,7 +150,7 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
* A map recording modified INodeFile and INodeDirectory and their relative
|
|
* A map recording modified INodeFile and INodeDirectory and their relative
|
|
* path corresponding to the snapshot root. Sorted based on their names.
|
|
* path corresponding to the snapshot root. Sorted based on their names.
|
|
*/
|
|
*/
|
|
- private final SortedMap<INode, byte[][]> diffMap =
|
|
|
|
|
|
+ private final SortedMap<INode, byte[][]> diffMap =
|
|
new TreeMap<INode, byte[][]>(INODE_COMPARATOR);
|
|
new TreeMap<INode, byte[][]>(INODE_COMPARATOR);
|
|
/**
|
|
/**
|
|
* A map capturing the detailed difference about file creation/deletion.
|
|
* A map capturing the detailed difference about file creation/deletion.
|
|
@@ -119,7 +160,10 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
*/
|
|
*/
|
|
private final Map<INodeDirectory, ChildrenDiff> dirDiffMap =
|
|
private final Map<INodeDirectory, ChildrenDiff> dirDiffMap =
|
|
new HashMap<INodeDirectory, ChildrenDiff>();
|
|
new HashMap<INodeDirectory, ChildrenDiff>();
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ private final Map<Long, RenameEntry> renameMap =
|
|
|
|
+ new HashMap<Long, RenameEntry>();
|
|
|
|
+
|
|
SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
|
|
SnapshotDiffInfo(INodeDirectorySnapshottable snapshotRoot, Snapshot start,
|
|
Snapshot end) {
|
|
Snapshot end) {
|
|
this.snapshotRoot = snapshotRoot;
|
|
this.snapshotRoot = snapshotRoot;
|
|
@@ -132,8 +176,36 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
ChildrenDiff diff) {
|
|
ChildrenDiff diff) {
|
|
dirDiffMap.put(dir, diff);
|
|
dirDiffMap.put(dir, diff);
|
|
diffMap.put(dir, relativePath);
|
|
diffMap.put(dir, relativePath);
|
|
|
|
+ // detect rename
|
|
|
|
+ for (INode created : diff.getList(ListType.CREATED)) {
|
|
|
|
+ if (created.isReference()) {
|
|
|
|
+ RenameEntry entry = getEntry(created.getId());
|
|
|
|
+ if (entry.getTargetPath() == null) {
|
|
|
|
+ entry.setTarget(created, relativePath);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ for (INode deleted : diff.getList(ListType.DELETED)) {
|
|
|
|
+ if (deleted instanceof INodeReference.WithName) {
|
|
|
|
+ RenameEntry entry = getEntry(deleted.getId());
|
|
|
|
+ entry.setSource(deleted, relativePath);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ private RenameEntry getEntry(long inodeId) {
|
|
|
|
+ RenameEntry entry = renameMap.get(inodeId);
|
|
|
|
+ if (entry == null) {
|
|
|
|
+ entry = new RenameEntry();
|
|
|
|
+ renameMap.put(inodeId, entry);
|
|
|
|
+ }
|
|
|
|
+ return entry;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void setRenameTarget(long inodeId, byte[][] path) {
|
|
|
|
+ getEntry(inodeId).setTarget(path);
|
|
|
|
+ }
|
|
|
|
+
|
|
/** Add a modified file */
|
|
/** Add a modified file */
|
|
private void addFileDiff(INodeFile file, byte[][] relativePath) {
|
|
private void addFileDiff(INodeFile file, byte[][] relativePath) {
|
|
diffMap.put(file, relativePath);
|
|
diffMap.put(file, relativePath);
|
|
@@ -152,11 +224,11 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
List<DiffReportEntry> diffReportList = new ArrayList<DiffReportEntry>();
|
|
List<DiffReportEntry> diffReportList = new ArrayList<DiffReportEntry>();
|
|
for (INode node : diffMap.keySet()) {
|
|
for (INode node : diffMap.keySet()) {
|
|
diffReportList.add(new DiffReportEntry(DiffType.MODIFY, diffMap
|
|
diffReportList.add(new DiffReportEntry(DiffType.MODIFY, diffMap
|
|
- .get(node)));
|
|
|
|
|
|
+ .get(node), null));
|
|
if (node.isDirectory()) {
|
|
if (node.isDirectory()) {
|
|
ChildrenDiff dirDiff = dirDiffMap.get(node);
|
|
ChildrenDiff dirDiff = dirDiffMap.get(node);
|
|
List<DiffReportEntry> subList = dirDiff.generateReport(
|
|
List<DiffReportEntry> subList = dirDiff.generateReport(
|
|
- diffMap.get(node), isFromEarlier());
|
|
|
|
|
|
+ diffMap.get(node), isFromEarlier(), renameMap);
|
|
diffReportList.addAll(subList);
|
|
diffReportList.addAll(subList);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -423,25 +495,37 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
*/
|
|
*/
|
|
private void computeDiffRecursively(INode node, List<byte[]> parentPath,
|
|
private void computeDiffRecursively(INode node, List<byte[]> parentPath,
|
|
SnapshotDiffInfo diffReport) {
|
|
SnapshotDiffInfo diffReport) {
|
|
- ChildrenDiff diff = new ChildrenDiff();
|
|
|
|
|
|
+ final Snapshot earlierSnapshot = diffReport.isFromEarlier() ?
|
|
|
|
+ diffReport.from : diffReport.to;
|
|
|
|
+ final Snapshot laterSnapshot = diffReport.isFromEarlier() ?
|
|
|
|
+ diffReport.to : diffReport.from;
|
|
byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
|
|
byte[][] relativePath = parentPath.toArray(new byte[parentPath.size()][]);
|
|
if (node.isDirectory()) {
|
|
if (node.isDirectory()) {
|
|
|
|
+ final ChildrenDiff diff = new ChildrenDiff();
|
|
INodeDirectory dir = node.asDirectory();
|
|
INodeDirectory dir = node.asDirectory();
|
|
DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
|
|
DirectoryWithSnapshotFeature sf = dir.getDirectoryWithSnapshotFeature();
|
|
if (sf != null) {
|
|
if (sf != null) {
|
|
- boolean change = sf.computeDiffBetweenSnapshots(diffReport.from,
|
|
|
|
- diffReport.to, diff, dir);
|
|
|
|
|
|
+ boolean change = sf.computeDiffBetweenSnapshots(earlierSnapshot,
|
|
|
|
+ laterSnapshot, diff, dir);
|
|
if (change) {
|
|
if (change) {
|
|
diffReport.addDirDiff(dir, relativePath, diff);
|
|
diffReport.addDirDiff(dir, relativePath, diff);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- ReadOnlyList<INode> children = dir.getChildrenList(
|
|
|
|
- diffReport.isFromEarlier() ? Snapshot.getSnapshotId(diffReport.to) :
|
|
|
|
- Snapshot.getSnapshotId(diffReport.from));
|
|
|
|
|
|
+ ReadOnlyList<INode> children = dir.getChildrenList(earlierSnapshot
|
|
|
|
+ .getId());
|
|
for (INode child : children) {
|
|
for (INode child : children) {
|
|
final byte[] name = child.getLocalNameBytes();
|
|
final byte[] name = child.getLocalNameBytes();
|
|
- if (diff.searchIndex(ListType.CREATED, name) < 0
|
|
|
|
- && diff.searchIndex(ListType.DELETED, name) < 0) {
|
|
|
|
|
|
+ boolean toProcess = diff.searchIndex(ListType.DELETED, name) < 0;
|
|
|
|
+ if (!toProcess && child instanceof INodeReference.WithName) {
|
|
|
|
+ byte[][] renameTargetPath = findRenameTargetPath((WithName) child,
|
|
|
|
+ laterSnapshot == null ? Snapshot.CURRENT_STATE_ID :
|
|
|
|
+ laterSnapshot.getId());
|
|
|
|
+ if (renameTargetPath != null) {
|
|
|
|
+ toProcess = true;
|
|
|
|
+ diffReport.setRenameTarget(child.getId(), renameTargetPath);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ if (toProcess) {
|
|
parentPath.add(name);
|
|
parentPath.add(name);
|
|
computeDiffRecursively(child, parentPath, diffReport);
|
|
computeDiffRecursively(child, parentPath, diffReport);
|
|
parentPath.remove(parentPath.size() - 1);
|
|
parentPath.remove(parentPath.size() - 1);
|
|
@@ -449,18 +533,47 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
}
|
|
}
|
|
} else if (node.isFile() && node.asFile().isWithSnapshot()) {
|
|
} else if (node.isFile() && node.asFile().isWithSnapshot()) {
|
|
INodeFile file = node.asFile();
|
|
INodeFile file = node.asFile();
|
|
- Snapshot earlierSnapshot = diffReport.isFromEarlier() ? diffReport.from
|
|
|
|
- : diffReport.to;
|
|
|
|
- Snapshot laterSnapshot = diffReport.isFromEarlier() ? diffReport.to
|
|
|
|
- : diffReport.from;
|
|
|
|
- boolean change = file.getDiffs().changedBetweenSnapshots(earlierSnapshot,
|
|
|
|
- laterSnapshot);
|
|
|
|
|
|
+ boolean change = file.getFileWithSnapshotFeature()
|
|
|
|
+ .changedBetweenSnapshots(file, earlierSnapshot, laterSnapshot);
|
|
if (change) {
|
|
if (change) {
|
|
diffReport.addFileDiff(file, relativePath);
|
|
diffReport.addFileDiff(file, relativePath);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * We just found a deleted WithName node as the source of a rename operation.
|
|
|
|
+ * However, we should include it in our snapshot diff report as rename only
|
|
|
|
+ * if the rename target is also under the same snapshottable directory.
|
|
|
|
+ */
|
|
|
|
+ private byte[][] findRenameTargetPath(INodeReference.WithName wn,
|
|
|
|
+ final int snapshotId) {
|
|
|
|
+ INode inode = wn.getReferredINode();
|
|
|
|
+ final LinkedList<byte[]> ancestors = Lists.newLinkedList();
|
|
|
|
+ while (inode != null) {
|
|
|
|
+ if (inode == this) {
|
|
|
|
+ return ancestors.toArray(new byte[ancestors.size()][]);
|
|
|
|
+ }
|
|
|
|
+ if (inode instanceof INodeReference.WithCount) {
|
|
|
|
+ inode = ((WithCount) inode).getParentRef(snapshotId);
|
|
|
|
+ } else {
|
|
|
|
+ INode parent = inode.getParentReference() != null ? inode
|
|
|
|
+ .getParentReference() : inode.getParent();
|
|
|
|
+ if (parent != null && parent instanceof INodeDirectory) {
|
|
|
|
+ int sid = parent.asDirectory().searchChild(inode);
|
|
|
|
+ if (sid < snapshotId) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ if (!(parent instanceof WithCount)) {
|
|
|
|
+ ancestors.addFirst(inode.getLocalNameBytes());
|
|
|
|
+ }
|
|
|
|
+ inode = parent;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Replace itself with {@link INodeDirectoryWithSnapshot} or
|
|
* Replace itself with {@link INodeDirectoryWithSnapshot} or
|
|
* {@link INodeDirectory} depending on the latest snapshot.
|
|
* {@link INodeDirectory} depending on the latest snapshot.
|
|
@@ -549,4 +662,4 @@ public class INodeDirectorySnapshottable extends INodeDirectory {
|
|
});
|
|
});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-}
|
|
|
|
|
|
+}
|