|
@@ -32,29 +32,37 @@ import java.security.MessageDigest;
|
|
|
import java.util.Arrays;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
+import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
-import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
|
|
|
/**
|
|
|
- * Contains inner classes for reading or writing the on-disk format for FSImages
|
|
|
+ * Contains inner classes for reading or writing the on-disk format for FSImages.
|
|
|
*/
|
|
|
-public abstract class FSImageFormat {
|
|
|
+@InterfaceAudience.Private
|
|
|
+@InterfaceStability.Evolving
|
|
|
+class FSImageFormat {
|
|
|
private static final Log LOG = FSImage.LOG;
|
|
|
|
|
|
+ // Static-only class
|
|
|
+ private FSImageFormat() {}
|
|
|
+
|
|
|
/**
|
|
|
* A one-shot class responsible for loading an image. The load() function
|
|
|
* 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.
|
|
|
*/
|
|
|
- public static class Loader {
|
|
|
+ static class Loader {
|
|
|
private final Configuration conf;
|
|
|
+ /** which namesystem this loader is working for */
|
|
|
+ private final FSNamesystem namesystem;
|
|
|
|
|
|
/** Set to true once a file has been loaded using this loader. */
|
|
|
private boolean loaded = false;
|
|
@@ -66,8 +74,9 @@ public abstract class FSImageFormat {
|
|
|
/** The MD5 sum of the loaded file */
|
|
|
private MD5Hash imgDigest;
|
|
|
|
|
|
- public Loader(Configuration conf) {
|
|
|
+ Loader(Configuration conf, FSNamesystem namesystem) {
|
|
|
this.conf = conf;
|
|
|
+ this.namesystem = namesystem;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -115,14 +124,13 @@ public abstract class FSImageFormat {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void load(File curFile, FSNamesystem targetNamesystem)
|
|
|
+ void load(File curFile)
|
|
|
throws IOException
|
|
|
{
|
|
|
checkNotLoaded();
|
|
|
assert curFile != null : "curFile is null";
|
|
|
|
|
|
long startTime = now();
|
|
|
- FSDirectory fsDir = targetNamesystem.dir;
|
|
|
|
|
|
//
|
|
|
// Load in bits
|
|
@@ -155,7 +163,7 @@ public abstract class FSImageFormat {
|
|
|
// read in the last generation stamp.
|
|
|
if (imgVersion <= -12) {
|
|
|
long genstamp = in.readLong();
|
|
|
- targetNamesystem.setGenerationStamp(genstamp);
|
|
|
+ namesystem.setGenerationStamp(genstamp);
|
|
|
}
|
|
|
|
|
|
// read compression related info
|
|
@@ -169,110 +177,21 @@ public abstract class FSImageFormat {
|
|
|
|
|
|
LOG.info("Loading image file " + curFile + " using " + compression);
|
|
|
|
|
|
-
|
|
|
- // read file info
|
|
|
- short replication = targetNamesystem.getDefaultReplication();
|
|
|
-
|
|
|
+ // load all inodes
|
|
|
LOG.info("Number of files = " + numFiles);
|
|
|
-
|
|
|
- byte[][] pathComponents;
|
|
|
- byte[][] parentPath = {{}};
|
|
|
- INodeDirectory parentINode = fsDir.rootDir;
|
|
|
- for (long i = 0; i < numFiles; i++) {
|
|
|
- long modificationTime = 0;
|
|
|
- long atime = 0;
|
|
|
- long blockSize = 0;
|
|
|
- pathComponents = FSImageSerialization.readPathComponents(in);
|
|
|
- replication = in.readShort();
|
|
|
- replication = targetNamesystem.adjustReplication(replication);
|
|
|
- modificationTime = in.readLong();
|
|
|
- if (imgVersion <= -17) {
|
|
|
- atime = in.readLong();
|
|
|
- }
|
|
|
- if (imgVersion <= -8) {
|
|
|
- blockSize = in.readLong();
|
|
|
- }
|
|
|
- int numBlocks = in.readInt();
|
|
|
- Block blocks[] = null;
|
|
|
-
|
|
|
- // for older versions, a blocklist of size 0
|
|
|
- // indicates a directory.
|
|
|
- if ((-9 <= imgVersion && numBlocks > 0) ||
|
|
|
- (imgVersion < -9 && numBlocks >= 0)) {
|
|
|
- blocks = new Block[numBlocks];
|
|
|
- for (int j = 0; j < numBlocks; j++) {
|
|
|
- blocks[j] = new Block();
|
|
|
- if (-14 < imgVersion) {
|
|
|
- blocks[j].set(in.readLong(), in.readLong(),
|
|
|
- GenerationStamp.GRANDFATHER_GENERATION_STAMP);
|
|
|
- } else {
|
|
|
- blocks[j].readFields(in);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- // Older versions of HDFS does not store the block size in inode.
|
|
|
- // If the file has more than one block, use the size of the
|
|
|
- // first block as the blocksize. Otherwise use the default block size.
|
|
|
- //
|
|
|
- if (-8 <= imgVersion && blockSize == 0) {
|
|
|
- if (numBlocks > 1) {
|
|
|
- blockSize = blocks[0].getNumBytes();
|
|
|
- } else {
|
|
|
- long first = ((numBlocks == 1) ? blocks[0].getNumBytes(): 0);
|
|
|
- blockSize = Math.max(targetNamesystem.getDefaultBlockSize(), first);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // get quota only when the node is a directory
|
|
|
- long nsQuota = -1L;
|
|
|
- if (imgVersion <= -16 && blocks == null && numBlocks == -1) {
|
|
|
- nsQuota = in.readLong();
|
|
|
- }
|
|
|
- long dsQuota = -1L;
|
|
|
- if (imgVersion <= -18 && blocks == null && numBlocks == -1) {
|
|
|
- dsQuota = in.readLong();
|
|
|
- }
|
|
|
-
|
|
|
- // Read the symlink only when the node is a symlink
|
|
|
- String symlink = "";
|
|
|
- if (imgVersion <= -23 && numBlocks == -2) {
|
|
|
- symlink = Text.readString(in);
|
|
|
- }
|
|
|
-
|
|
|
- PermissionStatus permissions = targetNamesystem.getUpgradePermission();
|
|
|
- if (imgVersion <= -11) {
|
|
|
- permissions = PermissionStatus.read(in);
|
|
|
- }
|
|
|
-
|
|
|
- if (isRoot(pathComponents)) { // it is the root
|
|
|
- // update the root's attributes
|
|
|
- if (nsQuota != -1 || dsQuota != -1) {
|
|
|
- fsDir.rootDir.setQuota(nsQuota, dsQuota);
|
|
|
- }
|
|
|
- fsDir.rootDir.setModificationTime(modificationTime);
|
|
|
- fsDir.rootDir.setPermissionStatus(permissions);
|
|
|
- continue;
|
|
|
- }
|
|
|
- // check if the new inode belongs to the same parent
|
|
|
- if(!isParent(pathComponents, parentPath)) {
|
|
|
- parentINode = null;
|
|
|
- parentPath = getParent(pathComponents);
|
|
|
- }
|
|
|
- // add new inode
|
|
|
- // without propagating modification time to parent
|
|
|
- parentINode = fsDir.addToParent(pathComponents, parentINode, permissions,
|
|
|
- blocks, symlink, replication, modificationTime,
|
|
|
- atime, nsQuota, dsQuota, blockSize, false);
|
|
|
- }
|
|
|
+ loadFullNameINodes(numFiles, in);
|
|
|
|
|
|
// load datanode info
|
|
|
this.loadDatanodes(in);
|
|
|
|
|
|
// load Files Under Construction
|
|
|
- this.loadFilesUnderConstruction(in, targetNamesystem);
|
|
|
+ this.loadFilesUnderConstruction(in);
|
|
|
|
|
|
- this.loadSecretManagerState(in, targetNamesystem);
|
|
|
+ this.loadSecretManagerState(in);
|
|
|
|
|
|
+ // make sure to read to the end of file
|
|
|
+ int eof = in.read();
|
|
|
+ assert eof == -1 : "Should have reached the end of image file " + curFile;
|
|
|
} finally {
|
|
|
in.close();
|
|
|
}
|
|
@@ -284,6 +203,128 @@ public abstract class FSImageFormat {
|
|
|
+ (now() - startTime)/1000 + " seconds.");
|
|
|
}
|
|
|
|
|
|
+ /** Update the root node's attributes */
|
|
|
+ private void updateRootAttr(INode root) {
|
|
|
+ long nsQuota = root.getNsQuota();
|
|
|
+ long dsQuota = root.getDsQuota();
|
|
|
+ FSDirectory fsDir = namesystem.dir;
|
|
|
+ if (nsQuota != -1 || dsQuota != -1) {
|
|
|
+ fsDir.rootDir.setQuota(nsQuota, dsQuota);
|
|
|
+ }
|
|
|
+ fsDir.rootDir.setModificationTime(root.getModificationTime());
|
|
|
+ fsDir.rootDir.setPermissionStatus(root.getPermissionStatus());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * load fsimage files assuming full path names are stored
|
|
|
+ *
|
|
|
+ * @param numFiles total number of files to load
|
|
|
+ * @param in data input stream
|
|
|
+ * @throws IOException if any error occurs
|
|
|
+ */
|
|
|
+ private void loadFullNameINodes(long numFiles,
|
|
|
+ DataInputStream in) throws IOException {
|
|
|
+ byte[][] pathComponents;
|
|
|
+ byte[][] parentPath = {{}};
|
|
|
+ FSDirectory fsDir = namesystem.dir;
|
|
|
+ INodeDirectory parentINode = fsDir.rootDir;
|
|
|
+ for (long i = 0; i < numFiles; i++) {
|
|
|
+ pathComponents = FSImageSerialization.readPathComponents(in);
|
|
|
+ INode newNode = loadINode(in);
|
|
|
+
|
|
|
+ if (isRoot(pathComponents)) { // it is the root
|
|
|
+ // update the root's attributes
|
|
|
+ updateRootAttr(newNode);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ // check if the new inode belongs to the same parent
|
|
|
+ if(!isParent(pathComponents, parentPath)) {
|
|
|
+ parentINode = fsDir.getParent(pathComponents);
|
|
|
+ parentPath = getParent(pathComponents);
|
|
|
+ }
|
|
|
+
|
|
|
+ // add new inode
|
|
|
+ parentINode = fsDir.addToParent(pathComponents[pathComponents.length-1],
|
|
|
+ parentINode, newNode, false);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * load an inode from fsimage except for its name
|
|
|
+ *
|
|
|
+ * @param in data input stream from which image is read
|
|
|
+ * @return an inode
|
|
|
+ */
|
|
|
+ private INode loadINode(DataInputStream in)
|
|
|
+ throws IOException {
|
|
|
+ long modificationTime = 0;
|
|
|
+ long atime = 0;
|
|
|
+ long blockSize = 0;
|
|
|
+
|
|
|
+ short replication = in.readShort();
|
|
|
+ replication = namesystem.adjustReplication(replication);
|
|
|
+ modificationTime = in.readLong();
|
|
|
+ if (imgVersion <= -17) {
|
|
|
+ atime = in.readLong();
|
|
|
+ }
|
|
|
+ if (imgVersion <= -8) {
|
|
|
+ blockSize = in.readLong();
|
|
|
+ }
|
|
|
+ int numBlocks = in.readInt();
|
|
|
+ BlockInfo blocks[] = null;
|
|
|
+
|
|
|
+ // for older versions, a blocklist of size 0
|
|
|
+ // indicates a directory.
|
|
|
+ if ((-9 <= imgVersion && numBlocks > 0) ||
|
|
|
+ (imgVersion < -9 && numBlocks >= 0)) {
|
|
|
+ blocks = new BlockInfo[numBlocks];
|
|
|
+ for (int j = 0; j < numBlocks; j++) {
|
|
|
+ blocks[j] = new BlockInfo(replication);
|
|
|
+ if (-14 < imgVersion) {
|
|
|
+ blocks[j].set(in.readLong(), in.readLong(),
|
|
|
+ GenerationStamp.GRANDFATHER_GENERATION_STAMP);
|
|
|
+ } else {
|
|
|
+ blocks[j].readFields(in);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // Older versions of HDFS does not store the block size in inode.
|
|
|
+ // If the file has more than one block, use the size of the
|
|
|
+ // first block as the blocksize. Otherwise use the default block size.
|
|
|
+ //
|
|
|
+ if (-8 <= imgVersion && blockSize == 0) {
|
|
|
+ if (numBlocks > 1) {
|
|
|
+ blockSize = blocks[0].getNumBytes();
|
|
|
+ } else {
|
|
|
+ long first = ((numBlocks == 1) ? blocks[0].getNumBytes(): 0);
|
|
|
+ blockSize = Math.max(namesystem.getDefaultBlockSize(), first);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // get quota only when the node is a directory
|
|
|
+ long nsQuota = -1L;
|
|
|
+ if (imgVersion <= -16 && blocks == null && numBlocks == -1) {
|
|
|
+ nsQuota = in.readLong();
|
|
|
+ }
|
|
|
+ long dsQuota = -1L;
|
|
|
+ if (imgVersion <= -18 && blocks == null && numBlocks == -1) {
|
|
|
+ dsQuota = in.readLong();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Read the symlink only when the node is a symlink
|
|
|
+ String symlink = "";
|
|
|
+ if (imgVersion <= -23 && numBlocks == -2) {
|
|
|
+ symlink = Text.readString(in);
|
|
|
+ }
|
|
|
+
|
|
|
+ PermissionStatus permissions = namesystem.getUpgradePermission();
|
|
|
+ if (imgVersion <= -11) {
|
|
|
+ permissions = PermissionStatus.read(in);
|
|
|
+ }
|
|
|
+
|
|
|
+ return INode.newINode(permissions, blocks, symlink, replication,
|
|
|
+ modificationTime, atime, nsQuota, dsQuota, blockSize);
|
|
|
+ }
|
|
|
|
|
|
private void loadDatanodes(DataInputStream in) throws IOException {
|
|
|
if (imgVersion > -3) // pre datanode image version
|
|
@@ -298,9 +339,9 @@ public abstract class FSImageFormat {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void loadFilesUnderConstruction(DataInputStream in,
|
|
|
- FSNamesystem fs) throws IOException {
|
|
|
- FSDirectory fsDir = fs.dir;
|
|
|
+ private void loadFilesUnderConstruction(DataInputStream in)
|
|
|
+ throws IOException {
|
|
|
+ FSDirectory fsDir = namesystem.dir;
|
|
|
if (imgVersion > -13) // pre lease image version
|
|
|
return;
|
|
|
int size = in.readInt();
|
|
@@ -322,18 +363,17 @@ public abstract class FSImageFormat {
|
|
|
}
|
|
|
INodeFile oldnode = (INodeFile) old;
|
|
|
fsDir.replaceNode(path, oldnode, cons);
|
|
|
- fs.leaseManager.addLease(cons.getClientName(), path);
|
|
|
+ namesystem.leaseManager.addLease(cons.getClientName(), path);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void loadSecretManagerState(DataInputStream in,
|
|
|
- FSNamesystem fs) throws IOException {
|
|
|
+ private void loadSecretManagerState(DataInputStream in) throws IOException {
|
|
|
if (imgVersion > -23) {
|
|
|
//SecretManagerState is not available.
|
|
|
//This must not happen if security is turned on.
|
|
|
return;
|
|
|
}
|
|
|
- fs.loadSecretManagerState(in);
|
|
|
+ namesystem.loadSecretManagerState(in);
|
|
|
}
|
|
|
|
|
|
|
|
@@ -384,42 +424,42 @@ public abstract class FSImageFormat {
|
|
|
* The write() function should be called once, after which the getter
|
|
|
* functions may be used to retrieve information about the file that was written.
|
|
|
*/
|
|
|
- static class Writer {
|
|
|
+ static class Saver {
|
|
|
/** Set to true once an image has been written */
|
|
|
- private boolean written = false;
|
|
|
+ private boolean saved = false;
|
|
|
|
|
|
/** The MD5 checksum of the file that was written */
|
|
|
- private MD5Hash writtenDigest;
|
|
|
+ private MD5Hash savedDigest;
|
|
|
|
|
|
static private final byte[] PATH_SEPARATOR = DFSUtil.string2Bytes(Path.SEPARATOR);
|
|
|
|
|
|
- /** @throws IllegalStateException if the instance has not yet written an image */
|
|
|
- private void checkWritten() {
|
|
|
- if (!written) {
|
|
|
- throw new IllegalStateException("FSImageWriter has not written an image");
|
|
|
+ /** @throws IllegalStateException if the instance has not yet saved an image */
|
|
|
+ private void checkSaved() {
|
|
|
+ if (!saved) {
|
|
|
+ throw new IllegalStateException("FSImageSaver has not saved an image");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** @throws IllegalStateException if the instance has already written an image */
|
|
|
- private void checkNotWritten() {
|
|
|
- if (written) {
|
|
|
- throw new IllegalStateException("FSImageWriter has already written an image");
|
|
|
+ /** @throws IllegalStateException if the instance has already saved an image */
|
|
|
+ private void checkNotSaved() {
|
|
|
+ if (saved) {
|
|
|
+ throw new IllegalStateException("FSImageSaver has already saved an image");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Return the MD5 checksum of the image file that was saved.
|
|
|
*/
|
|
|
- MD5Hash getWrittenDigest() {
|
|
|
- checkWritten();
|
|
|
- return writtenDigest;
|
|
|
+ MD5Hash getSavedDigest() {
|
|
|
+ checkSaved();
|
|
|
+ return savedDigest;
|
|
|
}
|
|
|
|
|
|
- void write(File newFile,
|
|
|
- FSNamesystem sourceNamesystem,
|
|
|
- FSImageCompression compression)
|
|
|
+ void save(File newFile,
|
|
|
+ FSNamesystem sourceNamesystem,
|
|
|
+ FSImageCompression compression)
|
|
|
throws IOException {
|
|
|
- checkNotWritten();
|
|
|
+ checkNotSaved();
|
|
|
|
|
|
FSDirectory fsDir = sourceNamesystem.dir;
|
|
|
long startTime = now();
|
|
@@ -458,9 +498,9 @@ public abstract class FSImageFormat {
|
|
|
out.close();
|
|
|
}
|
|
|
|
|
|
- written = true;
|
|
|
+ saved = true;
|
|
|
// set md5 of the saved image
|
|
|
- writtenDigest = new MD5Hash(digester.digest());
|
|
|
+ savedDigest = new MD5Hash(digester.digest());
|
|
|
|
|
|
LOG.info("Image file of size " + newFile.length() + " saved in "
|
|
|
+ (now() - startTime)/1000 + " seconds.");
|