Skip to content

[Bug] Deadlock occurs when skipping non-recoverable ledger #21914

@hrzzzz

Description

@hrzzzz

Search before asking

  • I searched in the issues and found nothing similar.

Version

OS: Linux
Pulsar: 3.0.2

Minimal reproduce step

There are race conditions here and executing the following two methods in parallel may trigger the deadlock:

  • org.apache.bookkeeper.mledger.impl.ManagedCursorImpl#skipNonRecoverableLedger
  • org.apache.bookkeeper.mledger.impl.ManagedCursorImpl#asyncDelete

What did you expect to see?

no deadlock

What did you see instead?

deadlock occurs.

====> DEADLOCKS DETECTED <====


"BookKeeperClientWorker-OrderedExecutor-6-0" Id=55 in WAITING on lock=java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@24e08dc4
     owned by BookKeeperClientWorker-OrderedExecutor-11-0 Id=60
    at java.base@17.0.9/jdk.internal.misc.Unsafe.park(Native Method)
    at java.base@17.0.9/java.util.concurrent.locks.LockSupport.park(LockSupport.java:211)
    at java.base@17.0.9/java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:715)
    at java.base@17.0.9/java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:938)
    at java.base@17.0.9/java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:959)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$19.operationComplete(ManagedCursorImpl.java:2130)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.lambda$persistPositionToLedger$40(ManagedCursorImpl.java:3097)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$$Lambda$1497/0x00007febacac5ea8.addComplete(Unknown Source)
    at app//org.apache.bookkeeper.client.AsyncCallback$AddCallback.addCompleteWithLatency(AsyncCallback.java:92)
    at app//org.apache.bookkeeper.client.PendingAddOp.submitCallback(PendingAddOp.java:406)
      - locked org.apache.bookkeeper.client.PendingAddOp@3d914523
    at app//org.apache.bookkeeper.client.LedgerHandle.sendAddSuccessCallbacks(LedgerHandle.java:1835)
    at app//org.apache.bookkeeper.client.PendingAddOp.sendAddSuccessCallbacks(PendingAddOp.java:390)
    at app//org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:384)
      - locked org.apache.bookkeeper.client.PendingAddOp@3d914523
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.writeComplete(PerChannelBookieClient.java:2183)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleResponse(PerChannelBookieClient.java:2240)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.handleV2Response(PerChannelBookieClient.java:2219)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1397)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113)
    at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base@17.0.9/java.lang.Thread.run(Thread.java:840)

    Locked synchronizers: count = 0



"BookKeeperClientWorker-OrderedExecutor-11-0" Id=60 in BLOCKED on lock=java.util.ArrayDeque@73a768d4
     owned by BookKeeperClientWorker-OrderedExecutor-60-0 Id=109
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.internalAsyncMarkDelete(ManagedCursorImpl.java:2036)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncDelete(ManagedCursorImpl.java:2387)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncDelete(ManagedCursorImpl.java:2190)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.skipNonRecoverableLedger(ManagedCursorImpl.java:2779)
    at app//org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.skipNonRecoverableLedger(ManagedLedgerImpl.java:1767)
    at app//org.apache.bookkeeper.mledger.impl.OpReadEntry.readEntriesFailed(OpReadEntry.java:137)
    at app//org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager$PendingRead.lambda$attach$2(PendingReadsManager.java:299)
      - locked org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager$PendingRead@35457076
    at app//org.apache.bookkeeper.mledger.impl.cache.PendingReadsManager$PendingRead$$Lambda$1476/0x00007febacab88c0.apply(Unknown Source)
    at java.base@17.0.9/java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:990)
    at java.base@17.0.9/java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:974)
    at java.base@17.0.9/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
    at java.base@17.0.9/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
    at app//org.apache.bookkeeper.client.PendingReadOp.submitCallback(PendingReadOp.java:653)
    at app//org.apache.bookkeeper.client.PendingReadOp$LedgerEntryRequest.fail(PendingReadOp.java:175)
    at app//org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest.sendNextRead(PendingReadOp.java:396)
      - locked org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest@1d2d4d48
    at app//org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest.logErrorAndReattemptRead(PendingReadOp.java:439)
      - locked org.apache.bookkeeper.client.PendingReadOp$SequenceReadRequest@1d2d4d48
    at app//org.apache.bookkeeper.client.PendingReadOp.readEntryComplete(PendingReadOp.java:594)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion$1.readEntryComplete(PerChannelBookieClient.java:1885)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleReadResponse(PerChannelBookieClient.java:1967)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadCompletion.handleV2Response(PerChannelBookieClient.java:1919)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$ReadV2ResponseCallback.run(PerChannelBookieClient.java:1397)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113)
    at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base@17.0.9/java.lang.Thread.run(Thread.java:840)

    Locked synchronizers: count = 1
      - java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@24e08dc4



