pulsar icon indicating copy to clipboard operation
pulsar copied to clipboard

[Bug] Deadlock in Function Worker shutdown

Open lhotari opened this issue 7 months ago • 0 comments

Search before reporting

  • [x] I searched in the issues and found nothing similar.

Read release policy

  • [x] I understand that unsupported versions don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker.

User environment

master branch, happened in CI

Issue Description

Shutdown of Function Worker got dead locked in PulsarFunctionPublishTest.shutdown.

Error messages

Relevant threads:

"main" #1 [2602] prio=5 os_prio=0 cpu=10027.41ms elapsed=3510.56s tid=0x00007fc5d0031a80 nid=2602 waiting for monitor entry  [0x00007fc5d7dfc000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.getFunctionRuntimeInfo(FunctionRuntimeManager.java)
        - waiting to lock <0x000010003b163950> (a org.apache.pulsar.functions.worker.FunctionRuntimeManager)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.stopFunction(FunctionRuntimeManager.java:524)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.lambda$stopAllOwnedFunctions$1(FunctionRuntimeManager.java:513)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager$$Lambda/0x0000000080e0df70.accept(Unknown Source)
        at java.lang.Iterable.forEach([email protected]/Iterable.java:75)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.stopAllOwnedFunctions(FunctionRuntimeManager.java:510)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.close(FunctionRuntimeManager.java:936)
        at org.apache.pulsar.functions.worker.PulsarWorkerService.stop(PulsarWorkerService.java:613)
        at org.apache.pulsar.functions.worker.PulsarFunctionPublishTest.shutdown(PulsarFunctionPublishTest.java:231)

Other thread in deadlock:
"cluster-service-coordinator-timer-114-1" #216 [3937] prio=5 os_prio=0 cpu=435.16ms elapsed=3479.21s tid=0x00007fc31d44cbc0 nid=3937 waiting on condition  [0x00007fc3237ec000]
   java.lang.Thread.State: WAITING (parking)
        at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
        - parking to wait for  <0x000010003b089778> (a java.util.concurrent.locks.ReentrantLock$FairSync)
        at java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:221)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:754)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:990)
        at java.util.concurrent.locks.ReentrantLock$Sync.lock([email protected]/ReentrantLock.java:153)
        at java.util.concurrent.locks.ReentrantLock.lock([email protected]/ReentrantLock.java:322)
        at org.apache.pulsar.functions.worker.PulsarWorkerService.lambda$start$5(PulsarWorkerService.java:543)
        at org.apache.pulsar.functions.worker.PulsarWorkerService$$Lambda/0x0000000080c08230.run(Unknown Source)
        at org.apache.pulsar.functions.worker.ClusterServiceCoordinator.lambda$start$0(ClusterServiceCoordinator.java:85)
        at org.apache.pulsar.functions.worker.ClusterServiceCoordinator$$Lambda/0x0000000080c08880.run(Unknown Source)
        at org.apache.pulsar.common.util.Runnables$CatchingAndLoggingRunnable.run(Runnables.java:54)
        at java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Executors.java:572)
        at java.util.concurrent.FutureTask.runAndReset([email protected]/FutureTask.java:358)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run([email protected]/ScheduledThreadPoolExecutor.java:305)
        at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1144)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:642)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.lang.Thread.runWith([email protected]/Thread.java:1596)
        at java.lang.Thread.run([email protected]/Thread.java:1583)


"worker-scheduler-0" #231 [3988] prio=5 os_prio=0 cpu=429.91ms elapsed=3476.17s tid=0x00007fc374040690 nid=3988 runnable  [0x00007fc3238eb000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
        - parking to wait for  <0x000010003b07ca90> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.parkNanos([email protected]/LockSupport.java:269)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos([email protected]/AbstractQueuedSynchronizer.java:1763)
        at java.util.concurrent.LinkedBlockingQueue.poll([email protected]/LinkedBlockingQueue.java:460)
        at org.apache.distributedlog.ReadAheadEntryReader.getNextReadAheadEntry(ReadAheadEntryReader.java:630)
        at org.apache.distributedlog.BKSyncLogReader.readNextEntry(BKSyncLogReader.java:145)
        at org.apache.distributedlog.BKSyncLogReader.doReadNext(BKSyncLogReader.java:191)
        at org.apache.distributedlog.BKSyncLogReader.readNext(BKSyncLogReader.java:173)
        - locked <0x000010001fdd5220> (a org.apache.distributedlog.BKSyncLogReader)
        at org.apache.pulsar.functions.worker.dlog.DLInputStream.nextLogRecord(DLInputStream.java:79)
        at org.apache.pulsar.functions.worker.dlog.DLInputStream.nextLogRecord(DLInputStream.java:71)
        at org.apache.pulsar.functions.worker.dlog.DLInputStream.read(DLInputStream.java:120)
        at java.io.InputStream.read([email protected]/InputStream.java:220)
        at org.apache.pulsar.functions.worker.WorkerUtils.downloadFromBookkeeper(WorkerUtils.java:127)
        at org.apache.pulsar.functions.worker.FunctionActioner.downloadFile(FunctionActioner.java:270)
        at org.apache.pulsar.functions.worker.FunctionActioner.getPackageFile(FunctionActioner.java:177)
        at org.apache.pulsar.functions.worker.FunctionActioner.startFunction(FunctionActioner.java:124)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.conditionallyStartFunction(FunctionRuntimeManager.java:990)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.startFunctionInstance(FunctionRuntimeManager.java:891)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.addAssignment(FunctionRuntimeManager.java:873)
        at org.apache.pulsar.functions.worker.FunctionRuntimeManager.processAssignment(FunctionRuntimeManager.java:715)
        - locked <0x000010003b163950> (a org.apache.pulsar.functions.worker.FunctionRuntimeManager)
        at org.apache.pulsar.functions.worker.SchedulerManager.invokeScheduler(SchedulerManager.java:453)
        at org.apache.pulsar.functions.worker.SchedulerManager.lambda$schedule$1(SchedulerManager.java:231)
        at org.apache.pulsar.functions.worker.SchedulerManager$$Lambda/0x0000000080ce9e78.run(Unknown Source)
        at org.apache.pulsar.functions.worker.SchedulerManager.lambda$scheduleInternal$0(SchedulerManager.java:211)
        at org.apache.pulsar.functions.worker.SchedulerManager$$Lambda/0x0000000080cea090.run(Unknown Source)
        at java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Executors.java:572)
        at java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:317)
        at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1144)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:642)
        at java.lang.Thread.runWith([email protected]/Thread.java:1596)
        at java.lang.Thread.run([email protected]/Thread.java:1583)

   Locked ownable synchronizers:
        - <0x0000100028001648> (a java.util.concurrent.ThreadPoolExecutor$Worker)
        - <0x000010003b089778> (a java.util.concurrent.locks.ReentrantLock$FairSync)

Reproducing the issue

Possibly with PulsarFunctionPublishTest

Additional information

Thread dump in jstack.review tool: https://jstack.review/?https://gist.github.com/lhotari/47b51acb20b9f6ebe87902c56c615a41

Are you willing to submit a PR?

  • [ ] I'm willing to submit a PR!

lhotari avatar May 28 '25 09:05 lhotari