|
@@ -31,6 +31,9 @@ import java.security.DigestInputStream;
|
|
import java.security.DigestOutputStream;
|
|
import java.security.DigestOutputStream;
|
|
import java.security.MessageDigest;
|
|
import java.security.MessageDigest;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
|
|
+import java.util.HashMap;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -48,6 +51,10 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
|
|
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.INodesInPath;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
@@ -59,13 +66,14 @@ import org.apache.hadoop.io.Text;
|
|
* In particular, the format of the FSImage looks like:
|
|
* In particular, the format of the FSImage looks like:
|
|
* <pre>
|
|
* <pre>
|
|
* FSImage {
|
|
* FSImage {
|
|
- * LayoutVersion: int, NamespaceID: int, NumberItemsInFSDirectoryTree: long,
|
|
|
|
- * NamesystemGenerationStamp: long, TransactionID: long
|
|
|
|
|
|
+ * layoutVersion: int, namespaceID: int, numberItemsInFSDirectoryTree: long,
|
|
|
|
+ * namesystemGenerationStamp: long, transactionID: long,
|
|
|
|
+ * snapshotCounter: int, numberOfSnapshots: int, numOfSnapshottableDirs: int,
|
|
* {FSDirectoryTree, FilesUnderConstruction, SecretManagerState} (can be compressed)
|
|
* {FSDirectoryTree, FilesUnderConstruction, SecretManagerState} (can be compressed)
|
|
* }
|
|
* }
|
|
*
|
|
*
|
|
* FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported) {
|
|
* FSDirectoryTree (if {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported) {
|
|
- * INodeInfo of root, NumberOfChildren of root: int
|
|
|
|
|
|
+ * INodeInfo of root, numberOfChildren of root: int
|
|
* [list of INodeInfo of root's children],
|
|
* [list of INodeInfo of root's children],
|
|
* [list of INodeDirectoryInfo of root's directory children]
|
|
* [list of INodeDirectoryInfo of root's directory children]
|
|
* }
|
|
* }
|
|
@@ -76,38 +84,76 @@ import org.apache.hadoop.io.Text;
|
|
*
|
|
*
|
|
* INodeInfo {
|
|
* INodeInfo {
|
|
* {
|
|
* {
|
|
- * LocalName: short + byte[]
|
|
|
|
|
|
+ * localName: short + byte[]
|
|
* } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported
|
|
* } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is supported
|
|
* or
|
|
* or
|
|
* {
|
|
* {
|
|
- * FullPath: byte[]
|
|
|
|
|
|
+ * fullPath: byte[]
|
|
* } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is not supported
|
|
* } when {@link Feature#FSIMAGE_NAME_OPTIMIZATION} is not supported
|
|
- * ReplicationFactor: short, ModificationTime: long,
|
|
|
|
- * AccessTime: long, PreferredBlockSize: long,
|
|
|
|
- * NumberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
|
|
|
|
|
|
+ * replicationFactor: short, modificationTime: long,
|
|
|
|
+ * accessTime: long, preferredBlockSize: long,
|
|
|
|
+ * numberOfBlocks: int (-1 for INodeDirectory, -2 for INodeSymLink),
|
|
* {
|
|
* {
|
|
- * NsQuota: long, DsQuota: long, FsPermission: short, PermissionStatus
|
|
|
|
|
|
+ * nsQuota: long, dsQuota: long,
|
|
|
|
+ * {
|
|
|
|
+ * isINodeSnapshottable: byte,
|
|
|
|
+ * isINodeWithSnapshot: byte (if isINodeSnapshottable is false)
|
|
|
|
+ * } (when {@link Feature#SNAPSHOT} is supported),
|
|
|
|
+ * fsPermission: short, PermissionStatus
|
|
* } for INodeDirectory
|
|
* } for INodeDirectory
|
|
* or
|
|
* or
|
|
* {
|
|
* {
|
|
- * SymlinkString, FsPermission: short, PermissionStatus
|
|
|
|
|
|
+ * symlinkString, fsPermission: short, PermissionStatus
|
|
* } for INodeSymlink
|
|
* } for INodeSymlink
|
|
* or
|
|
* or
|
|
* {
|
|
* {
|
|
- * [list of BlockInfo], FsPermission: short, PermissionStatus
|
|
|
|
|
|
+ * containsBlock: byte (when {@link Feature#SNAPSHOT} is supported),
|
|
|
|
+ * [list of BlockInfo] (when {@link Feature#SNAPSHOT} is not supported or
|
|
|
|
+ * containsBlock is true),
|
|
|
|
+ * {
|
|
|
|
+ * snapshotFileSize: long,
|
|
|
|
+ * isINodeFileWithLink: byte (if ComputedFileSize is negative),
|
|
|
|
+ * } (when {@link Feature#SNAPSHOT} is supported),
|
|
|
|
+ * fsPermission: short, PermissionStatus
|
|
* } for INodeFile
|
|
* } for INodeFile
|
|
* }
|
|
* }
|
|
*
|
|
*
|
|
* INodeDirectoryInfo {
|
|
* INodeDirectoryInfo {
|
|
- * FullPath of the directory: short + byte[],
|
|
|
|
- * NumberOfChildren: int, [list of INodeInfo of children INode]
|
|
|
|
- * [list of INodeDirectoryInfo of the directory children]
|
|
|
|
|
|
+ * fullPath of the directory: short + byte[],
|
|
|
|
+ * numberOfChildren: int, [list of INodeInfo of children INode],
|
|
|
|
+ * {
|
|
|
|
+ * numberOfSnapshots: int,
|
|
|
|
+ * [list of Snapshot] (when NumberOfSnapshots is positive),
|
|
|
|
+ * numberOfSnapshotDiffs: int,
|
|
|
|
+ * [list of SnapshotDiff] (NumberOfSnapshotDiffs is positive),
|
|
|
|
+ * number of children that are directories,
|
|
|
|
+ * [list of INodeDirectoryInfo of the directory children] (includes
|
|
|
|
+ * snapshot copies of deleted sub-directories)
|
|
|
|
+ * } (when {@link Feature#SNAPSHOT} is supported),
|
|
|
|
+ * }
|
|
|
|
+ *
|
|
|
|
+ * Snapshot {
|
|
|
|
+ * snapshotID: int, root of Snapshot: INodeDirectoryInfo (its local name is
|
|
|
|
+ * the name of the snapshot)
|
|
|
|
+ * }
|
|
|
|
+ *
|
|
|
|
+ * SnapshotDiff {
|
|
|
|
+ * childrenSize: int,
|
|
|
|
+ * full path of the root of the associated Snapshot: short + byte[],
|
|
|
|
+ * isSnapshotRoot: byte,
|
|
|
|
+ * snapshotINodeIsNotNull: byte (when isSnapshotRoot is false),
|
|
|
|
+ * snapshotINode: INodeDirectory (when SnapshotINodeIsNotNull is true), Diff
|
|
|
|
+ * }
|
|
|
|
+ *
|
|
|
|
+ * Diff {
|
|
|
|
+ * createdListSize: int, [Local name of INode in created list],
|
|
|
|
+ * deletedListSize: int, [INode in deleted list: INodeInfo]
|
|
* }
|
|
* }
|
|
* </pre>
|
|
* </pre>
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@InterfaceStability.Evolving
|
|
@InterfaceStability.Evolving
|
|
-class FSImageFormat {
|
|
|
|
|
|
+public class FSImageFormat {
|
|
private static final Log LOG = FSImage.LOG;
|
|
private static final Log LOG = FSImage.LOG;
|
|
|
|
|
|
// Static-only class
|
|
// Static-only class
|
|
@@ -118,7 +164,7 @@ class FSImageFormat {
|
|
* should be called once, after which the getter methods may be used to retrieve
|
|
* should be called once, after which the getter methods may be used to retrieve
|
|
* information about the image that was loaded, if loading was successful.
|
|
* information about the image that was loaded, if loading was successful.
|
|
*/
|
|
*/
|
|
- static class Loader {
|
|
|
|
|
|
+ public static class Loader {
|
|
private final Configuration conf;
|
|
private final Configuration conf;
|
|
/** which namesystem this loader is working for */
|
|
/** which namesystem this loader is working for */
|
|
private final FSNamesystem namesystem;
|
|
private final FSNamesystem namesystem;
|
|
@@ -168,9 +214,7 @@ class FSImageFormat {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- void load(File curFile)
|
|
|
|
- throws IOException
|
|
|
|
- {
|
|
|
|
|
|
+ void load(File curFile) throws IOException {
|
|
checkNotLoaded();
|
|
checkNotLoaded();
|
|
assert curFile != null : "curFile is null";
|
|
assert curFile != null : "curFile is null";
|
|
|
|
|
|
@@ -209,6 +253,10 @@ class FSImageFormat {
|
|
} else {
|
|
} else {
|
|
imgTxId = 0;
|
|
imgTxId = 0;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
|
|
|
|
+ namesystem.getSnapshotManager().read(in);
|
|
|
|
+ }
|
|
|
|
|
|
// read compression related info
|
|
// read compression related info
|
|
FSImageCompression compression;
|
|
FSImageCompression compression;
|
|
@@ -226,7 +274,11 @@ class FSImageFormat {
|
|
LOG.info("Number of files = " + numFiles);
|
|
LOG.info("Number of files = " + numFiles);
|
|
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
|
|
if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
|
|
imgVersion)) {
|
|
imgVersion)) {
|
|
- loadLocalNameINodes(numFiles, in);
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
|
|
|
|
+ loadLocalNameINodesWithSnapshot(in);
|
|
|
|
+ } else {
|
|
|
|
+ loadLocalNameINodes(numFiles, in);
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
loadFullNameINodes(numFiles, in);
|
|
loadFullNameINodes(numFiles, in);
|
|
}
|
|
}
|
|
@@ -260,7 +312,25 @@ class FSImageFormat {
|
|
fsDir.rootDir.cloneModificationTime(root);
|
|
fsDir.rootDir.cloneModificationTime(root);
|
|
fsDir.rootDir.clonePermissionStatus(root);
|
|
fsDir.rootDir.clonePermissionStatus(root);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Load fsimage files when 1) only local names are stored,
|
|
|
|
+ * and 2) snapshot is supported.
|
|
|
|
+ *
|
|
|
|
+ * @param in Image input stream
|
|
|
|
+ */
|
|
|
|
+ private void loadLocalNameINodesWithSnapshot(DataInputStream in)
|
|
|
|
+ throws IOException {
|
|
|
|
+ assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
|
|
|
|
+ getLayoutVersion());
|
|
|
|
+ assert LayoutVersion.supports(Feature.SNAPSHOT, getLayoutVersion());
|
|
|
|
+
|
|
|
|
+ // load root
|
|
|
|
+ loadRoot(in);
|
|
|
|
+ // load rest of the nodes recursively
|
|
|
|
+ loadDirectoryWithSnapshot(in);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* load fsimage files assuming only local names are stored
|
|
* load fsimage files assuming only local names are stored
|
|
*
|
|
*
|
|
@@ -275,13 +345,9 @@ class FSImageFormat {
|
|
assert numFiles > 0;
|
|
assert numFiles > 0;
|
|
|
|
|
|
// load root
|
|
// load root
|
|
- if( in.readShort() != 0) {
|
|
|
|
- throw new IOException("First node is not root");
|
|
|
|
- }
|
|
|
|
- INode root = loadINode(in);
|
|
|
|
- // update the root's attributes
|
|
|
|
- updateRootAttr(root);
|
|
|
|
- numFiles--;
|
|
|
|
|
|
+ loadRoot(in);
|
|
|
|
+ // have loaded the first file (the root)
|
|
|
|
+ numFiles--;
|
|
|
|
|
|
// load rest of the nodes directory by directory
|
|
// load rest of the nodes directory by directory
|
|
while (numFiles > 0) {
|
|
while (numFiles > 0) {
|
|
@@ -292,6 +358,77 @@ class FSImageFormat {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Load information about root, and use the information to update the root
|
|
|
|
+ * directory of NameSystem.
|
|
|
|
+ * @param in The {@link DataInputStream} instance to read.
|
|
|
|
+ */
|
|
|
|
+ private void loadRoot(DataInputStream in) throws IOException {
|
|
|
|
+ // load root
|
|
|
|
+ if (in.readShort() != 0) {
|
|
|
|
+ throw new IOException("First node is not root");
|
|
|
|
+ }
|
|
|
|
+ INode root = loadINode(in);
|
|
|
|
+ // update the root's attributes
|
|
|
|
+ updateRootAttr(root);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Load children nodes for the parent directory. */
|
|
|
|
+ private void loadChildren(INodeDirectory parent, DataInputStream in)
|
|
|
|
+ throws IOException {
|
|
|
|
+ int numChildren = in.readInt();
|
|
|
|
+ for (int i = 0; i < numChildren; i++) {
|
|
|
|
+ // load single inode
|
|
|
|
+ byte[] localName = new byte[in.readShort()];
|
|
|
|
+ in.readFully(localName); // read local name
|
|
|
|
+ INode newNode = loadINode(in); // read rest of inode
|
|
|
|
+ newNode.setLocalName(localName);
|
|
|
|
+ addToParent(parent, newNode);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Load a directory when snapshot is supported.
|
|
|
|
+ * @param in The {@link DataInputStream} instance to read.
|
|
|
|
+ */
|
|
|
|
+ private void loadDirectoryWithSnapshot(DataInputStream 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);
|
|
|
|
+
|
|
|
|
+ // Step 2. Load children nodes under parent
|
|
|
|
+ loadChildren(parent, in);
|
|
|
|
+
|
|
|
|
+ // Step 3. Load snapshots if parent is snapshottable
|
|
|
|
+ int numSnapshots = in.readInt();
|
|
|
|
+ INodeDirectorySnapshottable snapshottableParent = null;
|
|
|
|
+ if (numSnapshots >= 0) {
|
|
|
|
+ snapshottableParent = (INodeDirectorySnapshottable) parent;
|
|
|
|
+ // load snapshots and snapshotQuota
|
|
|
|
+ SnapshotFSImageFormat.loadSnapshotList(snapshottableParent,
|
|
|
|
+ numSnapshots, in, this);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Step 4. load SnapshotDiff list
|
|
|
|
+ int numSnapshotDiffs = in.readInt();
|
|
|
|
+ if (numSnapshotDiffs >= 0) {
|
|
|
|
+ INodeDirectoryWithSnapshot parentWithSnapshot =
|
|
|
|
+ (INodeDirectoryWithSnapshot) parent;
|
|
|
|
+ // load SnapshotDiff list
|
|
|
|
+ SnapshotFSImageFormat.loadSnapshotDiffList(parentWithSnapshot,
|
|
|
|
+ numSnapshotDiffs, in, this);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Recursively load sub-directories, including snapshot copies of deleted
|
|
|
|
+ // directories
|
|
|
|
+ int numSubTree = in.readInt();
|
|
|
|
+ for (int i = 0; i < numSubTree; i++) {
|
|
|
|
+ loadDirectoryWithSnapshot(in);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Load all children of a directory
|
|
* Load all children of a directory
|
|
*
|
|
*
|
|
@@ -388,17 +525,25 @@ class FSImageFormat {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /** @return The FSDirectory of the namesystem where the fsimage is loaded */
|
|
|
|
+ public FSDirectory getFSDirectoryInLoading() {
|
|
|
|
+ return namesystem.dir;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* load an inode from fsimage except for its name
|
|
* load an inode from fsimage except for its name
|
|
*
|
|
*
|
|
* @param in data input stream from which image is read
|
|
* @param in data input stream from which image is read
|
|
* @return an inode
|
|
* @return an inode
|
|
*/
|
|
*/
|
|
- private INode loadINode(DataInputStream in)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ public INode loadINode(DataInputStream in) throws IOException {
|
|
long modificationTime = 0;
|
|
long modificationTime = 0;
|
|
long atime = 0;
|
|
long atime = 0;
|
|
long blockSize = 0;
|
|
long blockSize = 0;
|
|
|
|
+ long computeFileSize = -1;
|
|
|
|
+ boolean snapshottable = false;
|
|
|
|
+ boolean withSnapshot = false;
|
|
|
|
+ boolean withLink = false;
|
|
|
|
|
|
int imgVersion = getLayoutVersion();
|
|
int imgVersion = getLayoutVersion();
|
|
long inodeId = namesystem.allocateNewInodeId();
|
|
long inodeId = namesystem.allocateNewInodeId();
|
|
@@ -414,11 +559,22 @@ class FSImageFormat {
|
|
BlockInfo blocks[] = null;
|
|
BlockInfo blocks[] = null;
|
|
|
|
|
|
if (numBlocks >= 0) {
|
|
if (numBlocks >= 0) {
|
|
- blocks = new BlockInfo[numBlocks];
|
|
|
|
|
|
+ // to indicate INodeFileWithLink, blocks may be set as null while
|
|
|
|
+ // numBlocks is set to 0
|
|
|
|
+ blocks = LayoutVersion.supports(Feature.SNAPSHOT, imgVersion) ? (in
|
|
|
|
+ .readBoolean() ? new BlockInfo[numBlocks] : null)
|
|
|
|
+ : new BlockInfo[numBlocks];
|
|
|
|
+
|
|
for (int j = 0; j < numBlocks; j++) {
|
|
for (int j = 0; j < numBlocks; j++) {
|
|
blocks[j] = new BlockInfo(replication);
|
|
blocks[j] = new BlockInfo(replication);
|
|
blocks[j].readFields(in);
|
|
blocks[j].readFields(in);
|
|
}
|
|
}
|
|
|
|
+ if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
|
|
|
|
+ computeFileSize = in.readLong();
|
|
|
|
+ if (computeFileSize < 0) {
|
|
|
|
+ withLink = in.readBoolean();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
// get quota only when the node is a directory
|
|
// get quota only when the node is a directory
|
|
@@ -431,7 +587,14 @@ class FSImageFormat {
|
|
&& blocks == null && numBlocks == -1) {
|
|
&& blocks == null && numBlocks == -1) {
|
|
dsQuota = in.readLong();
|
|
dsQuota = in.readLong();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+ if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)
|
|
|
|
+ && blocks == null && numBlocks == -1) {
|
|
|
|
+ snapshottable = in.readBoolean();
|
|
|
|
+ if (!snapshottable) {
|
|
|
|
+ withSnapshot = in.readBoolean();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
// Read the symlink only when the node is a symlink
|
|
// Read the symlink only when the node is a symlink
|
|
String symlink = "";
|
|
String symlink = "";
|
|
if (numBlocks == -2) {
|
|
if (numBlocks == -2) {
|
|
@@ -441,7 +604,8 @@ class FSImageFormat {
|
|
PermissionStatus permissions = PermissionStatus.read(in);
|
|
PermissionStatus permissions = PermissionStatus.read(in);
|
|
|
|
|
|
return INode.newINode(inodeId, permissions, blocks, symlink, replication,
|
|
return INode.newINode(inodeId, permissions, blocks, symlink, replication,
|
|
- modificationTime, atime, nsQuota, dsQuota, blockSize);
|
|
|
|
|
|
+ modificationTime, atime, nsQuota, dsQuota, blockSize, numBlocks,
|
|
|
|
+ withLink, computeFileSize, snapshottable, withSnapshot);
|
|
}
|
|
}
|
|
|
|
|
|
private void loadFilesUnderConstruction(DataInputStream in)
|
|
private void loadFilesUnderConstruction(DataInputStream in)
|
|
@@ -557,9 +721,7 @@ class FSImageFormat {
|
|
return savedDigest;
|
|
return savedDigest;
|
|
}
|
|
}
|
|
|
|
|
|
- void save(File newFile,
|
|
|
|
- FSImageCompression compression)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ void save(File newFile, FSImageCompression compression) throws IOException {
|
|
checkNotSaved();
|
|
checkNotSaved();
|
|
|
|
|
|
final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
|
|
final FSNamesystem sourceNamesystem = context.getSourceNamesystem();
|
|
@@ -584,19 +746,19 @@ class FSImageFormat {
|
|
out.writeLong(fsDir.rootDir.numItemsInTree());
|
|
out.writeLong(fsDir.rootDir.numItemsInTree());
|
|
out.writeLong(sourceNamesystem.getGenerationStamp());
|
|
out.writeLong(sourceNamesystem.getGenerationStamp());
|
|
out.writeLong(context.getTxId());
|
|
out.writeLong(context.getTxId());
|
|
-
|
|
|
|
|
|
+ sourceNamesystem.getSnapshotManager().write(out);
|
|
|
|
+
|
|
// write compression info and set up compressed stream
|
|
// write compression info and set up compressed stream
|
|
out = compression.writeHeaderAndWrapStream(fos);
|
|
out = compression.writeHeaderAndWrapStream(fos);
|
|
LOG.info("Saving image file " + newFile +
|
|
LOG.info("Saving image file " + newFile +
|
|
" using " + compression);
|
|
" using " + compression);
|
|
|
|
|
|
-
|
|
|
|
byte[] byteStore = new byte[4*HdfsConstants.MAX_PATH_LENGTH];
|
|
byte[] byteStore = new byte[4*HdfsConstants.MAX_PATH_LENGTH];
|
|
ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
|
|
ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
|
|
// save the root
|
|
// save the root
|
|
FSImageSerialization.saveINode2Image(fsDir.rootDir, out);
|
|
FSImageSerialization.saveINode2Image(fsDir.rootDir, out);
|
|
// save the rest of the nodes
|
|
// save the rest of the nodes
|
|
- saveImage(strbuf, fsDir.rootDir, out);
|
|
|
|
|
|
+ saveImage(strbuf, fsDir.rootDir, out, null);
|
|
// save files under construction
|
|
// save files under construction
|
|
sourceNamesystem.saveFilesUnderConstruction(out);
|
|
sourceNamesystem.saveFilesUnderConstruction(out);
|
|
context.checkCancelled();
|
|
context.checkCancelled();
|
|
@@ -619,42 +781,143 @@ class FSImageFormat {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Save file tree image starting from the given root.
|
|
|
|
- * This is a recursive procedure, which first saves all children of
|
|
|
|
- * a current directory and then moves inside the sub-directories.
|
|
|
|
|
|
+ * Save children INodes.
|
|
|
|
+ * @param children The list of children INodes
|
|
|
|
+ * @param out The DataOutputStream to write
|
|
|
|
+ * @return Number of children that are directory
|
|
*/
|
|
*/
|
|
- private void saveImage(ByteBuffer currentDirName,
|
|
|
|
- INodeDirectory current,
|
|
|
|
- DataOutputStream out) throws IOException {
|
|
|
|
- final ReadOnlyList<INode> children = current.getChildrenList(null);
|
|
|
|
- if (children.isEmpty()) {
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- // print prefix (parent directory name)
|
|
|
|
- int prefixLen = currentDirName.position();
|
|
|
|
- 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);
|
|
|
|
- }
|
|
|
|
|
|
+ private int saveChildren(ReadOnlyList<INode> children, DataOutputStream out)
|
|
|
|
+ throws IOException {
|
|
|
|
+ // Write normal children INode.
|
|
out.writeInt(children.size());
|
|
out.writeInt(children.size());
|
|
|
|
+ int dirNum = 0;
|
|
int i = 0;
|
|
int i = 0;
|
|
for(INode child : children) {
|
|
for(INode child : children) {
|
|
// print all children first
|
|
// print all children first
|
|
FSImageSerialization.saveINode2Image(child, out);
|
|
FSImageSerialization.saveINode2Image(child, out);
|
|
|
|
+ if (child.isDirectory()) {
|
|
|
|
+ dirNum++;
|
|
|
|
+ }
|
|
if (i++ % 50 == 0) {
|
|
if (i++ % 50 == 0) {
|
|
context.checkCancelled();
|
|
context.checkCancelled();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ 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 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());
|
|
|
|
+ String snapshotFullPath = snapshotParentFullPath + Path.SEPARATOR
|
|
|
|
+ + HdfsConstants.DOT_SNAPSHOT_DIR + Path.SEPARATOR + snapshotName
|
|
|
|
+ + relativePath;
|
|
|
|
+ return snapshotFullPath;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * 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
|
|
|
|
+ */
|
|
|
|
+ private void saveImage(ByteBuffer currentDirName, INodeDirectory current,
|
|
|
|
+ DataOutputStream out, Snapshot snapshot)
|
|
|
|
+ throws IOException {
|
|
|
|
+ final ReadOnlyList<INode> children = current.getChildrenList(null);
|
|
|
|
+ int dirNum = 0;
|
|
|
|
+ Map<Snapshot, List<INodeDirectory>> snapshotDirMap = null;
|
|
|
|
+ if (current instanceof INodeDirectoryWithSnapshot) {
|
|
|
|
+ snapshotDirMap = new HashMap<Snapshot, List<INodeDirectory>>();
|
|
|
|
+ dirNum += ((INodeDirectoryWithSnapshot) current).
|
|
|
|
+ getSnapshotDirectory(snapshotDirMap);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // 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);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // 2. Write children INode
|
|
|
|
+ dirNum += saveChildren(children, out);
|
|
|
|
+
|
|
|
|
+ // 3. Write INodeDirectorySnapshottable#snapshotsByNames to record all
|
|
|
|
+ // Snapshots
|
|
|
|
+ if (current instanceof INodeDirectorySnapshottable) {
|
|
|
|
+ INodeDirectorySnapshottable snapshottableNode =
|
|
|
|
+ (INodeDirectorySnapshottable) current;
|
|
|
|
+ SnapshotFSImageFormat.saveSnapshots(snapshottableNode, out);
|
|
|
|
+ } else {
|
|
|
|
+ out.writeInt(-1); // # of snapshots
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // 4. Write SnapshotDiff lists.
|
|
|
|
+ if (current instanceof INodeDirectoryWithSnapshot) {
|
|
|
|
+ INodeDirectoryWithSnapshot sNode = (INodeDirectoryWithSnapshot) current;
|
|
|
|
+ SnapshotFSImageFormat.saveSnapshotDiffs(sNode, out);
|
|
|
|
+ } else {
|
|
|
|
+ out.writeInt(-1); // # of SnapshotDiffs
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Write sub-tree of sub-directories, including possible snapshots of
|
|
|
|
+ // deleted sub-directories
|
|
|
|
+ out.writeInt(dirNum); // the number of sub-directories
|
|
for(INode child : children) {
|
|
for(INode child : children) {
|
|
if(!child.isDirectory())
|
|
if(!child.isDirectory())
|
|
continue;
|
|
continue;
|
|
currentDirName.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
|
|
currentDirName.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
|
|
- saveImage(currentDirName, (INodeDirectory)child, out);
|
|
|
|
|
|
+ saveImage(currentDirName, (INodeDirectory)child, out, snapshot);
|
|
currentDirName.position(prefixLen);
|
|
currentDirName.position(prefixLen);
|
|
}
|
|
}
|
|
|
|
+ if (snapshotDirMap != null) {
|
|
|
|
+ for (Snapshot ss : snapshotDirMap.keySet()) {
|
|
|
|
+ List<INodeDirectory> snapshotSubDirs = snapshotDirMap.get(ss);
|
|
|
|
+ for (INodeDirectory subDir : snapshotSubDirs) {
|
|
|
|
+ currentDirName.put(PATH_SEPARATOR).put(subDir.getLocalNameBytes());
|
|
|
|
+ saveImage(currentDirName, subDir, out, ss);
|
|
|
|
+ currentDirName.position(prefixLen);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|