|
@@ -51,8 +51,11 @@ 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.FileWithSnapshot.FileDiffList;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
|
|
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.INodeDirectoryWithSnapshot;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileWithSnapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
@@ -107,17 +110,13 @@ import org.apache.hadoop.io.Text;
|
|
* } for INodeSymlink
|
|
* } for INodeSymlink
|
|
* or
|
|
* or
|
|
* {
|
|
* {
|
|
- * containsBlock: byte (when {@link Feature#SNAPSHOT} is supported),
|
|
|
|
- * [list of BlockInfo] (when {@link Feature#SNAPSHOT} is not supported or
|
|
|
|
- * containsBlock is true),
|
|
|
|
|
|
+ * [list of BlockInfo]
|
|
|
|
+ * [list of FileDiff]
|
|
* {
|
|
* {
|
|
- * snapshotFileSize: long (negative is the file is not a snapshot copy),
|
|
|
|
- * isINodeFileUnderConstructionSnapshot: byte (if snapshotFileSize
|
|
|
|
- * is positive),
|
|
|
|
|
|
+ * isINodeFileUnderConstructionSnapshot: byte,
|
|
* {clientName: short + byte[], clientMachine: short + byte[]} (when
|
|
* {clientName: short + byte[], clientMachine: short + byte[]} (when
|
|
* isINodeFileUnderConstructionSnapshot is true),
|
|
* isINodeFileUnderConstructionSnapshot is true),
|
|
- * isINodeFileWithSnapshot: byte (if snapshotFileSize is negative),
|
|
|
|
- * } (when {@link Feature#SNAPSHOT} is supported),
|
|
|
|
|
|
+ * } (when {@link Feature#SNAPSHOT} is supported and writing snapshotINode),
|
|
* fsPermission: short, PermissionStatus
|
|
* fsPermission: short, PermissionStatus
|
|
* } for INodeFile
|
|
* } for INodeFile
|
|
* }
|
|
* }
|
|
@@ -128,8 +127,8 @@ import org.apache.hadoop.io.Text;
|
|
* {
|
|
* {
|
|
* numberOfSnapshots: int,
|
|
* numberOfSnapshots: int,
|
|
* [list of Snapshot] (when NumberOfSnapshots is positive),
|
|
* [list of Snapshot] (when NumberOfSnapshots is positive),
|
|
- * numberOfSnapshotDiffs: int,
|
|
|
|
- * [list of SnapshotDiff] (NumberOfSnapshotDiffs is positive),
|
|
|
|
|
|
+ * numberOfDirectoryDiffs: int,
|
|
|
|
+ * [list of DirectoryDiff] (NumberOfDirectoryDiffs is positive),
|
|
* number of children that are directories,
|
|
* number of children that are directories,
|
|
* [list of INodeDirectoryInfo of the directory children] (includes
|
|
* [list of INodeDirectoryInfo of the directory children] (includes
|
|
* snapshot copies of deleted sub-directories)
|
|
* snapshot copies of deleted sub-directories)
|
|
@@ -141,9 +140,9 @@ import org.apache.hadoop.io.Text;
|
|
* the name of the snapshot)
|
|
* the name of the snapshot)
|
|
* }
|
|
* }
|
|
*
|
|
*
|
|
- * SnapshotDiff {
|
|
|
|
- * childrenSize: int,
|
|
|
|
|
|
+ * DirectoryDiff {
|
|
* full path of the root of the associated Snapshot: short + byte[],
|
|
* full path of the root of the associated Snapshot: short + byte[],
|
|
|
|
+ * childrenSize: int,
|
|
* isSnapshotRoot: byte,
|
|
* isSnapshotRoot: byte,
|
|
* snapshotINodeIsNotNull: byte (when isSnapshotRoot is false),
|
|
* snapshotINodeIsNotNull: byte (when isSnapshotRoot is false),
|
|
* snapshotINode: INodeDirectory (when SnapshotINodeIsNotNull is true), Diff
|
|
* snapshotINode: INodeDirectory (when SnapshotINodeIsNotNull is true), Diff
|
|
@@ -153,6 +152,13 @@ import org.apache.hadoop.io.Text;
|
|
* createdListSize: int, [Local name of INode in created list],
|
|
* createdListSize: int, [Local name of INode in created list],
|
|
* deletedListSize: int, [INode in deleted list: INodeInfo]
|
|
* deletedListSize: int, [INode in deleted list: INodeInfo]
|
|
* }
|
|
* }
|
|
|
|
+ *
|
|
|
|
+ * FileDiff {
|
|
|
|
+ * full path of the root of the associated Snapshot: short + byte[],
|
|
|
|
+ * fileSize: long,
|
|
|
|
+ * snapshotINodeIsNotNull: byte,
|
|
|
|
+ * snapshotINode: INodeFile (when SnapshotINodeIsNotNull is true), Diff
|
|
|
|
+ * }
|
|
* </pre>
|
|
* </pre>
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@@ -374,23 +380,21 @@ public class FSImageFormat {
|
|
if (in.readShort() != 0) {
|
|
if (in.readShort() != 0) {
|
|
throw new IOException("First node is not root");
|
|
throw new IOException("First node is not root");
|
|
}
|
|
}
|
|
- INode root = loadINode(in);
|
|
|
|
|
|
+ final INode root = loadINode(null, false, in);
|
|
// update the root's attributes
|
|
// update the root's attributes
|
|
updateRootAttr(root);
|
|
updateRootAttr(root);
|
|
}
|
|
}
|
|
|
|
|
|
/** Load children nodes for the parent directory. */
|
|
/** Load children nodes for the parent directory. */
|
|
- private void loadChildren(INodeDirectory parent, DataInputStream in)
|
|
|
|
|
|
+ private int loadChildren(INodeDirectory parent, DataInputStream in)
|
|
throws IOException {
|
|
throws IOException {
|
|
int numChildren = in.readInt();
|
|
int numChildren = in.readInt();
|
|
for (int i = 0; i < numChildren; i++) {
|
|
for (int i = 0; i < numChildren; i++) {
|
|
// load single inode
|
|
// 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);
|
|
|
|
|
|
+ INode newNode = loadINodeWithLocalName(false, in);
|
|
addToParent(parent, newNode);
|
|
addToParent(parent, newNode);
|
|
}
|
|
}
|
|
|
|
+ return numChildren;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -404,28 +408,21 @@ public class FSImageFormat {
|
|
final INodeDirectory parent = INodeDirectory.valueOf(
|
|
final INodeDirectory parent = INodeDirectory.valueOf(
|
|
namesystem.dir.rootDir.getNode(parentPath, false), parentPath);
|
|
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
|
|
|
|
|
|
+ // Step 2. Load snapshots if parent is snapshottable
|
|
int numSnapshots = in.readInt();
|
|
int numSnapshots = in.readInt();
|
|
- INodeDirectorySnapshottable snapshottableParent = null;
|
|
|
|
if (numSnapshots >= 0) {
|
|
if (numSnapshots >= 0) {
|
|
- snapshottableParent = (INodeDirectorySnapshottable) parent;
|
|
|
|
|
|
+ final INodeDirectorySnapshottable snapshottableParent
|
|
|
|
+ = INodeDirectorySnapshottable.valueOf(parent, parentPath);
|
|
// load snapshots and snapshotQuota
|
|
// load snapshots and snapshotQuota
|
|
SnapshotFSImageFormat.loadSnapshotList(snapshottableParent,
|
|
SnapshotFSImageFormat.loadSnapshotList(snapshottableParent,
|
|
numSnapshots, in, this);
|
|
numSnapshots, in, this);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Step 3. Load children nodes under parent
|
|
|
|
+ loadChildren(parent, in);
|
|
|
|
|
|
- // 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);
|
|
|
|
- }
|
|
|
|
|
|
+ // Step 4. load Directory Diff List
|
|
|
|
+ SnapshotFSImageFormat.loadDirectoryDiffList(parent, in, this);
|
|
|
|
|
|
// Recursively load sub-directories, including snapshot copies of deleted
|
|
// Recursively load sub-directories, including snapshot copies of deleted
|
|
// directories
|
|
// directories
|
|
@@ -444,22 +441,9 @@ public class FSImageFormat {
|
|
*/
|
|
*/
|
|
private int loadDirectory(DataInputStream in) throws IOException {
|
|
private int loadDirectory(DataInputStream in) throws IOException {
|
|
String parentPath = FSImageSerialization.readString(in);
|
|
String parentPath = FSImageSerialization.readString(in);
|
|
- FSDirectory fsDir = namesystem.dir;
|
|
|
|
final INodeDirectory parent = INodeDirectory.valueOf(
|
|
final INodeDirectory parent = INodeDirectory.valueOf(
|
|
- fsDir.rootDir.getNode(parentPath, true), parentPath);
|
|
|
|
-
|
|
|
|
- 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
|
|
|
|
-
|
|
|
|
- // add to parent
|
|
|
|
- newNode.setLocalName(localName);
|
|
|
|
- addToParent(parent, newNode);
|
|
|
|
- }
|
|
|
|
- return numChildren;
|
|
|
|
|
|
+ namesystem.dir.rootDir.getNode(parentPath, true), parentPath);
|
|
|
|
+ return loadChildren(parent, in);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -477,7 +461,8 @@ public class FSImageFormat {
|
|
INodeDirectory parentINode = fsDir.rootDir;
|
|
INodeDirectory parentINode = fsDir.rootDir;
|
|
for (long i = 0; i < numFiles; i++) {
|
|
for (long i = 0; i < numFiles; i++) {
|
|
pathComponents = FSImageSerialization.readPathComponents(in);
|
|
pathComponents = FSImageSerialization.readPathComponents(in);
|
|
- INode newNode = loadINode(in);
|
|
|
|
|
|
+ final INode newNode = loadINode(pathComponents[pathComponents.length-1],
|
|
|
|
+ false, in);
|
|
|
|
|
|
if (isRoot(pathComponents)) { // it is the root
|
|
if (isRoot(pathComponents)) { // it is the root
|
|
// update the root's attributes
|
|
// update the root's attributes
|
|
@@ -491,7 +476,6 @@ public class FSImageFormat {
|
|
}
|
|
}
|
|
|
|
|
|
// add new inode
|
|
// add new inode
|
|
- newNode.setLocalName(pathComponents[pathComponents.length-1]);
|
|
|
|
addToParent(parentINode, newNode);
|
|
addToParent(parentINode, newNode);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -524,9 +508,11 @@ public class FSImageFormat {
|
|
// Add file->block mapping
|
|
// Add file->block mapping
|
|
final INodeFile file = (INodeFile)child;
|
|
final INodeFile file = (INodeFile)child;
|
|
final BlockInfo[] blocks = file.getBlocks();
|
|
final BlockInfo[] blocks = file.getBlocks();
|
|
- final BlockManager bm = namesystem.getBlockManager();
|
|
|
|
- for (int i = 0; i < blocks.length; i++) {
|
|
|
|
- file.setBlock(i, bm.addBlockCollection(blocks[i], file));
|
|
|
|
|
|
+ if (blocks != null) {
|
|
|
|
+ final BlockManager bm = namesystem.getBlockManager();
|
|
|
|
+ for (int i = 0; i < blocks.length; i++) {
|
|
|
|
+ file.setBlock(i, bm.addBlockCollection(blocks[i], file));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -535,6 +521,15 @@ public class FSImageFormat {
|
|
public FSDirectory getFSDirectoryInLoading() {
|
|
public FSDirectory getFSDirectoryInLoading() {
|
|
return namesystem.dir;
|
|
return namesystem.dir;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public INode loadINodeWithLocalName(boolean isSnapshotINode,
|
|
|
|
+ DataInputStream in) throws IOException {
|
|
|
|
+ final byte[] localName = new byte[in.readShort()];
|
|
|
|
+ in.readFully(localName);
|
|
|
|
+ final INode inode = loadINode(localName, isSnapshotINode, in);
|
|
|
|
+ inode.setLocalName(localName);
|
|
|
|
+ return inode;
|
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
/**
|
|
* load an inode from fsimage except for its name
|
|
* load an inode from fsimage except for its name
|
|
@@ -542,45 +537,43 @@ public class FSImageFormat {
|
|
* @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
|
|
*/
|
|
*/
|
|
- public INode loadINode(DataInputStream in) throws IOException {
|
|
|
|
- long modificationTime = 0;
|
|
|
|
- long atime = 0;
|
|
|
|
- long blockSize = 0;
|
|
|
|
- long computeFileSize = -1;
|
|
|
|
- boolean snapshottable = false;
|
|
|
|
- boolean withSnapshot = false;
|
|
|
|
-
|
|
|
|
- int imgVersion = getLayoutVersion();
|
|
|
|
- long inodeId = namesystem.allocateNewInodeId();
|
|
|
|
|
|
+ INode loadINode(final byte[] localName, boolean isSnapshotINode,
|
|
|
|
+ DataInputStream in) throws IOException {
|
|
|
|
+ final int imgVersion = getLayoutVersion();
|
|
|
|
+ final long inodeId = namesystem.allocateNewInodeId();
|
|
|
|
|
|
- short replication = in.readShort();
|
|
|
|
- replication = namesystem.getBlockManager().adjustReplication(replication);
|
|
|
|
- modificationTime = in.readLong();
|
|
|
|
|
|
+ final short replication = namesystem.getBlockManager().adjustReplication(
|
|
|
|
+ in.readShort());
|
|
|
|
+ final long modificationTime = in.readLong();
|
|
|
|
+ long atime = 0;
|
|
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) {
|
|
if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) {
|
|
atime = in.readLong();
|
|
atime = in.readLong();
|
|
}
|
|
}
|
|
- blockSize = in.readLong();
|
|
|
|
- int numBlocks = in.readInt();
|
|
|
|
- BlockInfo blocks[] = null;
|
|
|
|
|
|
+ final long blockSize = in.readLong();
|
|
|
|
+ final int numBlocks = in.readInt();
|
|
|
|
|
|
- String clientName = "";
|
|
|
|
- String clientMachine = "";
|
|
|
|
- boolean underConstruction = false;
|
|
|
|
if (numBlocks >= 0) {
|
|
if (numBlocks >= 0) {
|
|
- // to indicate INodeFileWithSnapshot, 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];
|
|
|
|
|
|
+ // file
|
|
|
|
|
|
- for (int j = 0; j < numBlocks; j++) {
|
|
|
|
- blocks[j] = new BlockInfo(replication);
|
|
|
|
- blocks[j].readFields(in);
|
|
|
|
|
|
+ // read blocks
|
|
|
|
+ BlockInfo[] blocks = null;
|
|
|
|
+ if (numBlocks > 0) {
|
|
|
|
+ blocks = new BlockInfo[numBlocks];
|
|
|
|
+ for (int j = 0; j < numBlocks; j++) {
|
|
|
|
+ blocks[j] = new BlockInfo(replication);
|
|
|
|
+ blocks[j].readFields(in);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ String clientName = "";
|
|
|
|
+ String clientMachine = "";
|
|
|
|
+ boolean underConstruction = false;
|
|
|
|
+ FileDiffList fileDiffs = null;
|
|
if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
|
|
if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
|
|
- computeFileSize = in.readLong();
|
|
|
|
- if (computeFileSize < 0) {
|
|
|
|
- } else {
|
|
|
|
|
|
+ // read diffs
|
|
|
|
+ fileDiffs = SnapshotFSImageFormat.loadFileDiffList(in, this);
|
|
|
|
+
|
|
|
|
+ if (isSnapshotINode) {
|
|
underConstruction = in.readBoolean();
|
|
underConstruction = in.readBoolean();
|
|
if (underConstruction) {
|
|
if (underConstruction) {
|
|
clientName = FSImageSerialization.readString(in);
|
|
clientName = FSImageSerialization.readString(in);
|
|
@@ -588,38 +581,56 @@ public class FSImageFormat {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- }
|
|
|
|
-
|
|
|
|
- // get quota only when the node is a directory
|
|
|
|
- long nsQuota = -1L;
|
|
|
|
- if (blocks == null && numBlocks == -1) {
|
|
|
|
- nsQuota = in.readLong();
|
|
|
|
- }
|
|
|
|
- long dsQuota = -1L;
|
|
|
|
- if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imgVersion)
|
|
|
|
- && blocks == null && numBlocks == -1) {
|
|
|
|
- dsQuota = in.readLong();
|
|
|
|
- }
|
|
|
|
- if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)
|
|
|
|
- && blocks == null && numBlocks == -1) {
|
|
|
|
- snapshottable = in.readBoolean();
|
|
|
|
- if (!snapshottable) {
|
|
|
|
- withSnapshot = in.readBoolean();
|
|
|
|
|
|
+
|
|
|
|
+ final PermissionStatus permissions = PermissionStatus.read(in);
|
|
|
|
+
|
|
|
|
+ // return
|
|
|
|
+ final INodeFile file = new INodeFile(inodeId, localName, permissions,
|
|
|
|
+ modificationTime, atime, blocks, replication, blockSize);
|
|
|
|
+ return fileDiffs != null? new INodeFileWithSnapshot(file, fileDiffs)
|
|
|
|
+ : underConstruction? new INodeFileUnderConstruction(
|
|
|
|
+ file, clientName, clientMachine, null)
|
|
|
|
+ : file;
|
|
|
|
+ } else if (numBlocks == -1) {
|
|
|
|
+ //directory
|
|
|
|
+
|
|
|
|
+ //read quotas
|
|
|
|
+ final long nsQuota = in.readLong();
|
|
|
|
+ long dsQuota = -1L;
|
|
|
|
+ if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imgVersion)) {
|
|
|
|
+ dsQuota = in.readLong();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ //read snapshot info
|
|
|
|
+ boolean snapshottable = false;
|
|
|
|
+ boolean withSnapshot = false;
|
|
|
|
+ if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
|
|
|
|
+ snapshottable = in.readBoolean();
|
|
|
|
+ if (!snapshottable) {
|
|
|
|
+ withSnapshot = in.readBoolean();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ final PermissionStatus permissions = PermissionStatus.read(in);
|
|
|
|
+
|
|
|
|
+ //return
|
|
|
|
+ final INodeDirectory dir = nsQuota >= 0 || dsQuota >= 0?
|
|
|
|
+ new INodeDirectoryWithQuota(inodeId, localName, permissions,
|
|
|
|
+ modificationTime, nsQuota, dsQuota)
|
|
|
|
+ : new INodeDirectory(inodeId, localName, permissions, modificationTime);
|
|
|
|
+ return snapshottable ? new INodeDirectorySnapshottable(dir)
|
|
|
|
+ : withSnapshot ? new INodeDirectoryWithSnapshot(dir)
|
|
|
|
+ : dir;
|
|
|
|
+ } else if (numBlocks == -2) {
|
|
|
|
+ //symlink
|
|
|
|
+
|
|
|
|
+ final String symlink = Text.readString(in);
|
|
|
|
+ final PermissionStatus permissions = PermissionStatus.read(in);
|
|
|
|
+ return new INodeSymlink(inodeId, localName, permissions,
|
|
|
|
+ modificationTime, atime, symlink);
|
|
}
|
|
}
|
|
|
|
|
|
- // Read the symlink only when the node is a symlink
|
|
|
|
- String symlink = "";
|
|
|
|
- if (numBlocks == -2) {
|
|
|
|
- symlink = Text.readString(in);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- PermissionStatus permissions = PermissionStatus.read(in);
|
|
|
|
-
|
|
|
|
- return INode.newINode(inodeId, permissions, blocks, symlink, replication,
|
|
|
|
- modificationTime, atime, nsQuota, dsQuota, blockSize, numBlocks,
|
|
|
|
- computeFileSize, snapshottable, withSnapshot,
|
|
|
|
- underConstruction, clientName, clientMachine);
|
|
|
|
|
|
+ throw new IOException("Unknown inode type: numBlocks=" + numBlocks);
|
|
}
|
|
}
|
|
|
|
|
|
private void loadFilesUnderConstruction(DataInputStream in,
|
|
private void loadFilesUnderConstruction(DataInputStream in,
|
|
@@ -630,16 +641,22 @@ public class FSImageFormat {
|
|
LOG.info("Number of files under construction = " + size);
|
|
LOG.info("Number of files under construction = " + size);
|
|
|
|
|
|
for (int i = 0; i < size; i++) {
|
|
for (int i = 0; i < size; i++) {
|
|
- INodeFileUnderConstruction cons =
|
|
|
|
- FSImageSerialization.readINodeUnderConstruction(in, supportSnapshot);
|
|
|
|
|
|
+ INodeFileUnderConstruction cons
|
|
|
|
+ = FSImageSerialization.readINodeUnderConstruction(in);
|
|
|
|
|
|
// verify that file exists in namespace
|
|
// verify that file exists in namespace
|
|
String path = cons.getLocalName();
|
|
String path = cons.getLocalName();
|
|
final INodesInPath iip = fsDir.getLastINodeInPath(path);
|
|
final INodesInPath iip = fsDir.getLastINodeInPath(path);
|
|
INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
|
|
INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
|
|
cons.setLocalName(oldnode.getLocalNameBytes());
|
|
cons.setLocalName(oldnode.getLocalNameBytes());
|
|
- fsDir.unprotectedReplaceINodeFile(path, oldnode, cons,
|
|
|
|
- iip.getLatestSnapshot());
|
|
|
|
|
|
+ cons.setParent(oldnode.getParent());
|
|
|
|
+
|
|
|
|
+ if (oldnode instanceof INodeFileWithSnapshot) {
|
|
|
|
+ cons = new INodeFileUnderConstructionWithSnapshot(cons,
|
|
|
|
+ ((INodeFileWithSnapshot)oldnode).getDiffs());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ fsDir.unprotectedReplaceINodeFile(path, oldnode, cons);
|
|
namesystem.leaseManager.addLease(cons.getClientName(), path);
|
|
namesystem.leaseManager.addLease(cons.getClientName(), path);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -892,10 +909,7 @@ public class FSImageFormat {
|
|
out.write(snapshotFullPathBytes);
|
|
out.write(snapshotFullPathBytes);
|
|
}
|
|
}
|
|
|
|
|
|
- // 2. Write children INode
|
|
|
|
- dirNum += saveChildren(children, out);
|
|
|
|
-
|
|
|
|
- // 3. Write INodeDirectorySnapshottable#snapshotsByNames to record all
|
|
|
|
|
|
+ // 2. Write INodeDirectorySnapshottable#snapshotsByNames to record all
|
|
// Snapshots
|
|
// Snapshots
|
|
if (current instanceof INodeDirectorySnapshottable) {
|
|
if (current instanceof INodeDirectorySnapshottable) {
|
|
INodeDirectorySnapshottable snapshottableNode =
|
|
INodeDirectorySnapshottable snapshottableNode =
|
|
@@ -904,14 +918,12 @@ public class FSImageFormat {
|
|
} else {
|
|
} else {
|
|
out.writeInt(-1); // # of snapshots
|
|
out.writeInt(-1); // # of snapshots
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // 3. Write children INode
|
|
|
|
+ dirNum += saveChildren(children, out);
|
|
|
|
|
|
- // 4. Write SnapshotDiff lists.
|
|
|
|
- if (current instanceof INodeDirectoryWithSnapshot) {
|
|
|
|
- INodeDirectoryWithSnapshot sNode = (INodeDirectoryWithSnapshot) current;
|
|
|
|
- SnapshotFSImageFormat.saveSnapshotDiffs(sNode, out);
|
|
|
|
- } else {
|
|
|
|
- out.writeInt(-1); // # of SnapshotDiffs
|
|
|
|
- }
|
|
|
|
|
|
+ // 4. Write DirectoryDiff lists, if there is any.
|
|
|
|
+ SnapshotFSImageFormat.saveDirectoryDiffList(current, out);
|
|
|
|
|
|
// Write sub-tree of sub-directories, including possible snapshots of
|
|
// Write sub-tree of sub-directories, including possible snapshots of
|
|
// deleted sub-directories
|
|
// deleted sub-directories
|