-
Notifications
You must be signed in to change notification settings - Fork 8.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HADOOP-17281 Implement FileSystem.listStatusIterator() in S3AFileSystem #2354
HADOOP-17281 Implement FileSystem.listStatusIterator() in S3AFileSystem #2354
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good. Annoying about the return types which force you to do that wrapping/casting. Can't you just forcibly cast the return type of the inner iterator? after all, type erasure means all type info will be lost in the actual compiled binary. I'd prefer that as it will give you automatic passthrough of the IOStatistics stuff.
Add text to filesystem.md, something which:
- specifies the result is exactly the same a listStatus, provided no other caller updates the directory during the list
- declares that it's not atomic and performance implementations will page
- and that if a path isn't there, that fact may not surface until next/hasNext...that is, we do lazy eval for all file IO
We need to similar new contract tests in AbstractContractGetFileStatusTest for all to use
- that in a dir with files and subdirectories, you get both returned in the listing
- that you can iterate through with next() to failure as well as hasNext/next, and get the same results
- listStatusIterator(file) returns the file
- listStatusIterator("/") gives you a listing of root (put that in AbstractContractRootDirectoryTest)
And two for changes partway through the iteration
- change the directory during a list to add/delete files
- deletes the actual path.
These tests can't assert on what will happen, and with paged IO aren't likely to pick up on changes...there just to show it can be done and pick up on any major issues with implementations.
This is not possible sadly.
Done
Actually this is not correct, we do throw FNFE if a path is not there, remember we have a check if the listing returns zeror results assuming the path as a directory, we fall back to file checks.
Done
As discussed not adding these tests. |
stevel and that if a path isn't there, that fact may not surface until next/hasNext...that is, we do lazy eval for all file IO mukund Actually this is not correct, we do throw FNFE if a path is not there, remember we have a check if the listing returns zeror results assuming the path as a directory, we fall back to file checks. I'd like the document to leave the option open of a fully async initial listing operation. so the people coding against it now are lined up for future filesystems that do that. Same for any other APIs we add, "late reporting of existence of paths/permissions" is something callers MUST expect |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think I would like the spec to say "potentially async", just to keep people happy, but let's not worry about that too much.
made some minor suggestions about the tests, primarily to use Assertions.assertThat throughout, just for the better reporting.
other than that -all good
Return an iterator enumerating the `FileStatus` entries under | ||
a path. This is similar to `listStatus(Path)` except the fact that | ||
rather than returning an entire list, an iterator is returned. | ||
The result is exactly the same as listStatus, provided no other caller |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
use listStatus()
with backticks
@SuppressWarnings("InfiniteLoopStatement") | ||
public static List<? extends FileStatus> iteratorToListThroughNextCallsAlone( | ||
RemoteIterator<? extends FileStatus> iterator) throws IOException { | ||
ArrayList<FileStatus> list = new ArrayList<>(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
same: List
as variable type
Actually, it may make sense to type this method
RemoteIterator<T extends FileStatus>
and have
List<T>
and
RemoteIterator<T>
where appropriate
@@ -242,6 +243,13 @@ public void testSimpleRootListing() throws IOException { | |||
+ "listStatus = " + listStatusResult | |||
+ "listFiles = " + listFilesResult, | |||
fileList.size() <= statuses.length); | |||
List<FileStatus> statusList = (List<FileStatus>) iteratorToList( | |||
fs.listStatusIterator(root)); | |||
String listStatusItrRes = join(statusList, "\n"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if you can have both sets of listing as collections, use Assertions.assertThat() as you've done earlier
List<FileStatus> statusList2 = | ||
(List<FileStatus>) iteratorToListThroughNextCallsAlone( | ||
getFileSystem().listStatusIterator(f)); | ||
assertEquals("size of file list returned through next() calls", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
AssertionsAssertThat to assert that the list size == 1
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM; down to final details
...op-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
Show resolved
Hide resolved
...mon-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java
Outdated
Show resolved
Hide resolved
…NoSuchElementException
@@ -1107,6 +1107,7 @@ public Void next(final FileSystem fs, final Path p) | |||
} | |||
|
|||
HdfsFileStatus[] partialListing = thisListing.getPartialListing(); | |||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please remove this before merging. Was just added to trigger hdfs tests by yetus.
yay! all the HDFS client tests passed first time! happy! |
OK, +1 on this. |
…em (#2354) Contains HADOOP-17300: FileSystem.DirListingIterator.next() call should return NoSuchElementException Contributed by Mukund Thakur Change-Id: I4e7e5c6e295525db9e2de6f416f32bbb81e146d3
Thanks @steveloughran |
…S3AFileSystem (apache#2354) Contains HADOOP-17300: FileSystem.DirListingIterator.next() call should return NoSuchElementException Contributed by Mukund Thakur Change-Id: I632956e705e9d4dd4bd692827ad63de775075c9e
Ran the new test using ap-south-1 bucket.
O/P-
(ContractTestUtils.java:end(1847)) - Duration of listing 1000 files using listFiles() api with batch size of 10 including 10ms of processing time for each file: 12,223,848,028 nS 2020-10-01 12:19:28,811 [JUnit-testMultiPagesListingPerformanceAndCorrectness] INFO contract.ContractTestUtils (ContractTestUtils.java:end(1847)) - Duration of listing 1000 files using listStatus() api with batch size of 10 including 10ms of processing time for each file: 15,988,037,357 nS 2020-10-01 12:19:41,050 [JUnit-testMultiPagesListingPerformanceAndCorrectness] INFO contract.ContractTestUtils (ContractTestUtils.java:end(1847)) - Duration of listing 1000 files using listStatusIterator() api with batch size of 10 including 10ms of processing time for each file: 12,214,813,052 nS
From the logs we can see that time taken using listStatusIterator() and listFiles() matches and is less than listStatus().