|
@@ -243,52 +243,51 @@ class FSDirStatAndListingOp {
|
|
}, 0);
|
|
}, 0);
|
|
}
|
|
}
|
|
|
|
|
|
- Map<ByteBuffer, Long> children = tx.childrenView(targetNode.id()).tailMap(
|
|
|
|
- ByteBuffer.wrap(startAfter.getBytes(Charsets.UTF_8)));
|
|
|
|
- int numOfListing = Math.min(children.size(), fsd.getLsLimit());
|
|
|
|
- int locationBudget = fsd.getLsLimit();
|
|
|
|
- int listingCnt = 0;
|
|
|
|
- int i = 0;
|
|
|
|
- HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
|
|
|
|
-
|
|
|
|
- for (Map.Entry<ByteBuffer, Long> e : children.entrySet()) {
|
|
|
|
- if (locationBudget < 0 && i >= listing.length) {
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
|
|
+ try (DBChildrenView children = tx.childrenView(targetNode.id())) {
|
|
|
|
+ children.seekTo(ByteBuffer.wrap(startAfter.getBytes(Charsets.UTF_8)));
|
|
|
|
+ int numOfListing = fsd.getLsLimit();
|
|
|
|
+ int locationBudget = fsd.getLsLimit();
|
|
|
|
+ int listingCnt = 0;
|
|
|
|
+ int i = 0;
|
|
|
|
+ HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
|
|
|
|
+
|
|
|
|
+ for (Map.Entry<ByteBuffer, Long> e : children) {
|
|
|
|
+ if (locationBudget < 0 && i >= listing.length) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
|
|
- FlatINode cur = tx.getINode(e.getValue());
|
|
|
|
- // TODO: Handle Storage policy
|
|
|
|
|
|
+ FlatINode cur = tx.getINode(e.getValue());
|
|
|
|
+ // TODO: Handle Storage policy
|
|
// byte curPolicy = isSuperUser && !cur.isSymlink()?
|
|
// byte curPolicy = isSuperUser && !cur.isSymlink()?
|
|
// cur.getLocalStoragePolicyID():
|
|
// cur.getLocalStoragePolicyID():
|
|
// HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
// HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
- byte curPolicy = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
|
- ByteBuffer b =e.getKey().duplicate();
|
|
|
|
- byte[] localName = new byte[b.remaining()];
|
|
|
|
- b.get(localName);
|
|
|
|
- listing[i] =
|
|
|
|
- createFileStatus(tx, fsd, cur, localName, needLocation,
|
|
|
|
- getStoragePolicyID(curPolicy, parentStoragePolicy));
|
|
|
|
- if (needLocation) {
|
|
|
|
- // Once we hit lsLimit locations, stop.
|
|
|
|
- // This helps to prevent excessively large response payloads.
|
|
|
|
- // Approximate #locations with locatedBlockCount() * repl_factor
|
|
|
|
- LocatedBlocks blks =
|
|
|
|
- ((HdfsLocatedFileStatus)listing[i]).getBlockLocations();
|
|
|
|
- locationBudget -= (blks == null) ? 0 :
|
|
|
|
- blks.locatedBlockCount() * listing[i].getReplication();
|
|
|
|
|
|
+ byte curPolicy = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
|
+ ByteBuffer b = e.getKey().duplicate();
|
|
|
|
+ byte[] localName = new byte[b.remaining()];
|
|
|
|
+ b.get(localName);
|
|
|
|
+ listing[i] = createFileStatus(tx, fsd, cur, localName, needLocation,
|
|
|
|
+ getStoragePolicyID(curPolicy,
|
|
|
|
+ parentStoragePolicy));
|
|
|
|
+ if (needLocation) {
|
|
|
|
+ // Once we hit lsLimit locations, stop.
|
|
|
|
+ // This helps to prevent excessively large response payloads.
|
|
|
|
+ // Approximate #locations with locatedBlockCount() * repl_factor
|
|
|
|
+ LocatedBlocks blks = ((HdfsLocatedFileStatus) listing[i]).getBlockLocations();
|
|
|
|
+ locationBudget -= (blks == null) ? 0 : blks.locatedBlockCount() * listing[i].getReplication();
|
|
|
|
+ }
|
|
|
|
+ ++i;
|
|
|
|
+ ++listingCnt;
|
|
}
|
|
}
|
|
- ++i;
|
|
|
|
- ++listingCnt;
|
|
|
|
- }
|
|
|
|
|
|
|
|
- // truncate return array if necessary
|
|
|
|
- if (listingCnt < numOfListing) {
|
|
|
|
- listing = Arrays.copyOf(listing, listingCnt);
|
|
|
|
- }
|
|
|
|
|
|
+ // truncate return array if necessary
|
|
|
|
+ if (listingCnt < numOfListing) {
|
|
|
|
+ listing = Arrays.copyOf(listing, listingCnt);
|
|
|
|
+ }
|
|
|
|
|
|
- return new DirectoryListing(
|
|
|
|
- listing,
|
|
|
|
- listingCnt < numOfListing ? 0 : children.size() - listingCnt);
|
|
|
|
|
|
+ return new DirectoryListing(
|
|
|
|
+ listing,
|
|
|
|
+ listingCnt < numOfListing ? 0 : children.size() - listingCnt);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/** Get the file info for a specific file.
|
|
/** Get the file info for a specific file.
|
|
@@ -340,8 +339,8 @@ class FSDirStatAndListingOp {
|
|
FSDirectory fsd, String fullPath, byte[] path, INode node,
|
|
FSDirectory fsd, String fullPath, byte[] path, INode node,
|
|
byte storagePolicy, int snapshot, boolean isRawPath,
|
|
byte storagePolicy, int snapshot, boolean isRawPath,
|
|
INodesInPath iip) throws IOException {
|
|
INodesInPath iip) throws IOException {
|
|
- INodeAttributes nodeAttrs = getINodeAttributes(
|
|
|
|
- fsd, fullPath, path, node, snapshot);
|
|
|
|
|
|
+ INodeAttributes nodeAttrs = getINodeAttributes(fsd, fullPath, path, node,
|
|
|
|
+ snapshot);
|
|
return createFileStatus(fsd, path, node, nodeAttrs,
|
|
return createFileStatus(fsd, path, node, nodeAttrs,
|
|
storagePolicy, snapshot, isRawPath, iip);
|
|
storagePolicy, snapshot, isRawPath, iip);
|
|
}
|
|
}
|
|
@@ -426,9 +425,12 @@ class FSDirStatAndListingOp {
|
|
isEncrypted = false;
|
|
isEncrypted = false;
|
|
}
|
|
}
|
|
|
|
|
|
- int childrenNum = node.isDirectory()
|
|
|
|
- ? tx.childrenView(node.id()).size()
|
|
|
|
- : 0;
|
|
|
|
|
|
+ int childrenNum = 0;
|
|
|
|
+ if (node.isDirectory()) {
|
|
|
|
+ try(DBChildrenView children = tx.childrenView(node.id())) {
|
|
|
|
+ childrenNum = children.size();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
PermissionStatus perm = node.permissionStatus(fsd.ugid());
|
|
PermissionStatus perm = node.permissionStatus(fsd.ugid());
|
|
|
|
|
|
@@ -488,9 +490,12 @@ class FSDirStatAndListingOp {
|
|
loc = attachFileInfo(loc, fileSize, isUc, false, feInfo);
|
|
loc = attachFileInfo(loc, fileSize, isUc, false, feInfo);
|
|
}
|
|
}
|
|
|
|
|
|
- int childrenNum = node.isDirectory()
|
|
|
|
- ? tx.childrenView(node.id()).size()
|
|
|
|
- : 0;
|
|
|
|
|
|
+ int childrenNum = 0;
|
|
|
|
+ if (node.isDirectory()) {
|
|
|
|
+ try(DBChildrenView children = tx.childrenView(node.id())) {
|
|
|
|
+ childrenNum = children.size();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
PermissionStatus perm = node.permissionStatus(fsd.ugid());
|
|
PermissionStatus perm = node.permissionStatus(fsd.ugid());
|
|
HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(
|
|
HdfsLocatedFileStatus status = new HdfsLocatedFileStatus(
|