lukecwik opened a new issue, #23809: URL: https://github.com/apache/beam/issues/23809
### What happened? Source of bug report: https://github.com/apache/beam/pull/22645#issuecomment-1289308040 We (spotify) are seeing ConcurrentModificationException errors due to https://github.com/apache/beam/pull/22645. The issue starts in [WriteFiles.java](https://github.com/apache/beam/blame/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java#L964) where you construct a new future, then clear the existing closeFutures. I think the underlying cause is in [MoreFutures.java](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MoreFutures.java#L172) 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) ``` ### Issue Priority Priority: 1 ### Issue Component Component: sdk-java-core -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
