|
@@ -773,6 +773,13 @@ class FSImage extends Storage {
|
|
|
if (imgVersion <= -11) {
|
|
|
permissions = PermissionStatus.read(in);
|
|
|
}
|
|
|
+ // check if this is a root node
|
|
|
+ if (path.length() == 0) {
|
|
|
+ // update the root's attributes
|
|
|
+ fsDir.rootDir.setModificationTime(modificationTime);
|
|
|
+ fsDir.rootDir.setPermissionStatus(permissions);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
// check if the new inode belongs to the same parent
|
|
|
if(!isParent(path, parentPath)) {
|
|
|
parentINode = null;
|
|
@@ -840,10 +847,13 @@ class FSImage extends Storage {
|
|
|
try {
|
|
|
out.writeInt(FSConstants.LAYOUT_VERSION);
|
|
|
out.writeInt(namespaceID);
|
|
|
- out.writeInt(fsDir.rootDir.numItemsInTree() - 1);
|
|
|
+ out.writeInt(fsDir.rootDir.numItemsInTree());
|
|
|
out.writeLong(fsNamesys.getGenerationStamp());
|
|
|
byte[] byteStore = new byte[4*FSConstants.MAX_PATH_LENGTH];
|
|
|
ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
|
|
|
+ // save the root
|
|
|
+ saveINode2Image(strbuf, fsDir.rootDir, out);
|
|
|
+ // save the rest of the nodes
|
|
|
saveImage(strbuf, 0, fsDir.rootDir, out);
|
|
|
fsNamesys.saveFilesUnderConstruction(out);
|
|
|
strbuf = null;
|
|
@@ -921,6 +931,39 @@ class FSImage extends Storage {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /*
|
|
|
+ * Save one inode's attributes to the image.
|
|
|
+ */
|
|
|
+ private static void saveINode2Image(ByteBuffer name,
|
|
|
+ INode node,
|
|
|
+ DataOutputStream out) throws IOException {
|
|
|
+ int nameLen = name.position();
|
|
|
+ out.writeShort(nameLen);
|
|
|
+ out.write(name.array(), name.arrayOffset(), nameLen);
|
|
|
+ if (!node.isDirectory()) { // write file inode
|
|
|
+ INodeFile fileINode = (INodeFile)node;
|
|
|
+ out.writeShort(fileINode.getReplication());
|
|
|
+ out.writeLong(fileINode.getModificationTime());
|
|
|
+ out.writeLong(fileINode.getPreferredBlockSize());
|
|
|
+ Block[] blocks = fileINode.getBlocks();
|
|
|
+ out.writeInt(blocks.length);
|
|
|
+ for (Block blk : blocks)
|
|
|
+ blk.write(out);
|
|
|
+ FILE_PERM.fromShort(fileINode.getFsPermissionShort());
|
|
|
+ PermissionStatus.write(out, fileINode.getUserName(),
|
|
|
+ fileINode.getGroupName(),
|
|
|
+ FILE_PERM);
|
|
|
+ } else { // write directory inode
|
|
|
+ out.writeShort(0); // replication
|
|
|
+ out.writeLong(node.getModificationTime());
|
|
|
+ out.writeLong(0); // preferred block size
|
|
|
+ out.writeInt(-1); // # of blocks
|
|
|
+ FILE_PERM.fromShort(node.getFsPermissionShort());
|
|
|
+ PermissionStatus.write(out, node.getUserName(),
|
|
|
+ node.getGroupName(),
|
|
|
+ FILE_PERM);
|
|
|
+ }
|
|
|
+ }
|
|
|
/**
|
|
|
* Save file tree image starting from the given root.
|
|
|
* This is a recursive procedure, which first saves all children of
|
|
@@ -934,37 +977,10 @@ class FSImage extends Storage {
|
|
|
if (current.getChildrenRaw() == null)
|
|
|
return;
|
|
|
for(INode child : current.getChildren()) {
|
|
|
- // print all children first
|
|
|
+ // print all children first
|
|
|
parentPrefix.position(prefixLength);
|
|
|
parentPrefix.put(PATH_SEPARATOR).put(child.getLocalNameBytes());
|
|
|
- newPrefixLength = parentPrefix.position();
|
|
|
- out.writeShort(newPrefixLength);
|
|
|
- out.write(parentPrefix.array(), parentPrefix.arrayOffset(),
|
|
|
- newPrefixLength);
|
|
|
- if (!child.isDirectory()) { // write file inode
|
|
|
- INodeFile fileINode = (INodeFile)child;
|
|
|
- out.writeShort(fileINode.getReplication());
|
|
|
- out.writeLong(fileINode.getModificationTime());
|
|
|
- out.writeLong(fileINode.getPreferredBlockSize());
|
|
|
- Block[] blocks = fileINode.getBlocks();
|
|
|
- out.writeInt(blocks.length);
|
|
|
- for (Block blk : blocks)
|
|
|
- blk.write(out);
|
|
|
- FILE_PERM.fromShort(fileINode.getFsPermissionShort());
|
|
|
- PermissionStatus.write(out, fileINode.getUserName(),
|
|
|
- fileINode.getGroupName(),
|
|
|
- FILE_PERM);
|
|
|
- continue;
|
|
|
- }
|
|
|
- // write directory inode
|
|
|
- out.writeShort(0); // replication
|
|
|
- out.writeLong(child.getModificationTime());
|
|
|
- out.writeLong(0); // preferred block size
|
|
|
- out.writeInt(-1); // # of blocks
|
|
|
- FILE_PERM.fromShort(child.getFsPermissionShort());
|
|
|
- PermissionStatus.write(out, child.getUserName(),
|
|
|
- child.getGroupName(),
|
|
|
- FILE_PERM);
|
|
|
+ saveINode2Image(parentPrefix, child, out);
|
|
|
}
|
|
|
for(INode child : current.getChildren()) {
|
|
|
if(!child.isDirectory())
|