Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Bug]: FileIO: lack of timeouts may cause the pipeline to get stuck indefinitely #29926

Closed
1 of 16 tasks
an2x opened this issue Jan 4, 2024 · 2 comments · Fixed by #33042
Closed
1 of 16 tasks

[Bug]: FileIO: lack of timeouts may cause the pipeline to get stuck indefinitely #29926

an2x opened this issue Jan 4, 2024 · 2 comments · Fixed by #33042
Assignees

Comments

@an2x
Copy link
Contributor

an2x commented Jan 4, 2024

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: on FinishBundle, WriteFiles waits for all futures that close writers to complete, but they never do.

writer.cleanup from here 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
  • 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
@liferoad
Copy link
Collaborator

liferoad commented Jan 5, 2024

cc @shunping

@scwhittle
Copy link
Contributor

This was observed again recently. Looking at thread stacks showed that the futures that should be running to flush files were not stuck but were instead neither running nor scheduled. Full thread stacks are below.

I believe that this may be an issue with the use of ForkJoinPool in beam's MoreFutures.runAsync . The common pool is not suitable for asynchronous non-joining work and I believe that might be the case due to the indirect notification of futures in this method. I believe that #33042 may fix the issue by ensuring that the futures we join upon are directly those submitted to the ForkJoinPool.

--- Threads (10): [Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main], Thread[gcs-async-channel-pool-0,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  [email protected]/java.io.PipedInputStream.read(PipedInputStream.java:326)
  [email protected]/java.io.PipedInputStream.read(PipedInputStream.java:377)
  app//com.google.api.client.util.ByteStreams.read(ByteStreams.java:172)
  app//com.google.api.client.googleapis.media.MediaHttpUploader.buildContentChunk(MediaHttpUploader.java:610)
  app//com.google.api.client.googleapis.media.MediaHttpUploader.resumableUpload(MediaHttpUploader.java:380)
  app//com.google.api.client.googleapis.media.MediaHttpUploader.upload(MediaHttpUploader.java:308)
  app//com.google.api.client.googleapis.services.AbstractGoogleClientRequest.executeUnparsed(AbstractGoogleClientRequest.java:566)
  app//com.google.api.client.googleapis.services.AbstractGoogleClientRequest.executeUnparsed(AbstractGoogleClientRequest.java:493)
  app//com.google.api.client.googleapis.services.AbstractGoogleClientRequest.execute(AbstractGoogleClientRequest.java:603)
  app//com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel$UploadOperation.call(AbstractGoogleAsyncWriteChannel.java:85)
  [email protected]/java.util.concurrent.FutureTask.run(FutureTask.java:264)
  [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:829)

--- Threads (10): [Thread[DataflowWorkUnits-1430477,5,main], Thread[DataflowWorkUnits-1430764,5,main], Thread[DataflowWorkUnits-1430492,5,main], Thread[DataflowWorkUnits-1430575,5,main], Thread[DataflowWorkUnits-1430643,5,main], Thread[DataflowWorkUnits-1430870,5,main], Thread[DataflowWorkUnits-1430846,5,main], Thread[DataflowWorkUnits-1430637,5,main], Thread[DataflowWorkUnits-1430662,5,main], Thread[DataflowWorkUnits-1430774,5,main]] State: WAITING stack: ---
  [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:1225)
  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:222)
  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.streaming.ComputationWorkExecutor.executeWork(ComputationWorkExecutor.java:78)
  app//org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler.executeWork(StreamingWorkScheduler.java:377)
  app//org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler.processWork(StreamingWorkScheduler.java:252)
  app//org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler.lambda$scheduleWork$2(StreamingWorkScheduler.java:211)
  app//org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler$$Lambda$299/0x0000000800476440.accept(Unknown Source)
  app//org.apache.beam.runners.dataflow.worker.streaming.ExecutableWork.run(ExecutableWork.java:38)
  app//org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor.lambda$executeMonitorHeld$0(BoundedQueueExecutor.java:232)
  app//org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor$$Lambda$301/0x0000000800475040.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:829)

--- Threads (10): [Thread[pool-60-thread-1,5,main], Thread[pool-62-thread-1,5,main], Thread[pool-58-thread-1,5,main], Thread[pool-55-thread-1,5,main], Thread[pool-59-thread-1,5,main], Thread[pool-56-thread-1,5,main], Thread[pool-53-thread-1,5,main], Thread[pool-54-thread-1,5,main], Thread[pool-61-thread-1,5,main], Thread[pool-57-thread-1,5,main]] State: WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  [email protected]/java.lang.Object.wait(Object.java:328)
  app//org.apache.beam.sdk.util.UnboundedScheduledExecutorService$TaskLauncher.call(UnboundedScheduledExecutorService.java:303)
  app//org.apache.beam.sdk.util.UnboundedScheduledExecutorService$TaskLauncher.call(UnboundedScheduledExecutorService.java:280)
  [email protected]/java.util.concurrent.FutureTask.run(FutureTask.java:264)
  [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:829)

