beam
beam copied to clipboard
[WIP][BEAM-12776, fixes #21095] Limit parallel closes from the prior element to the next element.
This will reduce OOMs in the case where we are adding new writes faster then we are able to clean-up older ones across elements.
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
- [ ] Choose reviewer(s) and mention them in a comment (
R: @username
). - [ ] Mention the appropriate issue in your description (for example:
addresses #123
), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, commentfixes #<ISSUE NUMBER>
instead. - [ ] Update
CHANGES.md
with noteworthy changes. - [ ] If this contribution is large, please file an Apache Individual Contributor License Agreement.
See the Contributor Guide for more tips on how to make review process smoother.
To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md
GitHub Actions Tests Status (on master branch)
See CI.md for more information about GitHub Actions CI.
@scwhittle What do you think about this solution for limiting OOMs?
Run Java PreCommit
Run Java PreCommit
Run Java_Examples_Dataflow PreCommit
Run Spotless PreCommit
Run Spotless PreCommit
Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer
:
R: @robertwb for label java.
Available commands:
-
stop reviewer notifications
- opt out of the automated review tooling -
remind me after tests pass
- tag the comment author after tests pass -
waiting on author
- shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
The PR bot will only process comments in the main thread (not review comments).
Run Java PreCommit
Run Java PreCommit
@lukecwik We (spotify) are seeing ConcurrentModificationException
errors due to this PR.
The issue starts in WriteFiles.java where you construct a new future, then clear the existing closeFuture
s.
I think the underlying cause is in MoreFutures.java where you close over the input futures
inside thenApply
. If you instead used the array result of futuresToCompletableFutures
I suspect the issue could be avoided.
Something like this?
CompletableFuture<? extends T>[] f = futuresToCompletableFutures(futures);
CompletionStage<Void> blockAndDiscard = CompletableFuture.allOf(f);
return blockAndDiscard.thenApply(
nothing ->
Arrays.stream(f)
.map(CompletableFuture::join)
.collect(Collectors.toList()));
Similarly for allAsListWithExceptions
.
Stack trace for the error (from a scio test):
[info] org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.util.concurrent.ExecutionException: java.util.ConcurrentModificationException
[info] at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:374)
[info] at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:342)
[info] at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:218)
[info] at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
[info] at org.apache.beam.sdk.Pipeline.run(Pipeline.java:323)
[info] at org.apache.beam.sdk.Pipeline.run(Pipeline.java:309)
[info] at com.spotify.scio.ScioContext.execute(ScioContext.scala:586)
[info] at com.spotify.scio.ScioContext$$anonfun$run$1.apply(ScioContext.scala:573)
[info] at com.spotify.scio.ScioContext$$anonfun$run$1.apply(ScioContext.scala:561)
[info] at com.spotify.scio.ScioContext.requireNotClosed(ScioContext.scala:652)
[info] ...
[info] Cause: java.util.concurrent.ExecutionException: java.util.ConcurrentModificationException
[info] at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
[info] at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908)
[info] at org.apache.beam.sdk.util.MoreFutures.get(MoreFutures.java:60)
[info] at org.apache.beam.sdk.io.WriteFiles$WriteShardsIntoTempFilesFn.processElement(WriteFiles.java:984)
[info] ...
[info] Cause: java.util.ConcurrentModificationException:
[info] at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1390)
[info] at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
[info] at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
[info] at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
[info] at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
[info] at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
[info] at org.apache.beam.sdk.util.MoreFutures.lambda$allAsList$5(MoreFutures.java:174)
[info] at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:616)
[info] at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:591)
[info] at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
@kellen Filed https://github.com/apache/beam/issues/23809 with a fix in https://github.com/apache/beam/pull/23811 based upon your supplied patch
@kellen If you could review the PR then we could have it merged. Comment LGTM on PR if it looks good to you.