|
@@ -37,8 +37,10 @@ import java.util.Iterator;
|
|
import java.util.Properties;
|
|
import java.util.Properties;
|
|
import java.util.Random;
|
|
import java.util.Random;
|
|
import java.lang.Math;
|
|
import java.lang.Math;
|
|
|
|
+import java.nio.ByteBuffer;
|
|
|
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.dfs.FSConstants.CheckpointStates;
|
|
import org.apache.hadoop.dfs.FSConstants.CheckpointStates;
|
|
import org.apache.hadoop.dfs.FSConstants.StartupOption;
|
|
import org.apache.hadoop.dfs.FSConstants.StartupOption;
|
|
@@ -84,6 +86,13 @@ class FSImage extends Storage {
|
|
*/
|
|
*/
|
|
volatile private CheckpointStates ckptState = CheckpointStates.START;
|
|
volatile private CheckpointStates ckptState = CheckpointStates.START;
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Used for saving the image to disk
|
|
|
|
+ */
|
|
|
|
+ static private final FsPermission fileperm = new FsPermission((short)0);
|
|
|
|
+ static private final byte[] separator = INode.string2Bytes("/");
|
|
|
|
+ static private byte[] byteStore = null;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
*/
|
|
*/
|
|
FSImage() {
|
|
FSImage() {
|
|
@@ -785,7 +794,7 @@ class FSImage extends Storage {
|
|
/**
|
|
/**
|
|
* Save the contents of the FS image to the file.
|
|
* Save the contents of the FS image to the file.
|
|
*/
|
|
*/
|
|
- void saveFSImage(File newFile ) throws IOException {
|
|
|
|
|
|
+ void saveFSImage(File newFile) throws IOException {
|
|
FSNamesystem fsNamesys = FSNamesystem.getFSNamesystem();
|
|
FSNamesystem fsNamesys = FSNamesystem.getFSNamesystem();
|
|
FSDirectory fsDir = fsNamesys.dir;
|
|
FSDirectory fsDir = fsNamesys.dir;
|
|
//
|
|
//
|
|
@@ -799,8 +808,12 @@ class FSImage extends Storage {
|
|
out.writeInt(namespaceID);
|
|
out.writeInt(namespaceID);
|
|
out.writeInt(fsDir.rootDir.numItemsInTree() - 1);
|
|
out.writeInt(fsDir.rootDir.numItemsInTree() - 1);
|
|
out.writeLong(fsNamesys.getGenerationStamp());
|
|
out.writeLong(fsNamesys.getGenerationStamp());
|
|
- saveImage("", fsDir.rootDir, out);
|
|
|
|
|
|
+ byteStore = new byte[4*FSConstants.MAX_PATH_LENGTH];
|
|
|
|
+ ByteBuffer strbuf = ByteBuffer.wrap(byteStore);
|
|
|
|
+ saveImage(strbuf, 0, fsDir.rootDir, out);
|
|
fsNamesys.saveFilesUnderConstruction(out);
|
|
fsNamesys.saveFilesUnderConstruction(out);
|
|
|
|
+ byteStore = null;
|
|
|
|
+ strbuf = null;
|
|
} finally {
|
|
} finally {
|
|
out.close();
|
|
out.close();
|
|
}
|
|
}
|
|
@@ -875,13 +888,16 @@ class FSImage extends Storage {
|
|
/**
|
|
/**
|
|
* Save file tree image starting from the given root.
|
|
* Save file tree image starting from the given root.
|
|
*/
|
|
*/
|
|
- private static void saveImage(String parentPrefix,
|
|
|
|
|
|
+ private static void saveImage(ByteBuffer parentPrefix,
|
|
|
|
+ int prefixLength,
|
|
INode inode,
|
|
INode inode,
|
|
DataOutputStream out) throws IOException {
|
|
DataOutputStream out) throws IOException {
|
|
- String fullName = "";
|
|
|
|
|
|
+ int newPrefixLength = prefixLength;
|
|
if (inode.getParent() != null) {
|
|
if (inode.getParent() != null) {
|
|
- fullName = parentPrefix + "/" + inode.getLocalName();
|
|
|
|
- new UTF8(fullName).write(out);
|
|
|
|
|
|
+ parentPrefix.put(separator).put(inode.getLocalNameBytes());
|
|
|
|
+ newPrefixLength += separator.length + inode.getLocalNameBytes().length;
|
|
|
|
+ out.writeShort(newPrefixLength);
|
|
|
|
+ out.write(byteStore, 0, newPrefixLength);
|
|
if (!inode.isDirectory()) { // write file inode
|
|
if (!inode.isDirectory()) { // write file inode
|
|
INodeFile fileINode = (INodeFile)inode;
|
|
INodeFile fileINode = (INodeFile)inode;
|
|
out.writeShort(fileINode.getReplication());
|
|
out.writeShort(fileINode.getReplication());
|
|
@@ -891,7 +907,11 @@ class FSImage extends Storage {
|
|
out.writeInt(blocks.length);
|
|
out.writeInt(blocks.length);
|
|
for (Block blk : blocks)
|
|
for (Block blk : blocks)
|
|
blk.write(out);
|
|
blk.write(out);
|
|
- fileINode.getPermissionStatus().write(out);
|
|
|
|
|
|
+ fileperm.fromShort(fileINode.getFsPermissionShort());
|
|
|
|
+ PermissionStatus.write(out, fileINode.getUserName(),
|
|
|
|
+ fileINode.getGroupName(),
|
|
|
|
+ fileperm);
|
|
|
|
+ parentPrefix.position(prefixLength);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
// write directory inode
|
|
// write directory inode
|
|
@@ -899,11 +919,17 @@ class FSImage extends Storage {
|
|
out.writeLong(inode.getModificationTime());
|
|
out.writeLong(inode.getModificationTime());
|
|
out.writeLong(0); // preferred block size
|
|
out.writeLong(0); // preferred block size
|
|
out.writeInt(-1); // # of blocks
|
|
out.writeInt(-1); // # of blocks
|
|
- inode.getPermissionStatus().write(out);
|
|
|
|
|
|
+ fileperm.fromShort(inode.getFsPermissionShort());
|
|
|
|
+ PermissionStatus.write(out, inode.getUserName(),
|
|
|
|
+ inode.getGroupName(),
|
|
|
|
+ fileperm);
|
|
}
|
|
}
|
|
- for(INode child : ((INodeDirectory)inode).getChildren()) {
|
|
|
|
- saveImage(fullName, child, out);
|
|
|
|
|
|
+ if (((INodeDirectory)inode).getChildrenRaw() != null) {
|
|
|
|
+ for(INode child : ((INodeDirectory)inode).getChildren()) {
|
|
|
|
+ saveImage(parentPrefix, newPrefixLength, child, out);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ parentPrefix.position(prefixLength);
|
|
}
|
|
}
|
|
|
|
|
|
void loadDatanodes(int version, DataInputStream in) throws IOException {
|
|
void loadDatanodes(int version, DataInputStream in) throws IOException {
|