--- Threads (4): [Thread[Connector-Scheduler-26c89563-1,5,main], Thread[ReportWorkerMessage,5,main], Thread[GlobalWorkerUpdates,5,main], Thread[RefreshWork,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/jdk.internal.misc.Unsafe.park(Native Method)
  [email protected]/java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:234)
  [email protected]/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2123)
  [email protected]/java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1182)
  [email protected]/java.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:899)
  [email protected]/java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1054)
  [email protected]/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1114)
  [email protected]/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (3): [Thread[qtp1152554134-23,5,main], Thread[qtp1152554134-24,5,main], Thread[qtp1152554134-20,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/jdk.internal.misc.Unsafe.park(Native Method)
  [email protected]/java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:234)
  [email protected]/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2123)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:382)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.idleJobPoll(QueuedThreadPool.java:974)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1018)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (2): [Thread[ForkJoinPool.commonPool-worker-1,5,main], Thread[ForkJoinPool.commonPool-worker-7,5,main]] State: WAITING stack: ---
  [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.ForkJoinPool.runWorker(ForkJoinPool.java:1628)
  [email protected]/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)

--- Threads (2): [Thread[qtp1152554134-22,5,main], Thread[qtp1152554134-25,5,main]] State: RUNNABLE stack: ---
  [email protected]/sun.nio.ch.EPoll.wait(Native Method)
  [email protected]/sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:120)
  [email protected]/sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:124)
  [email protected]/sun.nio.ch.SelectorImpl.select(SelectorImpl.java:141)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.io.ManagedSelector.nioSelect(ManagedSelector.java:183)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.io.ManagedSelector.select(ManagedSelector.java:190)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:606)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:543)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.produceTask(EatWhatYouKill.java:362)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.doProduce(EatWhatYouKill.java:186)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.tryProduce(EatWhatYouKill.java:173)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.strategy.EatWhatYouKill.run(EatWhatYouKill.java:131)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:409)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:883)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1034)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (2): [Thread[Signal Dispatcher,9,system], Thread[DestroyJavaVM,5,main]] State: RUNNABLE stack: ---

--- Threads (1): [Thread[Reference Handler,10,system]] State: RUNNABLE stack: ---
  [email protected]/java.lang.ref.Reference.waitForReferencePendingList(Native Method)
  [email protected]/java.lang.ref.Reference.processPendingReferences(Reference.java:241)
  [email protected]/java.lang.ref.Reference$ReferenceHandler.run(Reference.java:213)

--- Threads (1): [Thread[RMI TCP Accept-0,5,system]] State: RUNNABLE stack: ---
  [email protected]/java.net.PlainSocketImpl.socketAccept(Native Method)
  [email protected]/java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:474)
  [email protected]/java.net.ServerSocket.implAccept(ServerSocket.java:565)
  [email protected]/java.net.ServerSocket.accept(ServerSocket.java:533)
  [email protected]/sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept(LocalRMIServerSocketFactory.java:52)
  [email protected]/sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:394)
  [email protected]/sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:366)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[OpenCensus.Disruptor-0,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/jdk.internal.misc.Unsafe.park(Native Method)
  [email protected]/java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:357)
  app//com.lmax.disruptor.SleepingWaitStrategy.applyWaitMethod(SleepingWaitStrategy.java:92)
  app//com.lmax.disruptor.SleepingWaitStrategy.waitFor(SleepingWaitStrategy.java:65)
  app//com.lmax.disruptor.ProcessingSequenceBarrier.waitFor(ProcessingSequenceBarrier.java:56)
  app//com.lmax.disruptor.BatchEventProcessor.processEvents(BatchEventProcessor.java:159)
  app//com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:125)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[ExportComponent.ServiceExporterThread-0,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  app//io.opencensus.implcore.trace.export.SpanExporterImpl$Worker.run(SpanExporterImpl.java:278)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[state-sampler-0,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Thread.sleep(Native Method)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.metrics.ExecutionStateSampler.lambda$start$0(ExecutionStateSampler.java:114)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.metrics.ExecutionStateSampler$$Lambda$282/0x0000000800422840.call(Unknown Source)
  [email protected]/java.util.concurrent.FutureTask.run(FutureTask.java:264)
  [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:829)

--- Threads (1): [Thread[Finalizer,8,system]] State: WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  [email protected]/java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:155)
  [email protected]/java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:176)
  [email protected]/java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:170)

--- Threads (1): [Thread[Common-Cleaner,8,InnocuousThreadGroup]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  [email protected]/java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:155)
  [email protected]/jdk.internal.ref.CleanerImpl.run(CleanerImpl.java:148)
  [email protected]/java.lang.Thread.run(Thread.java:829)
  [email protected]/jdk.internal.misc.InnocuousThread.run(InnocuousThread.java:161)

