|
@@ -22,6 +22,7 @@ import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
@@ -30,12 +31,15 @@ import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
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.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.Permission;
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.Permission;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
|
|
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.protobuf.ByteString;
|
|
import com.google.protobuf.ByteString;
|
|
@@ -56,14 +60,13 @@ final class FSImageFormatPBINode {
|
|
}
|
|
}
|
|
|
|
|
|
void loadINodeDirectorySection(InputStream in) throws IOException {
|
|
void loadINodeDirectorySection(InputStream in) throws IOException {
|
|
- final INodeMap inodeMap = dir.getINodeMap();
|
|
|
|
while (true) {
|
|
while (true) {
|
|
INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
|
|
INodeDirectorySection.DirEntry e = INodeDirectorySection.DirEntry
|
|
.parseDelimitedFrom(in);
|
|
.parseDelimitedFrom(in);
|
|
-
|
|
|
|
- if (e == null)
|
|
|
|
|
|
+ // note that in is a LimitedInputStream
|
|
|
|
+ if (e == null) {
|
|
break;
|
|
break;
|
|
-
|
|
|
|
|
|
+ }
|
|
INodeDirectory p = dir.getInode(e.getParent()).asDirectory();
|
|
INodeDirectory p = dir.getInode(e.getParent()).asDirectory();
|
|
for (long id : e.getChildrenList()) {
|
|
for (long id : e.getChildrenList()) {
|
|
INode child = dir.getInode(id);
|
|
INode child = dir.getInode(id);
|
|
@@ -86,6 +89,24 @@ final class FSImageFormatPBINode {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Load the under-construction files section, and update the lease map
|
|
|
|
+ */
|
|
|
|
+ void loadFilesUnderConstructionSection(InputStream in) throws IOException {
|
|
|
|
+ while (true) {
|
|
|
|
+ FileUnderConstructionEntry entry = FileUnderConstructionEntry
|
|
|
|
+ .parseDelimitedFrom(in);
|
|
|
|
+ if (entry == null) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ // update the lease manager
|
|
|
|
+ INodeFile file = fsn.dir.getInode(entry.getInodeId()).asFile();
|
|
|
|
+ FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
|
|
|
|
+ Preconditions.checkState(uc != null); // file must be under-construction
|
|
|
|
+ fsn.leaseManager.addLease(uc.getClientName(), entry.getFullPath());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
private void addToParent(INodeDirectory parent, INode child) {
|
|
private void addToParent(INodeDirectory parent, INode child) {
|
|
FSDirectory fsDir = fsn.dir;
|
|
FSDirectory fsDir = fsn.dir;
|
|
if (parent == fsDir.rootDir && FSDirectory.isReservedName(child)) {
|
|
if (parent == fsDir.rootDir && FSDirectory.isReservedName(child)) {
|
|
@@ -142,11 +163,23 @@ final class FSImageFormatPBINode {
|
|
for (int i = 0, e = bp.size(); i < e; ++i) {
|
|
for (int i = 0, e = bp.size(); i < e; ++i) {
|
|
blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
|
|
blocks[i] = new BlockInfo(PBHelper.convert(bp.get(i)), replication);
|
|
}
|
|
}
|
|
-
|
|
|
|
final PermissionStatus permissions = loadPermission(f.getPermission());
|
|
final PermissionStatus permissions = loadPermission(f.getPermission());
|
|
|
|
+
|
|
final INodeFile file = new INodeFile(n.getId(),
|
|
final INodeFile file = new INodeFile(n.getId(),
|
|
n.getName().toByteArray(), permissions, f.getModificationTime(),
|
|
n.getName().toByteArray(), permissions, f.getModificationTime(),
|
|
f.getAccessTime(), blocks, replication, f.getPreferredBlockSize());
|
|
f.getAccessTime(), blocks, replication, f.getPreferredBlockSize());
|
|
|
|
+ // under-construction information
|
|
|
|
+ if (f.hasFileUC()) {
|
|
|
|
+ INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
|
|
|
|
+ file.toUnderConstruction(uc.getClientName(), uc.getClientMachine(),
|
|
|
|
+ null);
|
|
|
|
+ if (blocks.length > 0) {
|
|
|
|
+ BlockInfo lastBlk = file.getLastBlock();
|
|
|
|
+ // replace the last block of file
|
|
|
|
+ file.setBlock(file.numBlocks() - 1, new BlockInfoUnderConstruction(
|
|
|
|
+ lastBlk, replication));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
return file;
|
|
return file;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -193,15 +226,14 @@ final class FSImageFormatPBINode {
|
|
if (!n.isDirectory())
|
|
if (!n.isDirectory())
|
|
continue;
|
|
continue;
|
|
|
|
|
|
- INodeDirectory d = n.asDirectory();
|
|
|
|
-
|
|
|
|
- INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection.DirEntry
|
|
|
|
- .newBuilder().setParent(n.getId());
|
|
|
|
-
|
|
|
|
- for (INode inode : d.getChildrenList(Snapshot.CURRENT_STATE_ID))
|
|
|
|
- b.addChildren(inode.getId());
|
|
|
|
-
|
|
|
|
- if (b.getChildrenCount() != 0) {
|
|
|
|
|
|
+ ReadOnlyList<INode> children = n.asDirectory().getChildrenList(
|
|
|
|
+ Snapshot.CURRENT_STATE_ID);
|
|
|
|
+ if (children.size() > 0) {
|
|
|
|
+ INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection.
|
|
|
|
+ DirEntry.newBuilder().setParent(n.getId());
|
|
|
|
+ for (INode inode : children) {
|
|
|
|
+ b.addChildren(inode.getId());
|
|
|
|
+ }
|
|
INodeDirectorySection.DirEntry e = b.build();
|
|
INodeDirectorySection.DirEntry e = b.build();
|
|
e.writeDelimitedTo(out);
|
|
e.writeDelimitedTo(out);
|
|
}
|
|
}
|
|
@@ -211,23 +243,32 @@ final class FSImageFormatPBINode {
|
|
}
|
|
}
|
|
|
|
|
|
void serializeINodeSection() throws IOException {
|
|
void serializeINodeSection() throws IOException {
|
|
- final INodeDirectory rootDir = fsn.dir.rootDir;
|
|
|
|
- final long numINodes = rootDir.getDirectoryWithQuotaFeature()
|
|
|
|
- .getSpaceConsumed().get(Quota.NAMESPACE);
|
|
|
|
|
|
+ INodeMap inodesMap = fsn.dir.getINodeMap();
|
|
INodeSection.Builder b = INodeSection.newBuilder()
|
|
INodeSection.Builder b = INodeSection.newBuilder()
|
|
- .setLastInodeId(fsn.getLastInodeId()).setNumInodes(numINodes);
|
|
|
|
|
|
+ .setLastInodeId(fsn.getLastInodeId()).setNumInodes(inodesMap.size());
|
|
INodeSection s = b.build();
|
|
INodeSection s = b.build();
|
|
s.writeDelimitedTo(out);
|
|
s.writeDelimitedTo(out);
|
|
|
|
|
|
- long i = 0;
|
|
|
|
- for (INodeWithAdditionalFields n : fsn.dir.getINodeMap().getMap()) {
|
|
|
|
|
|
+ for (INodeWithAdditionalFields n : inodesMap.getMap()) {
|
|
save(n);
|
|
save(n);
|
|
- ++i;
|
|
|
|
}
|
|
}
|
|
- Preconditions.checkState(numINodes == i);
|
|
|
|
parent.commitSection(headers, FSImageFormatProtobuf.SectionName.INODE);
|
|
parent.commitSection(headers, FSImageFormatProtobuf.SectionName.INODE);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ void serializeFilesUCSection() throws IOException {
|
|
|
|
+ Map<String, INodeFile> ucMap = fsn.getFilesUnderConstruction();
|
|
|
|
+ for (Map.Entry<String, INodeFile> entry : ucMap.entrySet()) {
|
|
|
|
+ String path = entry.getKey();
|
|
|
|
+ INodeFile file = entry.getValue();
|
|
|
|
+ FileUnderConstructionEntry.Builder b = FileUnderConstructionEntry
|
|
|
|
+ .newBuilder().setInodeId(file.getId()).setFullPath(path);
|
|
|
|
+ FileUnderConstructionEntry e = b.build();
|
|
|
|
+ e.writeDelimitedTo(out);
|
|
|
|
+ }
|
|
|
|
+ parent.commitSection(headers,
|
|
|
|
+ FSImageFormatProtobuf.SectionName.FILES_UNDERCONSTRUCTION);
|
|
|
|
+ }
|
|
|
|
+
|
|
private INodeSection.Permission.Builder buildPermissionStatus(INode n) {
|
|
private INodeSection.Permission.Builder buildPermissionStatus(INode n) {
|
|
return INodeSection.Permission.newBuilder().setUser(n.getUserName())
|
|
return INodeSection.Permission.newBuilder().setUser(n.getUserName())
|
|
.setGroup(n.getGroupName()).setPermission(n.getFsPermissionShort());
|
|
.setGroup(n.getGroupName()).setPermission(n.getFsPermissionShort());
|
|
@@ -263,8 +304,17 @@ final class FSImageFormatPBINode {
|
|
.setPreferredBlockSize(n.getPreferredBlockSize())
|
|
.setPreferredBlockSize(n.getPreferredBlockSize())
|
|
.setReplication(n.getFileReplication());
|
|
.setReplication(n.getFileReplication());
|
|
|
|
|
|
- for (Block block : n.getBlocks())
|
|
|
|
|
|
+ for (Block block : n.getBlocks()) {
|
|
b.addBlocks(PBHelper.convert(block));
|
|
b.addBlocks(PBHelper.convert(block));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
|
|
|
|
+ if (uc != null) {
|
|
|
|
+ INodeSection.FileUnderConstructionFeature f = INodeSection.FileUnderConstructionFeature
|
|
|
|
+ .newBuilder().setClientName(uc.getClientName())
|
|
|
|
+ .setClientMachine(uc.getClientMachine()).build();
|
|
|
|
+ b.setFileUC(f);
|
|
|
|
+ }
|
|
|
|
|
|
INodeSection.INode r = INodeSection.INode.newBuilder()
|
|
INodeSection.INode r = INodeSection.INode.newBuilder()
|
|
.setType(INodeSection.INode.Type.FILE).setId(n.getId())
|
|
.setType(INodeSection.INode.Type.FILE).setId(n.getId())
|