|
@@ -24,6 +24,9 @@ import com.google.protobuf.ByteString;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.Options;
|
|
|
+import org.apache.hadoop.fs.PathHandle;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
|
|
|
import org.apache.hadoop.hdfs.server.common.FileRegion;
|
|
@@ -31,6 +34,8 @@ import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
|
|
|
+
|
|
|
+import static org.apache.hadoop.hdfs.DFSUtil.LOG;
|
|
|
import static org.apache.hadoop.hdfs.DFSUtil.string2Bytes;
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.DEFAULT_NAMESPACE_QUOTA;
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.DEFAULT_STORAGE_SPACE_QUOTA;
|
|
@@ -46,11 +51,14 @@ public class TreePath {
|
|
|
private final long parentId;
|
|
|
private final FileStatus stat;
|
|
|
private final TreeWalk.TreeIterator i;
|
|
|
+ private final FileSystem fs;
|
|
|
|
|
|
- protected TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i) {
|
|
|
+ protected TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i,
|
|
|
+ FileSystem fs) {
|
|
|
this.i = i;
|
|
|
this.stat = stat;
|
|
|
this.parentId = parentId;
|
|
|
+ this.fs = fs;
|
|
|
}
|
|
|
|
|
|
public FileStatus getFileStatus() {
|
|
@@ -74,10 +82,9 @@ public class TreePath {
|
|
|
}
|
|
|
|
|
|
public INode toINode(UGIResolver ugi, BlockResolver blk,
|
|
|
- BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
|
|
|
- throws IOException {
|
|
|
+ BlockAliasMap.Writer<FileRegion> out) throws IOException {
|
|
|
if (stat.isFile()) {
|
|
|
- return toFile(ugi, blk, out, blockPoolID);
|
|
|
+ return toFile(ugi, blk, out);
|
|
|
} else if (stat.isDirectory()) {
|
|
|
return toDirectory(ugi);
|
|
|
} else if (stat.isSymlink()) {
|
|
@@ -103,16 +110,16 @@ public class TreePath {
|
|
|
return (int)(pId ^ (pId >>> 32));
|
|
|
}
|
|
|
|
|
|
- void writeBlock(long blockId, long offset, long length,
|
|
|
- long genStamp, String blockPoolID,
|
|
|
- BlockAliasMap.Writer<FileRegion> out) throws IOException {
|
|
|
+ void writeBlock(long blockId, long offset, long length, long genStamp,
|
|
|
+ PathHandle pathHandle, BlockAliasMap.Writer<FileRegion> out)
|
|
|
+ throws IOException {
|
|
|
FileStatus s = getFileStatus();
|
|
|
- out.store(new FileRegion(blockId, s.getPath(), offset, length, genStamp));
|
|
|
+ out.store(new FileRegion(blockId, s.getPath(), offset, length, genStamp,
|
|
|
+ (pathHandle != null ? pathHandle.toByteArray() : new byte[0])));
|
|
|
}
|
|
|
|
|
|
INode toFile(UGIResolver ugi, BlockResolver blk,
|
|
|
- BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
|
|
|
- throws IOException {
|
|
|
+ BlockAliasMap.Writer<FileRegion> out) throws IOException {
|
|
|
final FileStatus s = getFileStatus();
|
|
|
// TODO should this store resolver's user/group?
|
|
|
ugi.addUser(s.getOwner());
|
|
@@ -124,12 +131,23 @@ public class TreePath {
|
|
|
.setPreferredBlockSize(blk.preferredBlockSize(s))
|
|
|
.setPermission(ugi.resolve(s))
|
|
|
.setStoragePolicyID(HdfsConstants.PROVIDED_STORAGE_POLICY_ID);
|
|
|
+
|
|
|
+ // pathhandle allows match as long as the file matches exactly.
|
|
|
+ PathHandle pathHandle = null;
|
|
|
+ if (fs != null) {
|
|
|
+ try {
|
|
|
+ pathHandle = fs.getPathHandle(s, Options.HandleOpt.exact());
|
|
|
+ } catch (UnsupportedOperationException e) {
|
|
|
+ LOG.warn(
|
|
|
+ "Exact path handle not supported by filesystem " + fs.toString());
|
|
|
+ }
|
|
|
+ }
|
|
|
//TODO: storage policy should be configurable per path; use BlockResolver
|
|
|
long off = 0L;
|
|
|
for (BlockProto block : blk.resolve(s)) {
|
|
|
b.addBlocks(block);
|
|
|
writeBlock(block.getBlockId(), off, block.getNumBytes(),
|
|
|
- block.getGenStamp(), blockPoolID, out);
|
|
|
+ block.getGenStamp(), pathHandle, out);
|
|
|
off += block.getNumBytes();
|
|
|
}
|
|
|
INode.Builder ib = INode.newBuilder()
|