an2x opened a new issue, #29926:
URL: https://github.com/apache/beam/issues/29926
### What happened?
A java pipeline running on Dataflow using Beam 2.47 (but the latest 2.52 is
affected by the same issue) went into an unrecoverable state, with workers
being stuck for at least 4 days with the following stack trace:
```
[email protected]/jdk.internal.misc.Unsafe.park(Native Method)
[email protected]/java.util.concurrent.locks.LockSupport.park(LockSupport.java:194)
[email protected]/java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1796)
[email protected]/java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3128)
[email protected]/java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1823)
[email protected]/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1998)
app//org.apache.beam.sdk.util.MoreFutures.get(MoreFutures.java:61)
app//org.apache.beam.sdk.io.WriteFiles$WriteShardsIntoTempFilesFn.finishBundle(WriteFiles.java:1027)
app//org.apache.beam.sdk.io.WriteFiles$WriteShardsIntoTempFilesFn$DoFnInvoker.invokeFinishBundle(Unknown
Source)
app//org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner.finishBundle(SimpleDoFnRunner.java:221)
app//org.apache.beam.runners.dataflow.worker.SimpleParDoFn.finishBundle(SimpleParDoFn.java:433)
app//org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoOperation.finish(ParDoOperation.java:56)
app//org.apache.beam.runners.dataflow.worker.util.common.worker.MapTaskExecutor.execute(MapTaskExecutor.java:94)
app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.process(StreamingDataflowWorker.java:1401)
app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.access$800(StreamingDataflowWorker.java:153)
app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker$4.run(StreamingDataflowWorker.java:1043)
app//org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor.lambda$executeLockHeld$0(BoundedQueueExecutor.java:133)
app//org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor$$Lambda$251/0x00000008005e7840.run(Unknown
Source)
[email protected]/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
[email protected]/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
[email protected]/java.lang.Thread.run(Thread.java:834)
```
The source is
[here](https://github.com/apache/beam/blob/0b1b07398cf139ee0824a75c6ac1c9d3b49616ce/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java#L1007):
on FinishBundle, `WriteFiles` waits for all futures that close writers to
complete, but they never do.
`writer.cleanup` from
[here](https://github.com/apache/beam/blob/0b1b07398cf139ee0824a75c6ac1c9d3b49616ce/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java#L1018)
produces a log message, which we didn't observe in this case, so the futures
seemed to be stuck on `writer.close`. It's unclear why (possibly, some network
issues, or other issues with Cloud Storage, which was the target file system),
but I think Beam SDK should implement a timeout here: if a writer can't be
closed within reasonable time, it should fail and the bundle should be
re-processed.
### Issue Priority
Priority: 1 (data loss / total loss of function)
### Issue Components
- [ ] Component: Python SDK
- [X] Component: Java SDK
- [ ] Component: Go SDK
- [ ] Component: Typescript SDK
- [ ] Component: IO connector
- [ ] Component: Beam YAML
- [ ] Component: Beam examples
- [ ] Component: Beam playground
- [ ] Component: Beam katas
- [ ] Component: Website
- [ ] Component: Spark Runner
- [ ] Component: Flink Runner
- [ ] Component: Samza Runner
- [ ] Component: Twister2 Runner
- [ ] Component: Hazelcast Jet Runner
- [ ] Component: Google Cloud Dataflow Runner
--
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]