[ https://issues.apache.org/jira/browse/MAPREDUCE-7370?focusedWorklogId=783993&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-783993 ]
ASF GitHub Bot logged work on MAPREDUCE-7370: --------------------------------------------- Author: ASF GitHub Bot Created on: 22/Jun/22 23:04 Start Date: 22/Jun/22 23:04 Worklog Time Spent: 10m Work Description: cnauroth commented on code in PR #4248: URL: https://github.com/apache/hadoop/pull/4248#discussion_r904363794 ########## hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java: ########## @@ -570,8 +570,14 @@ public void setStatus(String status) { */ @SuppressWarnings("unchecked") public void close() throws IOException, InterruptedException { - for (RecordWriter writer : recordWriters.values()) { - writer.close(context); - } + recordWriters.values().parallelStream().forEach(writer -> { Review Comment: I'm concerned that this could have unintended side effects for callers, because it changes the error contract. Errors during `close()` that were formerly visible as a checked `IOException` or `InterruptedException` now become an unchecked `RuntimeException`. In the case of thread interruption, the interrupt now occurs on the background thread with no propagation of interrupted status back up to the coordinating thread. Unfortunately, `parallelStream()` with a lambda puts us down this path. It would be more code, but directly managing a `ThreadPoolExecutor` would give you the chance to preserve the contract by unwrapping checked exceptions from the `Future` and propagating. Issue Time Tracking ------------------- Worklog Id: (was: 783993) Time Spent: 1h 10m (was: 1h) > Parallelize MultipleOutputs#close call > -------------------------------------- > > Key: MAPREDUCE-7370 > URL: https://issues.apache.org/jira/browse/MAPREDUCE-7370 > Project: Hadoop Map/Reduce > Issue Type: Improvement > Affects Versions: 3.3.0 > Reporter: Prabhu Joseph > Assignee: Ashutosh Gupta > Priority: Major > Labels: pull-request-available > Time Spent: 1h 10m > Remaining Estimate: 0h > > This call takes more time when there are lot of files to close and there is a > high latency to close. Parallelize MultipleOutputs#close call to improve the > speed. > {code} > public void close() throws IOException { > for (RecordWriter writer : recordWriters.values()) { > writer.close(null); > } > } > {code} > Idea is from [~ste...@apache.org] -- This message was sent by Atlassian Jira (v8.20.7#820007) --------------------------------------------------------------------- To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org