|
@@ -24,7 +24,6 @@ import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
|
|
|
import org.apache.hadoop.fs.FileEncryptionInfo;
|
|
|
import org.apache.hadoop.fs.InvalidPathException;
|
|
|
-import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
@@ -180,10 +179,14 @@ class FSDirStatAndListingOp {
|
|
|
.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
|
|
|
if (!targetNode.isDirectory()) {
|
|
|
+ INodeAttributes nodeAttrs = getINodeAttributes(
|
|
|
+ fsd, src, HdfsFileStatus.EMPTY_NAME, targetNode,
|
|
|
+ snapshot);
|
|
|
return new DirectoryListing(
|
|
|
- new HdfsFileStatus[]{createFileStatus(fsd, src,
|
|
|
- HdfsFileStatus.EMPTY_NAME, targetNode, needLocation,
|
|
|
- parentStoragePolicy, snapshot, isRawPath, iip)}, 0);
|
|
|
+ new HdfsFileStatus[]{ createFileStatus(
|
|
|
+ fsd, HdfsFileStatus.EMPTY_NAME, targetNode, nodeAttrs,
|
|
|
+ needLocation, parentStoragePolicy, snapshot, isRawPath, iip)
|
|
|
+ }, 0);
|
|
|
}
|
|
|
|
|
|
final INodeDirectory dirInode = targetNode.asDirectory();
|
|
@@ -200,8 +203,11 @@ class FSDirStatAndListingOp {
|
|
|
byte curPolicy = isSuperUser && !cur.isSymlink()?
|
|
|
cur.getLocalStoragePolicyID():
|
|
|
HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
- listing[i] = createFileStatus(fsd, src, cur.getLocalNameBytes(), cur,
|
|
|
- needLocation, getStoragePolicyID(curPolicy,
|
|
|
+ INodeAttributes nodeAttrs = getINodeAttributes(
|
|
|
+ fsd, src, cur.getLocalNameBytes(), cur,
|
|
|
+ snapshot);
|
|
|
+ listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(),
|
|
|
+ cur, nodeAttrs, needLocation, getStoragePolicyID(curPolicy,
|
|
|
parentStoragePolicy), snapshot, isRawPath, iip);
|
|
|
listingCnt++;
|
|
|
if (needLocation) {
|
|
@@ -253,9 +259,15 @@ class FSDirStatAndListingOp {
|
|
|
final HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
|
|
|
for (int i = 0; i < numOfListing; i++) {
|
|
|
Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
|
|
|
- listing[i] = createFileStatus(fsd, src, sRoot.getLocalNameBytes(), sRoot,
|
|
|
- HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
|
|
|
- false, INodesInPath.fromINode(sRoot));
|
|
|
+ INodeAttributes nodeAttrs = getINodeAttributes(
|
|
|
+ fsd, src, sRoot.getLocalNameBytes(),
|
|
|
+ node, Snapshot.CURRENT_STATE_ID);
|
|
|
+ listing[i] = createFileStatus(
|
|
|
+ fsd, sRoot.getLocalNameBytes(),
|
|
|
+ sRoot, nodeAttrs,
|
|
|
+ HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
|
|
|
+ Snapshot.CURRENT_STATE_ID, false,
|
|
|
+ INodesInPath.fromINode(sRoot));
|
|
|
}
|
|
|
return new DirectoryListing(
|
|
|
listing, snapshots.size() - skipSize - numOfListing);
|
|
@@ -276,11 +288,20 @@ class FSDirStatAndListingOp {
|
|
|
fsd.readLock();
|
|
|
try {
|
|
|
final INode i = src.getLastINode();
|
|
|
- byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
|
|
|
- i.getStoragePolicyID() : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
- return i == null ? null : createFileStatus(
|
|
|
- fsd, path, HdfsFileStatus.EMPTY_NAME, i, policyId,
|
|
|
- src.getPathSnapshotId(), isRawPath, src);
|
|
|
+ if (i == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ byte policyId = includeStoragePolicy && !i.isSymlink() ?
|
|
|
+ i.getStoragePolicyID() : HdfsConstantsClient
|
|
|
+ .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
+ INodeAttributes nodeAttrs = getINodeAttributes(
|
|
|
+ fsd, path, HdfsFileStatus.EMPTY_NAME, i, src.getPathSnapshotId());
|
|
|
+ return createFileStatus(
|
|
|
+ fsd, HdfsFileStatus.EMPTY_NAME,
|
|
|
+ i, nodeAttrs, policyId,
|
|
|
+ src.getPathSnapshotId(),
|
|
|
+ isRawPath, src);
|
|
|
} finally {
|
|
|
fsd.readUnlock();
|
|
|
}
|
|
@@ -309,23 +330,6 @@ class FSDirStatAndListingOp {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Currently we only support "ls /xxx/.snapshot" which will return all the
|
|
|
- * snapshots of a directory. The FSCommand Ls will first call getFileInfo to
|
|
|
- * make sure the file/directory exists (before the real getListing call).
|
|
|
- * Since we do not have a real INode for ".snapshot", we return an empty
|
|
|
- * non-null HdfsFileStatus here.
|
|
|
- */
|
|
|
- private static HdfsFileStatus getFileInfo4DotSnapshot(
|
|
|
- FSDirectory fsd, String src)
|
|
|
- throws UnresolvedLinkException {
|
|
|
- if (fsd.getINode4DotSnapshot(src) != null) {
|
|
|
- return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
|
|
|
- HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
|
|
|
- HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
|
|
|
- }
|
|
|
- return null;
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* create an hdfs file status from an inode
|
|
@@ -339,52 +343,63 @@ class FSDirStatAndListingOp {
|
|
|
* @return a file status
|
|
|
* @throws java.io.IOException if any error occurs
|
|
|
*/
|
|
|
- static HdfsFileStatus createFileStatus(
|
|
|
- FSDirectory fsd, String fullPath, byte[] path, INode node,
|
|
|
+ private static HdfsFileStatus createFileStatus(
|
|
|
+ FSDirectory fsd, byte[] path, INode node, INodeAttributes nodeAttrs,
|
|
|
boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath,
|
|
|
INodesInPath iip)
|
|
|
throws IOException {
|
|
|
if (needLocation) {
|
|
|
- return createLocatedFileStatus(fsd, fullPath, path, node, storagePolicy,
|
|
|
- snapshot, isRawPath, iip);
|
|
|
+ return createLocatedFileStatus(fsd, path, node, nodeAttrs, storagePolicy,
|
|
|
+ snapshot, isRawPath, iip);
|
|
|
} else {
|
|
|
- return createFileStatus(fsd, fullPath, path, node, storagePolicy, snapshot,
|
|
|
- isRawPath, iip);
|
|
|
+ return createFileStatus(fsd, path, node, nodeAttrs, storagePolicy,
|
|
|
+ snapshot, isRawPath, iip);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Create FileStatus by file INode
|
|
|
*/
|
|
|
- static HdfsFileStatus createFileStatus(
|
|
|
+ static HdfsFileStatus createFileStatusForEditLog(
|
|
|
FSDirectory fsd, String fullPath, byte[] path, INode node,
|
|
|
byte storagePolicy, int snapshot, boolean isRawPath,
|
|
|
INodesInPath iip) throws IOException {
|
|
|
- long size = 0; // length is zero for directories
|
|
|
- short replication = 0;
|
|
|
- long blocksize = 0;
|
|
|
- final boolean isEncrypted;
|
|
|
-
|
|
|
- final FileEncryptionInfo feInfo = isRawPath ? null :
|
|
|
- fsd.getFileEncryptionInfo(node, snapshot, iip);
|
|
|
-
|
|
|
- if (node.isFile()) {
|
|
|
- final INodeFile fileNode = node.asFile();
|
|
|
- size = fileNode.computeFileSize(snapshot);
|
|
|
- replication = fileNode.getFileReplication(snapshot);
|
|
|
- blocksize = fileNode.getPreferredBlockSize();
|
|
|
- isEncrypted = (feInfo != null) ||
|
|
|
- (isRawPath && fsd.isInAnEZ(INodesInPath.fromINode(node)));
|
|
|
- } else {
|
|
|
- isEncrypted = fsd.isInAnEZ(INodesInPath.fromINode(node));
|
|
|
- }
|
|
|
-
|
|
|
- int childrenNum = node.isDirectory() ?
|
|
|
- node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
|
-
|
|
|
- INodeAttributes nodeAttrs =
|
|
|
- fsd.getAttributes(fullPath, path, node, snapshot);
|
|
|
- return new HdfsFileStatus(
|
|
|
+ INodeAttributes nodeAttrs = getINodeAttributes(
|
|
|
+ fsd, fullPath, path, node, snapshot);
|
|
|
+ return createFileStatus(fsd, path, node, nodeAttrs,
|
|
|
+ storagePolicy, snapshot, isRawPath, iip);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create FileStatus by file INode
|
|
|
+ */
|
|
|
+ static HdfsFileStatus createFileStatus(
|
|
|
+ FSDirectory fsd, byte[] path, INode node,
|
|
|
+ INodeAttributes nodeAttrs, byte storagePolicy, int snapshot,
|
|
|
+ boolean isRawPath, INodesInPath iip) throws IOException {
|
|
|
+ long size = 0; // length is zero for directories
|
|
|
+ short replication = 0;
|
|
|
+ long blocksize = 0;
|
|
|
+ final boolean isEncrypted;
|
|
|
+
|
|
|
+ final FileEncryptionInfo feInfo = isRawPath ? null :
|
|
|
+ fsd.getFileEncryptionInfo(node, snapshot, iip);
|
|
|
+
|
|
|
+ if (node.isFile()) {
|
|
|
+ final INodeFile fileNode = node.asFile();
|
|
|
+ size = fileNode.computeFileSize(snapshot);
|
|
|
+ replication = fileNode.getFileReplication(snapshot);
|
|
|
+ blocksize = fileNode.getPreferredBlockSize();
|
|
|
+ isEncrypted = (feInfo != null) ||
|
|
|
+ (isRawPath && fsd.isInAnEZ(INodesInPath.fromINode(node)));
|
|
|
+ } else {
|
|
|
+ isEncrypted = fsd.isInAnEZ(INodesInPath.fromINode(node));
|
|
|
+ }
|
|
|
+
|
|
|
+ int childrenNum = node.isDirectory() ?
|
|
|
+ node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
|
+
|
|
|
+ return new HdfsFileStatus(
|
|
|
size,
|
|
|
node.isDirectory(),
|
|
|
replication,
|
|
@@ -402,13 +417,18 @@ class FSDirStatAndListingOp {
|
|
|
storagePolicy);
|
|
|
}
|
|
|
|
|
|
+ private static INodeAttributes getINodeAttributes(
|
|
|
+ FSDirectory fsd, String fullPath, byte[] path, INode node, int snapshot) {
|
|
|
+ return fsd.getAttributes(fullPath, path, node, snapshot);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Create FileStatus with location info by file INode
|
|
|
*/
|
|
|
private static HdfsLocatedFileStatus createLocatedFileStatus(
|
|
|
- FSDirectory fsd, String fullPath, byte[] path, INode node,
|
|
|
- byte storagePolicy, int snapshot, boolean isRawPath,
|
|
|
- INodesInPath iip) throws IOException {
|
|
|
+ FSDirectory fsd, byte[] path, INode node, INodeAttributes nodeAttrs,
|
|
|
+ byte storagePolicy, int snapshot,
|
|
|
+ boolean isRawPath, INodesInPath iip) throws IOException {
|
|
|
assert fsd.hasReadLock();
|
|
|
long size = 0; // length is zero for directories
|
|
|
short replication = 0;
|
|
@@ -442,8 +462,6 @@ class FSDirStatAndListingOp {
|
|
|
int childrenNum = node.isDirectory() ?
|
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
|
|
|
|
- INodeAttributes nodeAttrs =
|
|
|
- fsd.getAttributes(fullPath, path, node, snapshot);
|
|
|
HdfsLocatedFileStatus status =
|
|
|
new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
|
|
blocksize, node.getModificationTime(snapshot),
|
|
@@ -468,7 +486,6 @@ class FSDirStatAndListingOp {
|
|
|
* return an FsPermissionExtension.
|
|
|
*
|
|
|
* @param node INode to check
|
|
|
- * @param snapshot int snapshot ID
|
|
|
* @param isEncrypted boolean true if the file/dir is encrypted
|
|
|
* @return FsPermission from inode, with ACL bit on if the inode has an ACL
|
|
|
* and encrypted bit on if it represents an encrypted file/dir.
|