|
@@ -21,10 +21,13 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.Iterator;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
@@ -33,28 +36,80 @@ 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.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.StringMap;
|
|
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.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.INodeReference.DstReference;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
|
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
|
|
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 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;
|
|
|
|
|
|
-final class FSImageFormatPBINode {
|
|
|
|
- private final static int USER_GROUP_STRID_MASK = (1 << 24) - 1;
|
|
|
|
|
|
+@InterfaceAudience.Private
|
|
|
|
+public final class FSImageFormatPBINode {
|
|
|
|
+ private final static long USER_GROUP_STRID_MASK = (1 << 24) - 1;
|
|
private final static int USER_STRID_OFFSET = 40;
|
|
private final static int USER_STRID_OFFSET = 40;
|
|
private final static int GROUP_STRID_OFFSET = 16;
|
|
private final static int GROUP_STRID_OFFSET = 16;
|
|
|
|
|
|
- final static class Loader {
|
|
|
|
- private PermissionStatus loadPermission(long id) {
|
|
|
|
|
|
+ public final static class Loader {
|
|
|
|
+ public static PermissionStatus loadPermission(long id,
|
|
|
|
+ final String[] stringTable) {
|
|
short perm = (short) (id & ((1 << GROUP_STRID_OFFSET) - 1));
|
|
short perm = (short) (id & ((1 << GROUP_STRID_OFFSET) - 1));
|
|
int gsid = (int) ((id >> GROUP_STRID_OFFSET) & USER_GROUP_STRID_MASK);
|
|
int gsid = (int) ((id >> GROUP_STRID_OFFSET) & USER_GROUP_STRID_MASK);
|
|
int usid = (int) ((id >> USER_STRID_OFFSET) & USER_GROUP_STRID_MASK);
|
|
int usid = (int) ((id >> USER_STRID_OFFSET) & USER_GROUP_STRID_MASK);
|
|
- return new PermissionStatus(parent.stringTable[usid],
|
|
|
|
- parent.stringTable[gsid], new FsPermission(perm));
|
|
|
|
|
|
+ return new PermissionStatus(stringTable[usid], stringTable[gsid],
|
|
|
|
+ new FsPermission(perm));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static INodeReference loadINodeReference(
|
|
|
|
+ INodeSection.INodeReference r, FSDirectory dir) throws IOException {
|
|
|
|
+ long referredId = r.getReferredId();
|
|
|
|
+ INode referred = dir.getInode(referredId);
|
|
|
|
+ WithCount withCount = (WithCount) referred.getParentReference();
|
|
|
|
+ if (withCount == null) {
|
|
|
|
+ withCount = new INodeReference.WithCount(null, referred);
|
|
|
|
+ }
|
|
|
|
+ final INodeReference ref;
|
|
|
|
+ if (r.hasDstSnapshotId()) { // DstReference
|
|
|
|
+ ref = new INodeReference.DstReference(null, withCount,
|
|
|
|
+ r.getDstSnapshotId());
|
|
|
|
+ } else {
|
|
|
|
+ ref = new INodeReference.WithName(null, withCount, r.getName()
|
|
|
|
+ .toByteArray(), r.getLastSnapshotId());
|
|
|
|
+ }
|
|
|
|
+ return ref;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static INodeDirectory loadINodeDirectory(INodeSection.INode n,
|
|
|
|
+ final String[] stringTable) {
|
|
|
|
+ assert n.getType() == INodeSection.INode.Type.DIRECTORY;
|
|
|
|
+ INodeSection.INodeDirectory d = n.getDirectory();
|
|
|
|
+
|
|
|
|
+ final PermissionStatus permissions = loadPermission(d.getPermission(),
|
|
|
|
+ stringTable);
|
|
|
|
+ final INodeDirectory dir = new INodeDirectory(n.getId(), n.getName()
|
|
|
|
+ .toByteArray(), permissions, d.getModificationTime());
|
|
|
|
+
|
|
|
|
+ final long nsQuota = d.getNsQuota(), dsQuota = d.getDsQuota();
|
|
|
|
+ if (nsQuota >= 0 || dsQuota >= 0) {
|
|
|
|
+ dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
|
|
|
|
+ }
|
|
|
|
+ return dir;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static void updateBlocksMap(INodeFile file, BlockManager bm) {
|
|
|
|
+ // Add file->block mapping
|
|
|
|
+ final BlockInfo[] blocks = file.getBlocks();
|
|
|
|
+ if (blocks != null) {
|
|
|
|
+ for (int i = 0; i < blocks.length; i++) {
|
|
|
|
+ file.setBlock(i, bm.addBlockCollection(blocks[i], file));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private final FSDirectory dir;
|
|
private final FSDirectory dir;
|
|
@@ -80,9 +135,20 @@ final class FSImageFormatPBINode {
|
|
INode child = dir.getInode(id);
|
|
INode child = dir.getInode(id);
|
|
addToParent(p, child);
|
|
addToParent(p, child);
|
|
}
|
|
}
|
|
|
|
+ for (int i = 0; i < e.getNumOfRef(); i++) {
|
|
|
|
+ INodeReference ref = loadINodeReference(in);
|
|
|
|
+ addToParent(p, ref);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private INodeReference loadINodeReference(InputStream in)
|
|
|
|
+ throws IOException {
|
|
|
|
+ INodeSection.INodeReference ref = INodeSection.INodeReference
|
|
|
|
+ .parseDelimitedFrom(in);
|
|
|
|
+ return loadINodeReference(ref, dir);
|
|
|
|
+ }
|
|
|
|
+
|
|
void loadINodeSection(InputStream in) throws IOException {
|
|
void loadINodeSection(InputStream in) throws IOException {
|
|
INodeSection s = INodeSection.parseDelimitedFrom(in);
|
|
INodeSection s = INodeSection.parseDelimitedFrom(in);
|
|
fsn.resetLastInodeId(s.getLastInodeId());
|
|
fsn.resetLastInodeId(s.getLastInodeId());
|
|
@@ -108,7 +174,7 @@ final class FSImageFormatPBINode {
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
// update the lease manager
|
|
// update the lease manager
|
|
- INodeFile file = fsn.dir.getInode(entry.getInodeId()).asFile();
|
|
|
|
|
|
+ INodeFile file = dir.getInode(entry.getInodeId()).asFile();
|
|
FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
|
|
FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
|
|
Preconditions.checkState(uc != null); // file must be under-construction
|
|
Preconditions.checkState(uc != null); // file must be under-construction
|
|
fsn.leaseManager.addLease(uc.getClientName(), entry.getFullPath());
|
|
fsn.leaseManager.addLease(uc.getClientName(), entry.getFullPath());
|
|
@@ -116,8 +182,7 @@ final class FSImageFormatPBINode {
|
|
}
|
|
}
|
|
|
|
|
|
private void addToParent(INodeDirectory parent, INode child) {
|
|
private void addToParent(INodeDirectory parent, INode child) {
|
|
- FSDirectory fsDir = fsn.dir;
|
|
|
|
- if (parent == fsDir.rootDir && FSDirectory.isReservedName(child)) {
|
|
|
|
|
|
+ if (parent == dir.rootDir && FSDirectory.isReservedName(child)) {
|
|
throw new HadoopIllegalArgumentException("File name \""
|
|
throw new HadoopIllegalArgumentException("File name \""
|
|
+ child.getLocalName() + "\" is reserved. Please "
|
|
+ child.getLocalName() + "\" is reserved. Please "
|
|
+ " change the name of the existing file or directory to another "
|
|
+ " change the name of the existing file or directory to another "
|
|
@@ -127,10 +192,10 @@ final class FSImageFormatPBINode {
|
|
if (!parent.addChild(child)) {
|
|
if (!parent.addChild(child)) {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- fsn.dir.cacheName(child);
|
|
|
|
|
|
+ dir.cacheName(child);
|
|
|
|
|
|
if (child.isFile()) {
|
|
if (child.isFile()) {
|
|
- updateBlocksMap(child.asFile());
|
|
|
|
|
|
+ updateBlocksMap(child.asFile(), fsn.getBlockManager());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -139,7 +204,7 @@ final class FSImageFormatPBINode {
|
|
case FILE:
|
|
case FILE:
|
|
return loadINodeFile(n);
|
|
return loadINodeFile(n);
|
|
case DIRECTORY:
|
|
case DIRECTORY:
|
|
- return loadINodeDirectory(n);
|
|
|
|
|
|
+ return loadINodeDirectory(n, parent.getStringTable());
|
|
case SYMLINK:
|
|
case SYMLINK:
|
|
return loadINodeSymlink(n);
|
|
return loadINodeSymlink(n);
|
|
default:
|
|
default:
|
|
@@ -148,21 +213,6 @@ final class FSImageFormatPBINode {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- private INodeDirectory loadINodeDirectory(INodeSection.INode n) {
|
|
|
|
- assert n.getType() == INodeSection.INode.Type.DIRECTORY;
|
|
|
|
- INodeSection.INodeDirectory d = n.getDirectory();
|
|
|
|
-
|
|
|
|
- final PermissionStatus permissions = loadPermission(d.getPermission());
|
|
|
|
- final INodeDirectory dir = new INodeDirectory(n.getId(), n.getName()
|
|
|
|
- .toByteArray(), permissions, d.getModificationTime());
|
|
|
|
-
|
|
|
|
- final long nsQuota = d.getNsQuota(), dsQuota = d.getDsQuota();
|
|
|
|
- if (nsQuota >= 0 || dsQuota >= 0) {
|
|
|
|
- dir.addDirectoryWithQuotaFeature(nsQuota, dsQuota);
|
|
|
|
- }
|
|
|
|
- return dir;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
private INodeFile loadINodeFile(INodeSection.INode n) {
|
|
private INodeFile loadINodeFile(INodeSection.INode n) {
|
|
assert n.getType() == INodeSection.INode.Type.FILE;
|
|
assert n.getType() == INodeSection.INode.Type.FILE;
|
|
INodeSection.INodeFile f = n.getFile();
|
|
INodeSection.INodeFile f = n.getFile();
|
|
@@ -173,7 +223,8 @@ 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(),
|
|
|
|
+ parent.getStringTable());
|
|
|
|
|
|
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(),
|
|
@@ -197,13 +248,14 @@ final class FSImageFormatPBINode {
|
|
private INodeSymlink loadINodeSymlink(INodeSection.INode n) {
|
|
private INodeSymlink loadINodeSymlink(INodeSection.INode n) {
|
|
assert n.getType() == INodeSection.INode.Type.SYMLINK;
|
|
assert n.getType() == INodeSection.INode.Type.SYMLINK;
|
|
INodeSection.INodeSymlink s = n.getSymlink();
|
|
INodeSection.INodeSymlink s = n.getSymlink();
|
|
- final PermissionStatus permissions = loadPermission(s.getPermission());
|
|
|
|
|
|
+ final PermissionStatus permissions = loadPermission(s.getPermission(),
|
|
|
|
+ parent.getStringTable());
|
|
return new INodeSymlink(n.getId(), n.getName().toByteArray(), permissions,
|
|
return new INodeSymlink(n.getId(), n.getName().toByteArray(), permissions,
|
|
0, 0, s.getTarget().toStringUtf8());
|
|
0, 0, s.getTarget().toStringUtf8());
|
|
}
|
|
}
|
|
|
|
|
|
private void loadRootINode(INodeSection.INode p) {
|
|
private void loadRootINode(INodeSection.INode p) {
|
|
- INodeDirectory root = loadINodeDirectory(p);
|
|
|
|
|
|
+ INodeDirectory root = loadINodeDirectory(p, parent.getStringTable());
|
|
final Quota.Counts q = root.getQuotaCounts();
|
|
final Quota.Counts q = root.getQuotaCounts();
|
|
final long nsQuota = q.get(Quota.NAMESPACE);
|
|
final long nsQuota = q.get(Quota.NAMESPACE);
|
|
final long dsQuota = q.get(Quota.DISKSPACE);
|
|
final long dsQuota = q.get(Quota.DISKSPACE);
|
|
@@ -213,20 +265,53 @@ final class FSImageFormatPBINode {
|
|
dir.rootDir.cloneModificationTime(root);
|
|
dir.rootDir.cloneModificationTime(root);
|
|
dir.rootDir.clonePermissionStatus(root);
|
|
dir.rootDir.clonePermissionStatus(root);
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
|
|
- private void updateBlocksMap(INodeFile file) {
|
|
|
|
- // Add file->block mapping
|
|
|
|
- final BlockInfo[] blocks = file.getBlocks();
|
|
|
|
- if (blocks != null) {
|
|
|
|
- final BlockManager bm = fsn.getBlockManager();
|
|
|
|
- for (int i = 0; i < blocks.length; i++) {
|
|
|
|
- file.setBlock(i, bm.addBlockCollection(blocks[i], file));
|
|
|
|
- }
|
|
|
|
|
|
+ public final static class Saver {
|
|
|
|
+ private static long buildPermissionStatus(INodeAttributes n,
|
|
|
|
+ final StringMap stringMap) {
|
|
|
|
+ long userId = stringMap.getStringId(n.getUserName());
|
|
|
|
+ long groupId = stringMap.getStringId(n.getGroupName());
|
|
|
|
+ return ((userId & USER_GROUP_STRID_MASK) << USER_STRID_OFFSET)
|
|
|
|
+ | ((groupId & USER_GROUP_STRID_MASK) << GROUP_STRID_OFFSET)
|
|
|
|
+ | n.getFsPermissionShort();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static INodeSection.INodeFile.Builder buildINodeFile(
|
|
|
|
+ INodeFileAttributes file, final StringMap stringMap) {
|
|
|
|
+ INodeSection.INodeFile.Builder b = INodeSection.INodeFile.newBuilder()
|
|
|
|
+ .setAccessTime(file.getAccessTime())
|
|
|
|
+ .setModificationTime(file.getModificationTime())
|
|
|
|
+ .setPermission(buildPermissionStatus(file, stringMap))
|
|
|
|
+ .setPreferredBlockSize(file.getPreferredBlockSize())
|
|
|
|
+ .setReplication(file.getFileReplication());
|
|
|
|
+ return b;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static INodeSection.INodeDirectory.Builder buildINodeDirectory(
|
|
|
|
+ INodeDirectoryAttributes dir, final StringMap stringMap) {
|
|
|
|
+ Quota.Counts quota = dir.getQuotaCounts();
|
|
|
|
+ INodeSection.INodeDirectory.Builder b = INodeSection.INodeDirectory
|
|
|
|
+ .newBuilder().setModificationTime(dir.getModificationTime())
|
|
|
|
+ .setNsQuota(quota.get(Quota.NAMESPACE))
|
|
|
|
+ .setDsQuota(quota.get(Quota.DISKSPACE))
|
|
|
|
+ .setPermission(buildPermissionStatus(dir, stringMap));
|
|
|
|
+ return b;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static INodeSection.INodeReference.Builder buildINodeReference(
|
|
|
|
+ INodeReference ref) throws IOException {
|
|
|
|
+ INodeSection.INodeReference.Builder rb = INodeSection.INodeReference
|
|
|
|
+ .newBuilder().setReferredId(ref.getId());
|
|
|
|
+ if (ref instanceof WithName) {
|
|
|
|
+ rb.setLastSnapshotId(((WithName) ref).getLastSnapshotId()).setName(
|
|
|
|
+ ByteString.copyFrom(ref.getLocalNameBytes()));
|
|
|
|
+ } else if (ref instanceof DstReference) {
|
|
|
|
+ rb.setDstSnapshotId(((DstReference) ref).getDstSnapshotId());
|
|
}
|
|
}
|
|
|
|
+ return rb;
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- final static class Saver {
|
|
|
|
private final FSNamesystem fsn;
|
|
private final FSNamesystem fsn;
|
|
private final FileSummary.Builder summary;
|
|
private final FileSummary.Builder summary;
|
|
private final FSImageFormatProtobuf.Saver parent;
|
|
private final FSImageFormatProtobuf.Saver parent;
|
|
@@ -238,20 +323,33 @@ final class FSImageFormatPBINode {
|
|
}
|
|
}
|
|
|
|
|
|
void serializeINodeDirectorySection(OutputStream out) throws IOException {
|
|
void serializeINodeDirectorySection(OutputStream out) throws IOException {
|
|
- for (INodeWithAdditionalFields n : fsn.dir.getINodeMap().getMap()) {
|
|
|
|
- if (!n.isDirectory())
|
|
|
|
|
|
+ Iterator<INodeWithAdditionalFields> iter = fsn.getFSDirectory()
|
|
|
|
+ .getINodeMap().getMapIterator();
|
|
|
|
+ while (iter.hasNext()) {
|
|
|
|
+ INodeWithAdditionalFields n = iter.next();
|
|
|
|
+ if (!n.isDirectory()) {
|
|
continue;
|
|
continue;
|
|
-
|
|
|
|
|
|
+ }
|
|
ReadOnlyList<INode> children = n.asDirectory().getChildrenList(
|
|
ReadOnlyList<INode> children = n.asDirectory().getChildrenList(
|
|
Snapshot.CURRENT_STATE_ID);
|
|
Snapshot.CURRENT_STATE_ID);
|
|
if (children.size() > 0) {
|
|
if (children.size() > 0) {
|
|
INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection.
|
|
INodeDirectorySection.DirEntry.Builder b = INodeDirectorySection.
|
|
DirEntry.newBuilder().setParent(n.getId());
|
|
DirEntry.newBuilder().setParent(n.getId());
|
|
|
|
+ List<INodeReference> refs = new ArrayList<INodeReference>();
|
|
for (INode inode : children) {
|
|
for (INode inode : children) {
|
|
- b.addChildren(inode.getId());
|
|
|
|
|
|
+ if (!inode.isReference()) {
|
|
|
|
+ b.addChildren(inode.getId());
|
|
|
|
+ } else {
|
|
|
|
+ refs.add(inode.asReference());
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ b.setNumOfRef(refs.size());
|
|
INodeDirectorySection.DirEntry e = b.build();
|
|
INodeDirectorySection.DirEntry e = b.build();
|
|
e.writeDelimitedTo(out);
|
|
e.writeDelimitedTo(out);
|
|
|
|
+ for (INodeReference ref : refs) {
|
|
|
|
+ INodeSection.INodeReference.Builder rb = buildINodeReference(ref);
|
|
|
|
+ rb.build().writeDelimitedTo(out);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
parent.commitSection(summary,
|
|
parent.commitSection(summary,
|
|
@@ -266,7 +364,9 @@ final class FSImageFormatPBINode {
|
|
INodeSection s = b.build();
|
|
INodeSection s = b.build();
|
|
s.writeDelimitedTo(out);
|
|
s.writeDelimitedTo(out);
|
|
|
|
|
|
- for (INodeWithAdditionalFields n : inodesMap.getMap()) {
|
|
|
|
|
|
+ Iterator<INodeWithAdditionalFields> iter = inodesMap.getMapIterator();
|
|
|
|
+ while (iter.hasNext()) {
|
|
|
|
+ INodeWithAdditionalFields n = iter.next();
|
|
save(out, n);
|
|
save(out, n);
|
|
}
|
|
}
|
|
parent.commitSection(summary, FSImageFormatProtobuf.SectionName.INODE);
|
|
parent.commitSection(summary, FSImageFormatProtobuf.SectionName.INODE);
|
|
@@ -286,14 +386,6 @@ final class FSImageFormatPBINode {
|
|
FSImageFormatProtobuf.SectionName.FILES_UNDERCONSTRUCTION);
|
|
FSImageFormatProtobuf.SectionName.FILES_UNDERCONSTRUCTION);
|
|
}
|
|
}
|
|
|
|
|
|
- private long buildPermissionStatus(INode n) {
|
|
|
|
- int userId = parent.getStringId(n.getUserName());
|
|
|
|
- int groupId = parent.getStringId(n.getGroupName());
|
|
|
|
- return ((userId & USER_GROUP_STRID_MASK) << USER_STRID_OFFSET)
|
|
|
|
- | ((groupId & USER_GROUP_STRID_MASK) << GROUP_STRID_OFFSET)
|
|
|
|
- | n.getFsPermissionShort();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
private void save(OutputStream out, INode n) throws IOException {
|
|
private void save(OutputStream out, INode n) throws IOException {
|
|
if (n.isDirectory()) {
|
|
if (n.isDirectory()) {
|
|
save(out, n.asDirectory());
|
|
save(out, n.asDirectory());
|
|
@@ -305,25 +397,16 @@ final class FSImageFormatPBINode {
|
|
}
|
|
}
|
|
|
|
|
|
private void save(OutputStream out, INodeDirectory n) throws IOException {
|
|
private void save(OutputStream out, INodeDirectory n) throws IOException {
|
|
- Quota.Counts quota = n.getQuotaCounts();
|
|
|
|
- INodeSection.INodeDirectory.Builder b = INodeSection.INodeDirectory
|
|
|
|
- .newBuilder().setModificationTime(n.getModificationTime())
|
|
|
|
- .setNsQuota(quota.get(Quota.NAMESPACE))
|
|
|
|
- .setDsQuota(quota.get(Quota.DISKSPACE))
|
|
|
|
- .setPermission(buildPermissionStatus(n));
|
|
|
|
-
|
|
|
|
|
|
+ INodeSection.INodeDirectory.Builder b = buildINodeDirectory(n,
|
|
|
|
+ parent.getStringMap());
|
|
INodeSection.INode r = buildINodeCommon(n)
|
|
INodeSection.INode r = buildINodeCommon(n)
|
|
.setType(INodeSection.INode.Type.DIRECTORY).setDirectory(b).build();
|
|
.setType(INodeSection.INode.Type.DIRECTORY).setDirectory(b).build();
|
|
r.writeDelimitedTo(out);
|
|
r.writeDelimitedTo(out);
|
|
}
|
|
}
|
|
|
|
|
|
private void save(OutputStream out, INodeFile n) throws IOException {
|
|
private void save(OutputStream out, INodeFile n) throws IOException {
|
|
- INodeSection.INodeFile.Builder b = INodeSection.INodeFile.newBuilder()
|
|
|
|
- .setAccessTime(n.getAccessTime())
|
|
|
|
- .setModificationTime(n.getModificationTime())
|
|
|
|
- .setPermission(buildPermissionStatus(n))
|
|
|
|
- .setPreferredBlockSize(n.getPreferredBlockSize())
|
|
|
|
- .setReplication(n.getFileReplication());
|
|
|
|
|
|
+ INodeSection.INodeFile.Builder b = buildINodeFile(n,
|
|
|
|
+ parent.getStringMap());
|
|
|
|
|
|
for (Block block : n.getBlocks()) {
|
|
for (Block block : n.getBlocks()) {
|
|
b.addBlocks(PBHelper.convert(block));
|
|
b.addBlocks(PBHelper.convert(block));
|
|
@@ -331,7 +414,8 @@ final class FSImageFormatPBINode {
|
|
|
|
|
|
FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
|
|
FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
|
|
if (uc != null) {
|
|
if (uc != null) {
|
|
- INodeSection.FileUnderConstructionFeature f = INodeSection.FileUnderConstructionFeature
|
|
|
|
|
|
+ INodeSection.FileUnderConstructionFeature f =
|
|
|
|
+ INodeSection.FileUnderConstructionFeature
|
|
.newBuilder().setClientName(uc.getClientName())
|
|
.newBuilder().setClientName(uc.getClientName())
|
|
.setClientMachine(uc.getClientMachine()).build();
|
|
.setClientMachine(uc.getClientMachine()).build();
|
|
b.setFileUC(f);
|
|
b.setFileUC(f);
|
|
@@ -344,7 +428,8 @@ final class FSImageFormatPBINode {
|
|
|
|
|
|
private void save(OutputStream out, INodeSymlink n) throws IOException {
|
|
private void save(OutputStream out, INodeSymlink n) throws IOException {
|
|
INodeSection.INodeSymlink.Builder b = INodeSection.INodeSymlink
|
|
INodeSection.INodeSymlink.Builder b = INodeSection.INodeSymlink
|
|
- .newBuilder().setPermission(buildPermissionStatus(n))
|
|
|
|
|
|
+ .newBuilder()
|
|
|
|
+ .setPermission(buildPermissionStatus(n, parent.getStringMap()))
|
|
.setTarget(ByteString.copyFrom(n.getSymlink()));
|
|
.setTarget(ByteString.copyFrom(n.getSymlink()));
|
|
INodeSection.INode r = buildINodeCommon(n)
|
|
INodeSection.INode r = buildINodeCommon(n)
|
|
.setType(INodeSection.INode.Type.SYMLINK).setSymlink(b).build();
|
|
.setType(INodeSection.INode.Type.SYMLINK).setSymlink(b).build();
|