[
https://issues.apache.org/jira/browse/HADOOP-18107?focusedWorklogId=771579&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-771579
]
ASF GitHub Bot logged work on HADOOP-18107:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 17/May/22 20:43
Start Date: 17/May/22 20:43
Worklog Time Spent: 10m
Work Description: mukund-thakur commented on code in PR #4273:
URL: https://github.com/apache/hadoop/pull/4273#discussion_r875246013
##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java:
##########
@@ -1095,6 +1102,54 @@ public static void validateFileContent(byte[] concat,
byte[][] bytes) {
mismatch);
}
+ /**
+ * Utility to validate vectored read results.
+ * @param fileRanges input ranges.
+ * @param originalData original data.
+ * @throws IOException any ioe.
+ */
+ public static void validateVectoredReadResult(List<FileRange> fileRanges,
+ byte[] originalData)
+ throws IOException, TimeoutException {
+ CompletableFuture<?>[] completableFutures = new
CompletableFuture<?>[fileRanges.size()];
+ int i = 0;
+ for (FileRange res : fileRanges) {
+ completableFutures[i++] = res.getData();
+ }
+ CompletableFuture<Void> combinedFuture =
CompletableFuture.allOf(completableFutures);
+ FutureIO.awaitFuture(combinedFuture, 5, TimeUnit.MINUTES);
+
+ for (FileRange res : fileRanges) {
+ CompletableFuture<ByteBuffer> data = res.getData();
+ ByteBuffer buffer = FutureIO.awaitFuture(data, 5, TimeUnit.MINUTES);
Review Comment:
I don't think just using allOf will complete the combined future. The intent
here was to get the individual future execution in parallel by calling a get on
combined future rather than a for loop which will lead to serial execution.
Also from the doc, it suggests to call join.
`Among the applications of this method is to await completion of a set of
independent CompletableFutures before continuing a program, as in:
CompletableFuture.allOf(c1, c2, c3).join();.`
Issue Time Tracking
-------------------
Worklog Id: (was: 771579)
Time Spent: 2h 20m (was: 2h 10m)
> Vectored IO support for large S3 files.
> ----------------------------------------
>
> Key: HADOOP-18107
> URL: https://issues.apache.org/jira/browse/HADOOP-18107
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/s3
> Reporter: Mukund Thakur
> Assignee: Mukund Thakur
> Priority: Major
> Labels: pull-request-available
> Time Spent: 2h 20m
> Remaining Estimate: 0h
>
> This effort would mostly be adding more tests for large files under scale
> tests and see if any new issue surfaces.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]