--- Threads (1): [Thread[CommitThread-0,10,main]] State: WAITING stack: ---
  [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.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2081)
  [email protected]/java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:433)
  app//org.apache.beam.runners.dataflow.worker.streaming.WeightedBoundedQueue.take(WeightedBoundedQueue.java:88)
  app//org.apache.beam.runners.dataflow.worker.windmill.client.commits.StreamingApplianceWorkCommitter.commitLoop(StreamingApplianceWorkCommitter.java:113)
  app//org.apache.beam.runners.dataflow.worker.windmill.client.commits.StreamingApplianceWorkCommitter$$Lambda$283/0x0000000800422040.run(Unknown Source)
  [email protected]/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
  [email protected]/java.util.concurrent.FutureTask.run(FutureTask.java:264)
  [email protected]/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
  [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:829)

--- Threads (1): [Thread[qtp1152554134-21-acceptor-0@1b8f4a92-ServerConnector@26c89563{HTTP/1.1, (http/1.1)}{0.0.0.0:8081},3,main]] State: RUNNABLE stack: ---
  [email protected]/sun.nio.ch.ServerSocketChannelImpl.accept0(Native Method)
  [email protected]/sun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:533)
  [email protected]/sun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:285)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.server.ServerConnector.accept(ServerConnector.java:388)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.server.AbstractConnector$Acceptor.run(AbstractConnector.java:704)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:883)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1034)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[qtp1152554134-26,5,main]] State: TIMED_WAITING stack: ---
  [email protected]/jdk.internal.misc.Unsafe.park(Native Method)
  [email protected]/java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:234)
  [email protected]/java.util.concurrent.SynchronousQueue$TransferStack.awaitFulfill(SynchronousQueue.java:462)
  [email protected]/java.util.concurrent.SynchronousQueue$TransferStack.transfer(SynchronousQueue.java:361)
  [email protected]/java.util.concurrent.SynchronousQueue.poll(SynchronousQueue.java:937)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.reservedWait(ReservedThreadExecutor.java:324)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.ReservedThreadExecutor$ReservedThread.run(ReservedThreadExecutor.java:399)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:883)
  app//org.apache.beam.runners.dataflow.worker.repackaged.org.eclipse.jetty.util.thread.QueuedThreadPool$Runner.run(QueuedThreadPool.java:1034)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[MemoryMonitor,1,main]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  app//org.apache.beam.runners.dataflow.worker.util.MemoryMonitor.run(MemoryMonitor.java:594)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[DispatchThread,1,main]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Thread.sleep(Native Method)
  [email protected]/java.lang.Thread.sleep(Thread.java:334)
  [email protected]/java.util.concurrent.TimeUnit.sleep(TimeUnit.java:446)
  app//org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly(Uninterruptibles.java:406)
  app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.sleep(StreamingDataflowWorker.java:658)
  app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.dispatchLoop(StreamingDataflowWorker.java:782)
  app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.lambda$new$0(StreamingDataflowWorker.java:231)
  app//org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker$$Lambda$237/0x00000008003e0c40.run(Unknown Source)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[RMI TCP Accept-5555,5,system]] State: RUNNABLE stack: ---
  [email protected]/java.net.PlainSocketImpl.socketAccept(Native Method)
  [email protected]/java.net.AbstractPlainSocketImpl.accept(AbstractPlainSocketImpl.java:474)
  [email protected]/java.net.ServerSocket.implAccept(ServerSocket.java:565)
  [email protected]/java.net.ServerSocket.accept(ServerSocket.java:533)
  [email protected]/sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:394)
  [email protected]/sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:366)
  [email protected]/java.lang.Thread.run(Thread.java:829)

--- Threads (1): [Thread[Keep-Alive-SocketCleaner,8,InnocuousThreadGroup]] State: WAITING stack: ---
  [email protected]/java.lang.Object.wait(Native Method)
  [email protected]/sun.net.www.http.KeepAliveStreamCleaner.run(KeepAliveStreamCleaner.java:101)
  [email protected]/java.lang.Thread.run(Thread.java:829)
  [email protected]/jdk.internal.misc.InnocuousThread.run(InnocuousThread.java:161)

--- Threads (1): [Thread[Keep-Alive-Timer,8,InnocuousThreadGroup]] State: TIMED_WAITING stack: ---
  [email protected]/java.lang.Thread.sleep(Native Method)
  [email protected]/sun.net.www.http.KeepAliveCache.run(KeepAliveCache.java:222)
  [email protected]/java.lang.Thread.run(Thread.java:829)
  [email protected]/jdk.internal.misc.InnocuousThread.run(InnocuousThread.java:161)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment