Skip to content

[Bug][broker] MessageDeduplication replay timeout would cause topic loading stuck and become unavailable #23003

@TakaHiR07

Description

@TakaHiR07

Search before asking

  • I searched in the issues and found nothing similar.

Read release policy

  • 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.

Version

master

Minimal reproduce step

enter this dead loop, and topic loading keep failed.

  1. topic load
  2. MessageDeduplication replay need much time
  3. topic load timeout
  4. topic close and reload

What did you expect to see?

topic become available

What did you see instead?

The issue's root is as following:

#21540 , this pr modify that topic would be closed if 60s timeout.

#22479, this pr add a logic that takeSnapshot after MessageDeduplication replay, so that topic loading won't timeout.

#22860, this pr refactor the topic loading process. Now topic loading should not be concurrent. If topic loading would timeout, the loading process is sequentially "create -> timeout -> close -> create".

However, topic loading is still stuck. The reason is if topic loading timeout, the topic would close. However, topic close and takeSnapshot is executed concurrently, so takeSnapshot may throw exception since topic has been closed. This would result in each time we retry loading topic, we need to replaying the same entries in MessageDeduplication, and we are always 60s timeout.

The error log is :

17:49:30.300 [broker-topic-workers-OrderedExecutor-6-0] INFO org.apache.pulsar.broker.service.persistent.MessageDeduplication - [persistent://test/test/test-partition-0] Replaying 2383098 entries for deduplication
17:53:05.845 [BookKeeperClientWorker-OrderedExecutor-17-0] INFO org.apache.pulsar.broker.service.persistent.MessageDeduplication - [persistent://test/test/test-partition-0] Enabled deduplication
17:53:05.886 [bookkeeper-ml-scheduler-OrderedScheduler-5-0] WARN org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [test/test/persistent/test-partition-0] Failed to update cursor metadata for pulsar.dedup due to version conflict org.apache.pulsar.metadata.api.MetadataStoreException$BadVersionException: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /managed-ledgers/test/test/persistent/test-partition-0/pulsar.dedup
17:53:05.908 [BookKeeperClientWorker-OrderedExecutor-6-0] WARN org.apache.pulsar.broker.service.persistent.MessageDeduplication - [persistent://test/test/test-partition-0] Failed to store new deduplication snapshot at 1033467:11005. ex: org.apache.bookkeeper.mledger.ManagedLedgerException$BadVersionException: org.apache.pulsar.metadata.api.MetadataStoreException$BadVersionException: org.apache.zookeeper.KeeperException$BadVersionException: KeeperErrorCode = BadVersion for /managed-ledgers/test/test/persistent/test-partition-0/pulsar.dedup


17:53:05.974 [bookkeeper-ml-scheduler-OrderedScheduler-5-0] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [test/test/persistent/test-partition-0] Recovery for cursor pulsar.dedup completed. pos=1028289:2816218 -- todo=0
17:53:05.975 [broker-topic-workers-OrderedExecutor-3-0] INFO org.apache.pulsar.broker.service.persistent.MessageDeduplication - [persistent://test/test/test-partition-0] Replaying 2383098 entries for deduplication
17:56:06.764 [BookKeeperClientWorker-OrderedExecutor-19-0] WARN org.apache.pulsar.broker.service.persistent.MessageDeduplication - [persistent://test/test/test-partition-0] Failed to store new deduplication snapshot at 1033467:11005. ex: org.apache.bookkeeper.mledger.ManagedLedgerException$MetaStoreException: org.apache.bookkeeper.mledger.ManagedLedgerException$CursorAlreadyClosedException: pulsar.dedup cursor already closed

Anything else?

No response

Are you willing to submit a PR?

  • I'm willing to submit a PR!

Metadata

Metadata

Assignees

No one assigned

    Labels

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

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions