hardikbajaj commented on issue #16783:
URL: https://github.com/apache/druid/issues/16783#issuecomment-2316958920

   I took some thread dumps during when this happens. From stack traces, It 
looks like there is some kind of a deadlock that happens and 
`intermediateTempExecutor` is stuck WAITING and we wait for it to shut down for 
[365 
days](https://github.com/apache/druid/blob/1d292c5a59cf023acdab13515fd783c44b7ffd51/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java#L1087)
   These are the main task's stack trace for the effected `TASK_ID`. These were 
the threads that contain `TASK_ID` in name
   <details>
     <summary>
       Task main thread stack trace 
`**[TASK_ID]-threading-task-runner-executor-5**`
     </summary>
   
   ```
   "[TASK_ID]-threading-task-runner-executor-5" #1314 daemon prio=5 os_prio=0 
cpu=18599767.26ms elapsed=113684.84s allocated=25572G defined_classes=8 
tid=0x00007f754c035210 nid=0x763 waiting on condition  [0x00007f6d0d919000]
      java.lang.Thread.State: TIMED_WAITING (parking)
        at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
        - parking to wait for  <0x000000043ee128f0> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at 
java.util.concurrent.locks.LockSupport.parkNanos([email protected]/LockSupport.java:252)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos([email protected]/AbstractQueuedSynchronizer.java:1672)
        at 
java.util.concurrent.ThreadPoolExecutor.awaitTermination([email protected]/ThreadPoolExecutor.java:1464)
        at 
com.google.common.util.concurrent.MoreExecutors$ListeningDecorator.awaitTermination(MoreExecutors.java:459)
        at 
org.apache.druid.segment.realtime.appenderator.StreamAppenderator.closeNow(StreamAppenderator.java:1015)
        at 
org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.runInternal(SeekableStreamIndexTaskRunner.java:866)
        at 
org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.run(SeekableStreamIndexTaskRunner.java:266)
        at 
org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask.runTask(SeekableStreamIndexTask.java:151)
        at 
org.apache.druid.indexing.common.task.AbstractTask.run(AbstractTask.java:169)
        at 
org.apache.druid.indexing.overlord.ThreadingTaskRunner$1.call(ThreadingTaskRunner.java:210)
        at 
org.apache.druid.indexing.overlord.ThreadingTaskRunner$1.call(ThreadingTaskRunner.java:152)
        at 
java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:264)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
        at java.lang.Thread.run([email protected]/Thread.java:840)
   
      Locked ownable synchronizers:
        - <0x00000002ef0002c8> (a 
java.util.concurrent.ThreadPoolExecutor$Worker)
   
   ```
   </details>
   
   <details>
     <summary>
       Stack trace: [TASK_ID]-appenderator-abandon
     </summary>
   
      ```
   "[task_id]-appenderator-abandon" #1629 daemon prio=5 os_prio=0 cpu=1.68ms 
elapsed=4704.25s allocated=3384B defined_classes=0 tid=0x00007f6f14039b60 
nid=0xa96 waiting on condition  [0x00007f6bdbbaa000]
      java.lang.Thread.State: WAITING (parking)
        at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
        - parking to wait for  <0x00000003dc000700> (a 
java.util.concurrent.SynchronousQueue$TransferStack)
        at 
java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:341)
        at 
java.util.concurrent.SynchronousQueue$TransferStack$SNode.block([email protected]/SynchronousQueue.java:288)
        at 
java.util.concurrent.ForkJoinPool.unmanagedBlock([email protected]/ForkJoinPool.java:3465)
        at 
java.util.concurrent.ForkJoinPool.managedBlock([email protected]/ForkJoinPool.java:3436)
        at 
java.util.concurrent.SynchronousQueue$TransferStack.transfer([email protected]/SynchronousQueue.java:397)
        at 
java.util.concurrent.SynchronousQueue.put([email protected]/SynchronousQueue.java:839)
        at 
org.apache.druid.java.util.common.concurrent.Execs$1.rejectedExecution(Execs.java:148)
        at 
java.util.concurrent.ThreadPoolExecutor.reject([email protected]/ThreadPoolExecutor.java:833)
        at 
java.util.concurrent.ThreadPoolExecutor.execute([email protected]/ThreadPoolExecutor.java:1365)
        at 
com.google.common.util.concurrent.MoreExecutors$ListeningDecorator.execute(MoreExecutors.java:484)
        at 
com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
        at 
com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
        at 
com.google.common.util.concurrent.ListenableFutureTask.done(ListenableFutureTask.java:91)
        at 
java.util.concurrent.FutureTask.finishCompletion([email protected]/FutureTask.java:381)
        at 
java.util.concurrent.FutureTask.setException([email protected]/FutureTask.java:250)
        at 
java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:269)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
        at java.lang.Thread.run([email protected]/Thread.java:840)
   
      Locked ownable synchronizers:
        - <0x00000003dc0007e8> (a 
java.util.concurrent.ThreadPoolExecutor$Worker)
      ```
     
   </details>
   
   <details>
     <summary>
       Stack trace: [TASK_ID]-publish
     </summary>
   
   ```
   "[task_id]-publish" #1626 daemon prio=5 os_prio=0 cpu=63.69ms 
elapsed=4881.78s allocated=42167K defined_classes=0 tid=0x00007f6f280057f0 
nid=0xa93 waiting on condition  [0x00007f6d181fe000]
      java.lang.Thread.State: WAITING (parking)
        at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
        - parking to wait for  <0x00000004427d0578> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at 
java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:341)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionNode.block([email protected]/AbstractQueuedSynchronizer.java:506)
        at 
java.util.concurrent.ForkJoinPool.unmanagedBlock([email protected]/ForkJoinPool.java:3465)
        at 
java.util.concurrent.ForkJoinPool.managedBlock([email protected]/ForkJoinPool.java:3436)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await([email protected]/AbstractQueuedSynchronizer.java:1623)
        at 
java.util.concurrent.LinkedBlockingQueue.take([email protected]/LinkedBlockingQueue.java:435)
        at 
java.util.concurrent.ThreadPoolExecutor.getTask([email protected]/ThreadPoolExecutor.java:1062)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1122)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
        at java.lang.Thread.run([email protected]/Thread.java:840)
   
      Locked ownable synchronizers:
        - None
   
   "[task_id]-appenderator-abandon" #1629 daemon prio=5 os_prio=0 cpu=1.68ms 
elapsed=4704.25s allocated=3384B defined_classes=0 tid=0x00007f6f14039b60 
nid=0xa96 waiting on condition  [0x00007f6bdbbaa000]
      java.lang.Thread.State: WAITING (parking)
        at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
        - parking to wait for  <0x00000003dc000700> (a 
java.util.concurrent.SynchronousQueue$TransferStack)
        at 
java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:341)
        at 
java.util.concurrent.SynchronousQueue$TransferStack$SNode.block([email protected]/SynchronousQueue.java:288)
        at 
java.util.concurrent.ForkJoinPool.unmanagedBlock([email protected]/ForkJoinPool.java:3465)
        at 
java.util.concurrent.ForkJoinPool.managedBlock([email protected]/ForkJoinPool.java:3436)
        at 
java.util.concurrent.SynchronousQueue$TransferStack.transfer([email protected]/SynchronousQueue.java:397)
        at 
java.util.concurrent.SynchronousQueue.put([email protected]/SynchronousQueue.java:839)
        at 
org.apache.druid.java.util.common.concurrent.Execs$1.rejectedExecution(Execs.java:148)
        at 
java.util.concurrent.ThreadPoolExecutor.reject([email protected]/ThreadPoolExecutor.java:833)
        at 
java.util.concurrent.ThreadPoolExecutor.execute([email protected]/ThreadPoolExecutor.java:1365)
        at 
com.google.common.util.concurrent.MoreExecutors$ListeningDecorator.execute(MoreExecutors.java:484)
        at 
com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
        at 
com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
        at 
com.google.common.util.concurrent.ListenableFutureTask.done(ListenableFutureTask.java:91)
        at 
java.util.concurrent.FutureTask.finishCompletion([email protected]/FutureTask.java:381)
        at 
java.util.concurrent.FutureTask.setException([email protected]/FutureTask.java:250)
        at 
java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:269)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
        at java.lang.Thread.run([email protected]/Thread.java:840)
   
      Locked ownable synchronizers:
        - <0x00000003dc0007e8> (a 
java.util.concurrent.ThreadPoolExecutor$Worker)
   ```
   </details>


-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to