|
@@ -32,6 +32,7 @@ import java.security.DigestOutputStream;
|
|
|
import java.security.MessageDigest;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
|
|
@@ -678,6 +680,12 @@ public class FSImageFormat {
|
|
|
if (underConstruction) {
|
|
|
clientName = FSImageSerialization.readString(in);
|
|
|
clientMachine = FSImageSerialization.readString(in);
|
|
|
+ // convert the last block to BlockUC
|
|
|
+ if (blocks != null && blocks.length > 0) {
|
|
|
+ BlockInfo lastBlk = blocks[blocks.length - 1];
|
|
|
+ blocks[blocks.length - 1] = new BlockInfoUnderConstruction(
|
|
|
+ lastBlk, replication);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -690,10 +698,15 @@ public class FSImageFormat {
|
|
|
}
|
|
|
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;
|
|
|
+ if (underConstruction) {
|
|
|
+ INodeFileUnderConstruction fileUC = new INodeFileUnderConstruction(
|
|
|
+ file, clientName, clientMachine, null);
|
|
|
+ return fileDiffs == null ? fileUC :
|
|
|
+ new INodeFileUnderConstructionWithSnapshot(fileUC, fileDiffs);
|
|
|
+ } else {
|
|
|
+ return fileDiffs == null ? file :
|
|
|
+ new INodeFileWithSnapshot(file, fileDiffs);
|
|
|
+ }
|
|
|
} else if (numBlocks == -1) {
|
|
|
//directory
|
|
|
|
|
@@ -821,8 +834,20 @@ public class FSImageFormat {
|
|
|
|
|
|
// verify that file exists in namespace
|
|
|
String path = cons.getLocalName();
|
|
|
- final INodesInPath iip = fsDir.getLastINodeInPath(path);
|
|
|
- INodeFile oldnode = INodeFile.valueOf(iip.getINode(0), path);
|
|
|
+ INodeFile oldnode = null;
|
|
|
+ boolean inSnapshot = false;
|
|
|
+ if (path != null && FSDirectory.isReservedName(path) &&
|
|
|
+ LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
|
|
|
+ // TODO: for HDFS-5428, we use reserved path for those INodeFileUC in
|
|
|
+ // snapshot. If we support INode ID in the layout version, we can use
|
|
|
+ // the inode id to find the oldnode.
|
|
|
+ oldnode = namesystem.dir.getInode(cons.getId()).asFile();
|
|
|
+ inSnapshot = true;
|
|
|
+ } else {
|
|
|
+ final INodesInPath iip = fsDir.getLastINodeInPath(path);
|
|
|
+ oldnode = INodeFile.valueOf(iip.getINode(0), path);
|
|
|
+ }
|
|
|
+
|
|
|
cons.setLocalName(oldnode.getLocalNameBytes());
|
|
|
INodeReference parentRef = oldnode.getParentReference();
|
|
|
if (parentRef != null) {
|
|
@@ -833,11 +858,23 @@ public class FSImageFormat {
|
|
|
|
|
|
if (oldnode instanceof INodeFileWithSnapshot) {
|
|
|
cons = new INodeFileUnderConstructionWithSnapshot(cons,
|
|
|
- ((INodeFileWithSnapshot)oldnode).getDiffs());
|
|
|
+ ((INodeFileWithSnapshot) oldnode).getDiffs());
|
|
|
}
|
|
|
|
|
|
- fsDir.replaceINodeFile(path, oldnode, cons);
|
|
|
- namesystem.leaseManager.addLease(cons.getClientName(), path);
|
|
|
+ if (!inSnapshot) {
|
|
|
+ fsDir.replaceINodeFile(path, oldnode, cons);
|
|
|
+ namesystem.leaseManager.addLease(cons.getClientName(), path);
|
|
|
+ } else {
|
|
|
+ if (parentRef != null) {
|
|
|
+ // replace oldnode with cons
|
|
|
+ parentRef.setReferredINode(cons);
|
|
|
+ } else {
|
|
|
+ // replace old node in its parent's children list and deleted list
|
|
|
+ oldnode.getParent().replaceChildFileInSnapshot(oldnode, cons);
|
|
|
+ namesystem.dir.addToInodeMap(cons);
|
|
|
+ updateBlocksMap(cons);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -916,6 +953,9 @@ public class FSImageFormat {
|
|
|
/** The MD5 checksum of the file that was written */
|
|
|
private MD5Hash savedDigest;
|
|
|
private final ReferenceMap referenceMap = new ReferenceMap();
|
|
|
+
|
|
|
+ private final Map<Long, INodeFileUnderConstruction> snapshotUCMap =
|
|
|
+ new HashMap<Long, INodeFileUnderConstruction>();
|
|
|
|
|
|
/** @throws IllegalStateException if the instance has not yet saved an image */
|
|
|
private void checkSaved() {
|
|
@@ -992,14 +1032,22 @@ public class FSImageFormat {
|
|
|
// save the root
|
|
|
saveINode2Image(fsDir.rootDir, out, false, referenceMap, counter);
|
|
|
// save the rest of the nodes
|
|
|
- saveImage(fsDir.rootDir, out, true, counter);
|
|
|
+ saveImage(fsDir.rootDir, out, true, false, counter);
|
|
|
prog.endStep(Phase.SAVING_CHECKPOINT, step);
|
|
|
// Now that the step is finished, set counter equal to total to adjust
|
|
|
// for possible under-counting due to reference inodes.
|
|
|
prog.setCount(Phase.SAVING_CHECKPOINT, step,
|
|
|
fsDir.rootDir.numItemsInTree());
|
|
|
// save files under construction
|
|
|
- sourceNamesystem.saveFilesUnderConstruction(out);
|
|
|
+ // TODO: for HDFS-5428, since we cannot break the compatibility of
|
|
|
+ // fsimage, we store part of the under-construction files that are only
|
|
|
+ // in snapshots in this "under-construction-file" section. As a
|
|
|
+ // temporary solution, we use "/.reserved/.inodes/<inodeid>" as their
|
|
|
+ // paths, so that when loading fsimage we do not put them into the lease
|
|
|
+ // map. In the future, we can remove this hack when we can bump the
|
|
|
+ // layout version.
|
|
|
+ sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap);
|
|
|
+
|
|
|
context.checkCancelled();
|
|
|
sourceNamesystem.saveSecretManagerState(out, sdPath);
|
|
|
context.checkCancelled();
|
|
@@ -1024,20 +1072,31 @@ public class FSImageFormat {
|
|
|
* Save children INodes.
|
|
|
* @param children The list of children INodes
|
|
|
* @param out The DataOutputStream to write
|
|
|
+ * @param inSnapshot Whether the parent directory or its ancestor is in
|
|
|
+ * the deleted list of some snapshot (caused by rename or
|
|
|
+ * deletion)
|
|
|
* @param counter Counter to increment for namenode startup progress
|
|
|
* @return Number of children that are directory
|
|
|
*/
|
|
|
- private int saveChildren(ReadOnlyList<INode> children, DataOutputStream out,
|
|
|
- Counter counter) throws IOException {
|
|
|
+ private int saveChildren(ReadOnlyList<INode> children,
|
|
|
+ DataOutputStream out, boolean inSnapshot, Counter counter)
|
|
|
+ throws IOException {
|
|
|
// Write normal children INode.
|
|
|
out.writeInt(children.size());
|
|
|
int dirNum = 0;
|
|
|
int i = 0;
|
|
|
for(INode child : children) {
|
|
|
// print all children first
|
|
|
+ // TODO: for HDFS-5428, we cannot change the format/content of fsimage
|
|
|
+ // here, thus even if the parent directory is in snapshot, we still
|
|
|
+ // do not handle INodeUC as those stored in deleted list
|
|
|
saveINode2Image(child, out, false, referenceMap, counter);
|
|
|
if (child.isDirectory()) {
|
|
|
dirNum++;
|
|
|
+ } else if (inSnapshot && child.isFile()
|
|
|
+ && child.asFile().isUnderConstruction()) {
|
|
|
+ this.snapshotUCMap.put(child.getId(),
|
|
|
+ (INodeFileUnderConstruction) child.asFile());
|
|
|
}
|
|
|
if (i++ % 50 == 0) {
|
|
|
context.checkCancelled();
|
|
@@ -1054,14 +1113,15 @@ public class FSImageFormat {
|
|
|
*
|
|
|
* @param current The current node
|
|
|
* @param out The DataoutputStream to write the image
|
|
|
- * @param snapshot The possible snapshot associated with the current node
|
|
|
* @param toSaveSubtree Whether or not to save the subtree to fsimage. For
|
|
|
* reference node, its subtree may already have been
|
|
|
* saved before.
|
|
|
+ * @param inSnapshot Whether the current directory is in snapshot
|
|
|
* @param counter Counter to increment for namenode startup progress
|
|
|
*/
|
|
|
private void saveImage(INodeDirectory current, DataOutputStream out,
|
|
|
- boolean toSaveSubtree, Counter counter) throws IOException {
|
|
|
+ boolean toSaveSubtree, boolean inSnapshot, Counter counter)
|
|
|
+ throws IOException {
|
|
|
// write the inode id of the directory
|
|
|
out.writeLong(current.getId());
|
|
|
|
|
@@ -1090,7 +1150,7 @@ public class FSImageFormat {
|
|
|
}
|
|
|
|
|
|
// 3. Write children INode
|
|
|
- dirNum += saveChildren(children, out, counter);
|
|
|
+ dirNum += saveChildren(children, out, inSnapshot, counter);
|
|
|
|
|
|
// 4. Write DirectoryDiff lists, if there is any.
|
|
|
SnapshotFSImageFormat.saveDirectoryDiffList(current, out, referenceMap);
|
|
@@ -1105,14 +1165,14 @@ public class FSImageFormat {
|
|
|
// make sure we only save the subtree under a reference node once
|
|
|
boolean toSave = child.isReference() ?
|
|
|
referenceMap.toProcessSubtree(child.getId()) : true;
|
|
|
- saveImage(child.asDirectory(), out, toSave, counter);
|
|
|
+ saveImage(child.asDirectory(), out, toSave, inSnapshot, counter);
|
|
|
}
|
|
|
if (snapshotDirs != null) {
|
|
|
for (INodeDirectory subDir : snapshotDirs) {
|
|
|
// make sure we only save the subtree under a reference node once
|
|
|
boolean toSave = subDir.getParentReference() != null ?
|
|
|
referenceMap.toProcessSubtree(subDir.getId()) : true;
|
|
|
- saveImage(subDir, out, toSave, counter);
|
|
|
+ saveImage(subDir, out, toSave, true, counter);
|
|
|
}
|
|
|
}
|
|
|
}
|