"BookKeeperClientWorker-OrderedExecutor-60-0" Id=109 in WAITING on lock=java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@24e08dc4
     owned by BookKeeperClientWorker-OrderedExecutor-11-0 Id=60
    at java.base@17.0.9/jdk.internal.misc.Unsafe.park(Native Method)
    at java.base@17.0.9/java.util.concurrent.locks.LockSupport.park(LockSupport.java:211)
    at java.base@17.0.9/java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:715)
    at java.base@17.0.9/java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(AbstractQueuedSynchronizer.java:1027)
    at java.base@17.0.9/java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(ReentrantReadWriteLock.java:738)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.buildIndividualDeletedMessageRanges(ManagedCursorImpl.java:2987)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.persistPositionToLedger(ManagedCursorImpl.java:3069)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.internalMarkDelete(ManagedCursorImpl.java:2179)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.internalAsyncMarkDelete(ManagedCursorImpl.java:2060)
      - locked java.util.ArrayDeque@73a768d4
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.internalResetCursor(ManagedCursorImpl.java:1359)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.lambda$asyncResetCursor$15(ManagedCursorImpl.java:1395)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$$Lambda$1540/0x00007febacb19710.run(Unknown Source)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137)
    at app//org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:107)
    at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
    at java.base@17.0.9/java.lang.Thread.run(Thread.java:840)

    Locked synchronizers: count = 0

Anything else?

The sequence of events leading to the deadlock when methods from org.apache.bookkeeper.mledger.impl.ManagedCursorImpl are invoked concurrently is as follows:

  1. Thread A calls asyncDelete, which then goes on to internally call internalAsyncMarkDelete. This results in acquiring a lock on pendingMarkDeleteOps through synchronized (pendingMarkDeleteOps).

  2. Inside internalAsyncMarkDelete, internalMarkDelete is called which subsequently calls persistPositionToLedger. At the start of persistPositionToLedger, buildIndividualDeletedMessageRanges is invoked, where it tries to acquire a read lock using lock.readLock().lock(). At this point, if the write lock is being held by another thread, Thread A will block waiting for the read lock.

  3. Concurrently, Thread B executes skipNonRecoverableLedger which first obtains a write lock using lock.writeLock().lock() and then proceeds to call asyncDelete.

  4. At this moment, Thread B already holds the write lock and is attempting to acquire the synchronized lock on pendingMarkDeleteOps that Thread A already holds, while Thread A is waiting for the read lock that Thread B needs to release.

In code, the deadlock appears as follows:

Thread A: synchronized (pendingMarkDeleteOps) -> lock.readLock().lock() (waiting)
Thread B: lock.writeLock().lock() -> synchronized (pendingMarkDeleteOps) (waiting)

Are you willing to submit a PR?

  • I'm willing to submit a PR!

Metadata

Metadata

Assignees

Labels

type/bugThe PR fixed a bug or issue reported a bug

Type

No type
No fields configured for issues without a type.

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions