Skip to content

Conversation

@lhotari
Copy link
Member

@lhotari lhotari commented Jun 8, 2022

Motivation

The broker can dead lock when skipping the updating of mark delete position happens. PR #15067 added the logic to skip updating mark delete position if there's already a later mark delete position. That logic introduced this particular deadlock issue.

Example stack traces of a deadlock (lines of code from fork based on branch-2.8):

"BookKeeperClientWorker-OrderedExecutor-5-0" Id=59 in BLOCKED on lock=org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers@5592e5cc
     owned by pulsar-io-4-6 Id=168
    at app//org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.markDeletePositionMoveForward(PersistentStickyKeyDispatcherMultipleConsumers.java:395)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription.notifyTheMarkDeletePositionMoveForwardIfNeeded(PersistentSubscription.java:552)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription.access$500(PersistentSubscription.java:86)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription$3.deleteComplete(PersistentSubscription.java:539)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$19.markDeleteComplete(ManagedCursorImpl.java:2122)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$MarkDeleteEntry.triggerComplete(ManagedCursorImpl.java:234)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$17.operationComplete(ManagedCursorImpl.java:1896)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.lambda$persistPositionToLedger$25(ManagedCursorImpl.java:2712)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$$Lambda$761/0x0000000840837840.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:431)
    at app//org.apache.bookkeeper.client.LedgerHandle.sendAddSuccessCallbacks(LedgerHandle.java:1832)
    at app//org.apache.bookkeeper.client.PendingAddOp.sendAddSuccessCallbacks(PendingAddOp.java:415)
    at app//org.apache.bookkeeper.client.PendingAddOp.writeComplete(PendingAddOp.java:409)
    at app//org.apache.bookkeeper.proto.PerChannelBookieClient$AddCompletion.writeComplete(PerChannelBookieClient.java:2151)

"pulsar-io-4-6" Id=168 in BLOCKED on lock=java.util.ArrayDeque@4efce20f
     owned by pulsar-io-4-8 Id=170
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.readOperationCompleted(ManagedCursorImpl.java:2848)
    at app//org.apache.bookkeeper.mledger.impl.OpReadEntry.checkReadCompletion(OpReadEntry.java:152)
    at app//org.apache.bookkeeper.mledger.impl.OpReadEntry.readEntriesComplete(OpReadEntry.java:87)
    at app//org.apache.bookkeeper.mledger.impl.EntryCacheImpl.asyncReadEntry0(EntryCacheImpl.java:293)
    at app//org.apache.bookkeeper.mledger.impl.EntryCacheImpl.asyncReadEntry(EntryCacheImpl.java:251)
    at app//org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncReadEntry(ManagedLedgerImpl.java:1935)
    at app//org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.internalReadFromLedger(ManagedLedgerImpl.java:1907)
    at app//org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncReadEntries(ManagedLedgerImpl.java:1707)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntries(ManagedCursorImpl.java:659)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntries(ManagedCursorImpl.java:642)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntriesOrWait(ManagedCursorImpl.java:793)
    at app//org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readMoreEntries(PersistentDispatcherMultipleConsumers.java:273)
      - locked org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers@5592e5cc
    at app//org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.consumerFlow(PersistentDispatcherMultipleConsumers.java:222)
      - locked org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers@5592e5cc
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription.consumerFlow(PersistentSubscription.java:369)
    at app//org.apache.pulsar.broker.service.Consumer.flowPermits(Consumer.java:645)
    at app//org.apache.pulsar.broker.service.ServerCnx.handleFlow(ServerCnx.java:1466)
    at app//org.apache.pulsar.common.protocol.PulsarDecoder.channelRead(PulsarDecoder.java:181)

"pulsar-io-4-8" Id=170 in BLOCKED on lock=org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers@5592e5cc
     owned by pulsar-io-4-6 Id=168
    at app//org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.markDeletePositionMoveForward(PersistentStickyKeyDispatcherMultipleConsumers.java:395)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription.notifyTheMarkDeletePositionMoveForwardIfNeeded(PersistentSubscription.java:552)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription.access$500(PersistentSubscription.java:86)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription$3.deleteComplete(PersistentSubscription.java:539)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$19.markDeleteComplete(ManagedCursorImpl.java:2122)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$MarkDeleteEntry.triggerComplete(ManagedCursorImpl.java:234)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.internalMarkDelete(ManagedCursorImpl.java:1847)
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.internalAsyncMarkDelete(ManagedCursorImpl.java:1810)
      - locked java.util.ArrayDeque@4efce20f
    at app//org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncDelete(ManagedCursorImpl.java:2119)
    at app//org.apache.pulsar.broker.service.persistent.PersistentSubscription.acknowledgeMessage(PersistentSubscription.java:394)
    at app//org.apache.pulsar.broker.service.Consumer.individualAckNormal(Consumer.java:442)
    at app//org.apache.pulsar.broker.service.Consumer.messageAcked(Consumer.java:392)
    at app//org.apache.pulsar.broker.service.ServerCnx.handleAck(ServerCnx.java:1437)
    at app//org.apache.pulsar.common.protocol.PulsarDecoder.channelRead(PulsarDecoder.java:144)

Modifications

  • when skipping updating mark delete position, execute callback with executor to prevent deadlock

@github-actions
Copy link

github-actions bot commented Jun 8, 2022

@lhotari:Thanks for your contribution. For this PR, do we need to update docs?
(The PR template contains info about doc, which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)

@lhotari lhotari added the doc-not-needed Your PR changes do not impact docs label Jun 8, 2022
@lhotari lhotari requested a review from dlg99 June 8, 2022 07:31
@github-actions
Copy link

github-actions bot commented Jun 8, 2022

@lhotari:Thanks for providing doc info!

Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great catch!

@nicoloboschi
Copy link
Contributor

/pulsarbot rerun-failure-checks

@eolivelli eolivelli merged commit bbc404b into apache:master Jun 8, 2022
nicoloboschi pushed a commit to datastax/pulsar that referenced this pull request Jun 8, 2022
…th executor to prevent deadlock (apache#15971)

(cherry picked from commit bbc404b)
lhotari added a commit that referenced this pull request Jun 9, 2022
…th executor to prevent deadlock (#15971)

(cherry picked from commit bbc404b)
lhotari added a commit that referenced this pull request Jun 9, 2022
…th executor to prevent deadlock (#15971)

(cherry picked from commit bbc404b)
lhotari added a commit that referenced this pull request Jun 9, 2022
…th executor to prevent deadlock (#15971)

(cherry picked from commit bbc404b)
@lhotari lhotari added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Jun 9, 2022
@lhotari lhotari deleted the lh-fix-skipping-updating-mark-delete-position-deadlock branch June 9, 2022 10:25
lhotari added a commit to datastax/pulsar that referenced this pull request Jun 13, 2022
…th executor to prevent deadlock (apache#15971)

(cherry picked from commit bbc404b)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants