|
@@ -27,15 +27,13 @@ import java.io.FileInputStream;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
-import java.nio.ByteBuffer;
|
|
|
import java.security.DigestInputStream;
|
|
|
import java.security.DigestOutputStream;
|
|
|
import java.security.MessageDigest;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
-import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Map.Entry;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
@@ -46,7 +44,6 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.PathIsNotDirectoryException;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
-import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
@@ -423,9 +420,9 @@ public class FSImageFormat {
|
|
|
private void loadDirectoryWithSnapshot(DataInput in)
|
|
|
throws IOException {
|
|
|
// Step 1. Identify the parent INode
|
|
|
- String parentPath = FSImageSerialization.readString(in);
|
|
|
- final INodeDirectory parent = INodeDirectory.valueOf(
|
|
|
- namesystem.dir.rootDir.getNode(parentPath, false), parentPath);
|
|
|
+ long inodeId = in.readLong();
|
|
|
+ final INodeDirectory parent = this.namesystem.dir.getInode(inodeId)
|
|
|
+ .asDirectory();
|
|
|
|
|
|
// Check if the whole subtree has been saved (for reference nodes)
|
|
|
boolean toLoadSubtree = referenceMap.toProcessSubtree(parent.getId());
|
|
@@ -437,7 +434,7 @@ public class FSImageFormat {
|
|
|
int numSnapshots = in.readInt();
|
|
|
if (numSnapshots >= 0) {
|
|
|
final INodeDirectorySnapshottable snapshottableParent
|
|
|
- = INodeDirectorySnapshottable.valueOf(parent, parentPath);
|
|
|
+ = INodeDirectorySnapshottable.valueOf(parent, parent.getLocalName());
|
|
|
if (snapshottableParent.getParent() != null) { // not root
|
|
|
this.namesystem.getSnapshotManager().addSnapshottable(
|
|
|
snapshottableParent);
|
|
@@ -563,7 +560,9 @@ public class FSImageFormat {
|
|
|
final byte[] localName = FSImageSerialization.readLocalName(in);
|
|
|
INode inode = loadINode(localName, isSnapshotINode, in);
|
|
|
if (LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
|
|
|
- namesystem.dir.addToInodeMapUnprotected(inode);
|
|
|
+ if (!inode.isReference()) { // reference node does not have its id
|
|
|
+ namesystem.dir.addToInodeMapUnprotected(inode);
|
|
|
+ }
|
|
|
}
|
|
|
return inode;
|
|
|
}
|
|
@@ -789,8 +788,6 @@ public class FSImageFormat {
|
|
|
private MD5Hash savedDigest;
|
|
|
private final ReferenceMap referenceMap = new ReferenceMap();
|
|
|
|
|
|
- static private final byte[] PATH_SEPARATOR = DFSUtil.string2Bytes(Path.SEPARATOR);
|
|
|
-
|
|
|
/** @throws IllegalStateException if the instance has not yet saved an image */
|
|
|
private void checkSaved() {
|
|
|
if (!saved) {
|
|
@@ -852,18 +849,15 @@ public class FSImageFormat {
|
|
|
LOG.info("Saving image file " + newFile +
|
|
|
" using " + compression);
|
|
|
|
|
|
- byte[] byteStore = new byte[4*HdfsConstants.MAX_PATH_LENGTH];
|
|
|
- ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
|
|
|
// save the root
|
|
|
FSImageSerialization.saveINode2Image(fsDir.rootDir, out, false,
|
|
|
referenceMap);
|
|
|
// save the rest of the nodes
|
|
|
- saveImage(strbuf, fsDir.rootDir, out, null, true);
|
|
|
+ saveImage(fsDir.rootDir, out, true);
|
|
|
// save files under construction
|
|
|
sourceNamesystem.saveFilesUnderConstruction(out);
|
|
|
context.checkCancelled();
|
|
|
sourceNamesystem.saveSecretManagerState(out);
|
|
|
- strbuf = null;
|
|
|
context.checkCancelled();
|
|
|
out.flush();
|
|
|
context.checkCancelled();
|
|
@@ -905,40 +899,12 @@ public class FSImageFormat {
|
|
|
return dirNum;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * The nonSnapshotPath is a path without snapshot in order to enable buffer
|
|
|
- * reuse. If the snapshot is not null, we need to compute a snapshot path.
|
|
|
- * E.g., when nonSnapshotPath is "/test/foo/bar/" and the snapshot is s1 of
|
|
|
- * /test, we actually want to save image for directory /test/foo/bar/ under
|
|
|
- * snapshot s1 of /test, and the path to save thus should be
|
|
|
- * "/test/.snapshot/s1/foo/bar/".
|
|
|
- *
|
|
|
- * @param nonSnapshotPath The path without snapshot related information.
|
|
|
- * @param snapshot The snapshot associated with the inode that the path
|
|
|
- * actually leads to.
|
|
|
- * @return The snapshot path.
|
|
|
- */
|
|
|
- private static String computeSnapshotPath(String nonSnapshotPath,
|
|
|
- Snapshot snapshot) {
|
|
|
- String snapshotParentFullPath = snapshot.getRoot().getParent()
|
|
|
- .getFullPathName();
|
|
|
- String snapshotName = snapshot.getRoot().getLocalName();
|
|
|
- String relativePath = nonSnapshotPath.equals(snapshotParentFullPath) ?
|
|
|
- Path.SEPARATOR : nonSnapshotPath.substring(
|
|
|
- snapshotParentFullPath.length());
|
|
|
- return Snapshot.getSnapshotPath(snapshotParentFullPath,
|
|
|
- snapshotName + relativePath);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Save file tree image starting from the given root.
|
|
|
* This is a recursive procedure, which first saves all children and
|
|
|
* snapshot diffs of a current directory and then moves inside the
|
|
|
* sub-directories.
|
|
|
*
|
|
|
- * @param currentDirName A ByteBuffer storing the path leading to the
|
|
|
- * current node. For a snapshot node, the path is
|
|
|
- * (the snapshot path - ".snapshot/snapshot_name")
|
|
|
* @param current The current node
|
|
|
* @param out The DataoutputStream to write the image
|
|
|
* @param snapshot The possible snapshot associated with the current node
|
|
@@ -946,28 +912,10 @@ public class FSImageFormat {
|
|
|
* reference node, its subtree may already have been
|
|
|
* saved before.
|
|
|
*/
|
|
|
- private void saveImage(ByteBuffer currentDirName, INodeDirectory current,
|
|
|
- DataOutputStream out, Snapshot snapshot, boolean toSaveSubtree)
|
|
|
- throws IOException {
|
|
|
- // 1. Print prefix (parent directory name)
|
|
|
- int prefixLen = currentDirName.position();
|
|
|
- if (snapshot == null) {
|
|
|
- if (prefixLen == 0) { // root
|
|
|
- out.writeShort(PATH_SEPARATOR.length);
|
|
|
- out.write(PATH_SEPARATOR);
|
|
|
- } else { // non-root directories
|
|
|
- out.writeShort(prefixLen);
|
|
|
- out.write(currentDirName.array(), 0, prefixLen);
|
|
|
- }
|
|
|
- } else {
|
|
|
- String nonSnapshotPath = prefixLen == 0 ? Path.SEPARATOR : DFSUtil
|
|
|
- .bytes2String(currentDirName.array(), 0, prefixLen);
|
|
|
- String snapshotFullPath = computeSnapshotPath(nonSnapshotPath,
|
|
|
- snapshot);
|
|
|
- byte[] snapshotFullPathBytes = DFSUtil.string2Bytes(snapshotFullPath);
|
|
|
- out.writeShort(snapshotFullPathBytes.length);
|
|
|
- out.write(snapshotFullPathBytes);
|
|
|
- }
|
|
|
+ private void saveImage(INodeDirectory current, DataOutputStream out,
|
|
|
+ boolean toSaveSubtree) throws IOException {
|
|
|
+ // write the inode id of the directory
|
|
|
+ out.writeLong(current.getId());
|
|
|
|
|
|
if (!toSaveSubtree) {
|
|
|
return;
|
|
@@ -975,11 +923,12 @@ public class FSImageFormat {
|
|
|
|
|
|
final ReadOnlyList<INode> children = current.getChildrenList(null);
|
|
|
int dirNum = 0;
|
|
|
- Map<Snapshot, List<INodeDirectory>> snapshotDirMap = null;
|
|
|
+ List<INodeDirectory> snapshotDirs = null;
|
|
|
if (current instanceof INodeDirectoryWithSnapshot) {
|
|
|
- snapshotDirMap = new HashMap<Snapshot, List<INodeDirectory>>();
|
|
|
- dirNum += ((INodeDirectoryWithSnapshot) current).
|
|
|
- getSnapshotDirectory(snapshotDirMap);
|
|
|
+ snapshotDirs = new ArrayList<INodeDirectory>();
|
|
|
+ ((INodeDirectoryWithSnapshot) current).getSnapshotDirectory(
|
|
|
+ snapshotDirs);
|
|
|
+ dirNum += snapshotDirs.size();
|
|
|
}
|
|
|
|
|
|
// 2. Write INodeDirectorySnapshottable#snapshotsByNames to record all
|
|
@@ -1008,20 +957,14 @@ public class FSImageFormat {
|
|
|
// make sure we only save the subtree under a reference node once
|
|
|
boolean toSave = child.isReference() ?
|
|
|
referenceMap.toProcessSubtree(child.getId()) : true;
|
|
|
- currentDirName.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
|
|
|
- saveImage(currentDirName, child.asDirectory(), out, snapshot, toSave);
|
|
|
- currentDirName.position(prefixLen);
|
|
|
+ saveImage(child.asDirectory(), out, toSave);
|
|
|
}
|
|
|
- if (snapshotDirMap != null) {
|
|
|
- for (Entry<Snapshot, List<INodeDirectory>> e : snapshotDirMap.entrySet()) {
|
|
|
- for (INodeDirectory subDir : e.getValue()) {
|
|
|
- // make sure we only save the subtree under a reference node once
|
|
|
- boolean toSave = subDir.getParentReference() != null ?
|
|
|
- referenceMap.toProcessSubtree(subDir.getId()) : true;
|
|
|
- currentDirName.put(PATH_SEPARATOR).put(subDir.getLocalNameBytes());
|
|
|
- saveImage(currentDirName, subDir, out, e.getKey(), toSave);
|
|
|
- currentDirName.position(prefixLen);
|
|
|
- }
|
|
|
+ if (snapshotDirs != null) {
|
|
|
+ for (INodeDirectory subDir : snapshotDirs) {
|
|
|
+ // make sure we only save the subtree under a reference node once
|
|
|
+ boolean toSave = subDir.getParentReference() != null ?
|
|
|
+ referenceMap.toProcessSubtree(subDir.getId()) : true;
|
|
|
+ saveImage(subDir, out, toSave);
|
|
|
}
|
|
|
}
|
|
|
}
|