apache / pulsar

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org/
Apache License 2.0
14.28k stars 3.59k forks source link

Flaky-test: ExtensibleLoadManagerImplBaseTest.cleanup hangs in TableViewLoadDataStoreImpl.closeTableView due to a deadlock #23472

Open lhotari opened 1 month ago

lhotari commented 1 month ago

Search before asking

Example failure

https://github.com/apache/pulsar/actions/runs/11368523712/job/31653386805?pr=23468#step:10:616

thread dump: https://gist.github.com/lhotari/17557838cea2e4d4f4f1556fd4caec98 jstack.review analysis: https://jstack.review/?https://gist.github.com/lhotari/17557838cea2e4d4f4f1556fd4caec98#tda_1_dump

image

Exception stacktrace

"main" #1 prio=5 os_prio=0 cpu=67851.86ms elapsed=5366.03s tid=0x00007f52bc02f210 nid=0x33a0 waiting on condition  [0x00007f52c21fb000]
   java.lang.Thread.State: WAITING (parking)
        at jdk.internal.misc.Unsafe.park(java.base@17.0.13/Native Method)
        - parking to wait for  <0x00001000319052f8> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.park(java.base@17.0.13/LockSupport.java:211)
        at java.util.concurrent.CompletableFuture$Signaller.block(java.base@17.0.13/CompletableFuture.java:1864)
        at java.util.concurrent.ForkJoinPool.unmanagedBlock(java.base@17.0.13/ForkJoinPool.java:3465)
        at java.util.concurrent.ForkJoinPool.managedBlock(java.base@17.0.13/ForkJoinPool.java:3436)
        at java.util.concurrent.CompletableFuture.waitingGet(java.base@17.0.13/CompletableFuture.java:1898)
        at java.util.concurrent.CompletableFuture.get(java.base@17.0.13/CompletableFuture.java:2072)
        at org.apache.pulsar.client.impl.TableViewImpl.close(TableViewImpl.java:200)
        at org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl.closeTableView(TableViewLoadDataStoreImpl.java:148)
        - locked <0x000010002025ce08> (a org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl)
        at org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl.close(TableViewLoadDataStoreImpl.java:204)
        - locked <0x000010002025ce08> (a org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl)
        at org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl.shutdown(TableViewLoadDataStoreImpl.java:209)
        - locked <0x000010002025ce08> (a org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl)
        at org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.stopLoadDataReportTasks(ExtensibleLoadManagerImpl.java:816)
        at org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.disableBroker(ExtensibleLoadManagerImpl.java:1036)
        at java.lang.invoke.LambdaForm$DMH/0x00007f5038410c00.invokeVirtual(java.base@17.0.13/LambdaForm$DMH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038540c00.invoke(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448400.invoke(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448800.invokeExact_MT(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.MethodHandle.invokeWithArguments(java.base@17.0.13/MethodHandle.java:732)
        at org.mockito.internal.util.reflection.InstrumentationMemberAccessor$Dispatcher$ByteBuddy$mHxiQzAK.invokeWithArguments(Unknown Source)
        at org.mockito.internal.util.reflection.InstrumentationMemberAccessor.invoke(InstrumentationMemberAccessor.java:251)
        at org.mockito.internal.util.reflection.ModuleMemberAccessor.invoke(ModuleMemberAccessor.java:55)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice.tryInvoke(MockMethodAdvice.java:314)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice$RealMethodCall.invoke(MockMethodAdvice.java:234)
        at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:142)
        at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:45)
        at org.mockito.Answers.answer(Answers.java:90)
        at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:111)
        at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29)
        at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:34)
        at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:82)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice.handle(MockMethodAdvice.java:134)
        at org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.disableBroker(ExtensibleLoadManagerImpl.java:1033)
        at org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerWrapper.disableBroker(ExtensibleLoadManagerWrapper.java:88)
        at org.apache.pulsar.broker.service.BrokerService.unloadNamespaceBundlesGracefully(BrokerService.java:975)
        at org.apache.pulsar.broker.service.BrokerService.unloadNamespaceBundlesGracefully(BrokerService.java:962)
        at org.apache.pulsar.broker.PulsarService.closeAsync(PulsarService.java:525)
        at java.lang.invoke.DirectMethodHandle$Holder.invokeVirtual(java.base@17.0.13/DirectMethodHandle$Holder)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448000.invoke(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448400.invoke(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448800.invokeExact_MT(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.MethodHandle.invokeWithArguments(java.base@17.0.13/MethodHandle.java:732)
        at org.mockito.internal.util.reflection.InstrumentationMemberAccessor$Dispatcher$ByteBuddy$mHxiQzAK.invokeWithArguments(Unknown Source)
        at org.mockito.internal.util.reflection.InstrumentationMemberAccessor.invoke(InstrumentationMemberAccessor.java:251)
        at org.mockito.internal.util.reflection.ModuleMemberAccessor.invoke(ModuleMemberAccessor.java:55)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice.tryInvoke(MockMethodAdvice.java:314)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice$RealMethodCall.invoke(MockMethodAdvice.java:234)
        at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:142)
        at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:45)
        at org.mockito.Answers.answer(Answers.java:90)
        at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:111)
        at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29)
        at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:34)
        at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:82)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice.handle(MockMethodAdvice.java:134)
        at org.apache.pulsar.broker.PulsarService.closeAsync(PulsarService.java:509)
        at org.apache.pulsar.broker.PulsarService.close(PulsarService.java:484)
        at java.lang.invoke.LambdaForm$DMH/0x00007f5038410c00.invokeVirtual(java.base@17.0.13/LambdaForm$DMH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038540c00.invoke(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448400.invoke(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.LambdaForm$MH/0x00007f5038448800.invokeExact_MT(java.base@17.0.13/LambdaForm$MH)
        at java.lang.invoke.MethodHandle.invokeWithArguments(java.base@17.0.13/MethodHandle.java:732)
        at org.mockito.internal.util.reflection.InstrumentationMemberAccessor$Dispatcher$ByteBuddy$mHxiQzAK.invokeWithArguments(Unknown Source)
        at org.mockito.internal.util.reflection.InstrumentationMemberAccessor.invoke(InstrumentationMemberAccessor.java:251)
        at org.mockito.internal.util.reflection.ModuleMemberAccessor.invoke(ModuleMemberAccessor.java:55)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice.tryInvoke(MockMethodAdvice.java:314)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice$RealMethodCall.invoke(MockMethodAdvice.java:234)
        at org.mockito.internal.invocation.InterceptedInvocation.callRealMethod(InterceptedInvocation.java:142)
        at org.mockito.internal.stubbing.answers.CallsRealMethods.answer(CallsRealMethods.java:45)
        at org.mockito.Answers.answer(Answers.java:90)
        at org.mockito.internal.handler.MockHandlerImpl.handle(MockHandlerImpl.java:111)
        at org.mockito.internal.handler.NullResultGuardian.handle(NullResultGuardian.java:29)
        at org.mockito.internal.handler.InvocationNotifierHandler.handle(InvocationNotifierHandler.java:34)
        at org.mockito.internal.creation.bytebuddy.MockMethodInterceptor.doIntercept(MockMethodInterceptor.java:82)
        at org.mockito.internal.creation.bytebuddy.MockMethodAdvice.handle(MockMethodAdvice.java:134)
        at org.apache.pulsar.broker.PulsarService.close(PulsarService.java:484)
        at org.apache.pulsar.broker.testcontext.PulsarTestContext$StartableCustomBuilder.lambda$initializePulsarServices$0(PulsarTestContext.java:778)
        at org.apache.pulsar.broker.testcontext.PulsarTestContext$StartableCustomBuilder$$Lambda$568/0x00007f503853d598.close(Unknown Source)
        at org.apache.pulsar.broker.testcontext.PulsarTestContext.callCloseables(PulsarTestContext.java:218)
        at org.apache.pulsar.broker.testcontext.PulsarTestContext.close(PulsarTestContext.java:212)
        at org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.internalCleanup(MockedPulsarServiceBaseTest.java:287)
        at org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImplBaseTest.cleanup(ExtensibleLoadManagerImplBaseTest.java:171)

Are you willing to submit a PR?

lhotari commented 1 month ago

2 other threads related to the dead lock, in TableViewLoadDataStoreImpl.removeAsync method:

pulsar-load-manager-6391-1waiting to acquire [ 0x000010002025ce08 ] , holding [ 0x000010001e5511f0 0x00001000202630c0 ] at org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl.removeAsync(TableViewLoadDataStoreImpl.java) at org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.doCleanup(ServiceUnitStateChannelImpl.java:1602) at org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.lambda$scheduleCleanup$45(ServiceUnitStateChannelImpl.java:1357) at org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl$$Lambda$3727/0x00007f50392595c0.run(Unknown Source) at java.util.concurrent.CompletableFuture$AsyncRun.run(java.base@17.0.13/CompletableFuture.java:1804) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(java.base@17.0.13/Thread.java:840)

broker-client-shared-internal-executor-6394-1waiting to acquire [ 0x000010002025ce08 ] , holding [ 0x0000100034a98bc8 0x00001005062490e8 ] at org.apache.pulsar.broker.loadbalance.extensions.store.TableViewLoadDataStoreImpl.removeAsync(TableViewLoadDataStoreImpl.java) at org.apache.pulsar.broker.loadbalance.extensions.reporter.TopBundleLoadDataReporter.tombstone(TopBundleLoadDataReporter.java:109) at org.apache.pulsar.broker.loadbalance.extensions.reporter.TopBundleLoadDataReporter.handleEvent(TopBundleLoadDataReporter.java:138) at org.apache.pulsar.broker.loadbalance.extensions.channel.StateChangeListeners.lambda$notify$3(StateChangeListeners.java:74) at org.apache.pulsar.broker.loadbalance.extensions.channel.StateChangeListeners$$Lambda$3849/0x00007f5038922d18.accept(Unknown Source) at java.util.concurrent.CopyOnWriteArrayList.forEach(java.base@17.0.13/CopyOnWriteArrayList.java:807) at org.apache.pulsar.broker.loadbalance.extensions.channel.StateChangeListeners.notify(StateChangeListeners.java:72) at java.util.concurrent.CompletableFuture$UniAccept.tryFire(java.base@17.0.13/CompletableFuture.java:718) at java.util.concurrent.CompletableFuture.postComplete(java.base@17.0.13/CompletableFuture.java:510) at java.util.concurrent.CompletableFuture.complete(java.base@17.0.13/CompletableFuture.java:2147) at org.apache.pulsar.client.impl.ConsumerBase.lambda$completePendingReceive$0(ConsumerBase.java:333) at org.apache.pulsar.client.impl.ConsumerBase$$Lambda$1932/0x00007f5038bd6358.run(Unknown Source) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.13/ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.13/ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(java.base@17.0.13/Thread.java:840)

lhotari commented 1 month ago

TableViewLoadDataStoreImpl.removeAsync was made synchronized in #21777

heesung-sn commented 1 month ago

"main" #1 prio=5 os_prio=0 cpu=67851.86ms elapsed=5366.03s tid=0x00007f52bc02f210 nid=0x33a0 waiting on condition [0x00007f52c21fb000] java.lang.Thread.State: WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@17.0.13/Native Method)

It's odd that the test main thread cannot complete the table view close operation, which blocks other TableViewLoadDataStoreImpl operations.

heesung-sn commented 1 month ago

Raised a PR to unsynchrnoize TableViewLoadDataStoreImpl operations. https://github.com/apache/pulsar/pull/23487