Skip to content

[Bug] The Bucket-based delayed queue has serious performance issues #24603

@keyboardbobo

Description

@keyboardbobo

Search before reporting

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

User environment

jdk:1.8
pulsar: 2.9(delayed queue cherry-pick from 4.1)

Issue Description

When I was testing delayed messages with a TPS of 100,000, the maximum delayed time reached to 6 minutes, but I switched to the InMemoryDelayedDeliveryTracker implementation and there was no delay at all. I used arthas to trace the method that affects the delay as follows:

---ts=2025-08-05 11:34:58;thread_name=BookKeeperClientWorker-OrderedExecutor-60-0;id=761;is_daemon=false;priority=5;TCCL=sun.misc.Launcher$AppClassLoader@18b4aac2 ---[1525.23451ms] org.apache.pulsar.broker.delayed.bucket.MutableBucket:createImmutableBucketAndAsyncPersistent()
+---[0.00% 0.004104ms ] org.slf4j.Logger:isDebugEnabled() #65
+---[0.00% 0.001213ms ] org.apache.pulsar.broker.delayed.bucket.DelayedIndexQueue:isEmpty() #70
+---[0.00% 6.05E-4ms ] org.apache.pulsar.broker.delayed.proto.SnapshotSegment:() #80
+---[0.00% 0.001309ms ] org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata:newBuilder() #81
+---[3.14% min=4.16E-4ms,max=0.100719ms,total=47.925984ms,count=100269] org.apache.pulsar.broker.delayed.bucket.DelayedIndexQueue:isEmpty() #86
+---[3.31% min=4.36E-4ms,max=0.078285ms,total=50.485672ms,count=100268] org.apache.pulsar.broker.delayed.bucket.DelayedIndexQueue:peekTimestamp() #87
+---[3.93% min=5.26E-4ms,max=0.09415ms,total=59.973353ms,count=100268] org.apache.pulsar.broker.delayed.proto.SnapshotSegment:addIndexe() #94
+---[18.37% min=6.61E-4ms,max=0.385861ms,total=280.154489ms,count=100268] org.apache.pulsar.broker.delayed.bucket.DelayedIndexQueue:popToObject() #95
+---[3.13% min=4.25E-4ms,max=0.077715ms,total=47.763836ms,count=100268] org.apache.pulsar.broker.delayed.proto.DelayedIndex:getLedgerId() #97
+---[3.38% min=4.15E-4ms,max=4.176822ms,total=51.522645ms,count=100268] org.apache.pulsar.broker.delayed.proto.DelayedIndex:getEntryId() #98
+---[4.17% min=5.25E-4ms,max=0.098286ms,total=63.590418ms,count=100268] org.apache.pulsar.broker.delayed.bucket.MutableBucket:removeIndexBit() #100
+---[3.01% min=4.1E-4ms,max=0.081268ms,total=45.86729ms,count=100268] com.google.common.base.Preconditions:checkArgument() #102
+---[0.24% min=4.97E-4ms,max=0.018513ms,total=3.613644ms,count=5000] org.apache.pulsar.common.util.collections.TripleLongPriorityQueue:add() #106
+---[4.19% min=4.24E-4ms,max=0.11698ms,total=63.957344ms,count=100268] org.roaringbitmap.RoaringBitmap:add() #109
+---[3.18% min=4.26E-4ms,max=0.093465ms,total=48.53501ms,count=100268] org.apache.pulsar.broker.delayed.bucket.DelayedIndexQueue:isEmpty() #113
+---[3.36% min=4.53E-4ms,max=0.079834ms,total=51.208435ms,count=100267] org.apache.pulsar.broker.delayed.bucket.DelayedIndexQueue:peekTimestamp() #113
+---[3.27% min=4.42E-4ms,max=0.202628ms,total=49.945296ms,count=100267] org.apache.pulsar.broker.delayed.proto.SnapshotSegment:getIndexesCount() #115
+---[0.00% min=9.88E-4ms,max=0.004782ms,total=0.029214ms,count=21] org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata$Builder:setMaxScheduleTimestamp() #116
+---[0.00% min=7.62E-4ms,max=0.001453ms,total=0.024201ms,count=21] org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata$Builder:setMinScheduleTimestamp() #117
+---[0.01% min=0.001099ms,max=0.006919ms,total=0.11485ms,count=21] org.roaringbitmap.RoaringBitmap:runOptimize() #125
+---[0.00% min=6.93E-4ms,max=0.001374ms,total=0.021659ms,count=21] org.roaringbitmap.RoaringBitmap:serializedSizeInBytes() #126
+---[0.01% min=0.001293ms,max=0.006299ms,total=0.119293ms,count=21] org.roaringbitmap.RoaringBitmap:serialize() #127
+---[0.00% min=0.001063ms,max=0.002327ms,total=0.040085ms,count=21] com.google.protobuf.UnsafeByteOperations:unsafeWrap() #129
+---[0.01% min=0.001763ms,max=0.004998ms,total=0.082605ms,count=21] org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata$Builder:putDelayedIndexBitMap() #129
+---[0.00% min=0.001002ms,max=0.001668ms,total=0.031553ms,count=21] org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata$Builder:build() #140
+---[0.01% min=0.001733ms,max=0.005025ms,total=0.088917ms,count=21] org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata$Builder:clear() #141
+---[0.00% min=6.68E-4ms,max=0.001178ms,total=0.019536ms,count=21] org.apache.pulsar.broker.delayed.proto.SnapshotSegment:() #144
+---[0.00% 0.002128ms ] org.apache.pulsar.broker.delayed.proto.SnapshotMetadata:newBuilder() #152
+---[0.00% 0.004728ms ] org.apache.pulsar.broker.delayed.proto.SnapshotMetadata$Builder:addAllMetadataList() #153
+---[0.00% 0.006883ms ] org.apache.pulsar.broker.delayed.proto.SnapshotMetadata$Builder:build() #154
+---[0.00% 0.013098ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:() #158
+---[0.00% 0.001101ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setCurrentSegmentEntryId() #160
+---[0.00% 0.00618ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setNumberBucketDelayedMessages() #161
+---[0.00% 0.001732ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setLastSegmentEntryId() #162
+---[0.00% 0.001285ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setFirstScheduleTimestamps() #163
+---[0.00% 0.001066ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setDelayedIndexBitMap() #164
+---[0.00% 9.01E-4ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setSnapshotSegments() #168
+---[0.00% 6.88E-4ms ] com.google.common.base.Preconditions:checkArgument() #171
+---[0.00% 6.24E-4ms ] org.apache.pulsar.broker.delayed.proto.SnapshotSegment:getIndexesCount() #173
+---[0.00% 0.001991ms ] org.apache.pulsar.broker.delayed.proto.SnapshotSegment:getIndexeAt() #173
+---[0.00% 0.001245ms ] org.apache.commons.lang3.tuple.Pair:of() #174
+---[0.03% 0.419479ms ] org.apache.pulsar.broker.delayed.bucket.MutableBucket:asyncSaveBucketSnapshot() #176
`---[0.00% 0.001589ms ] org.apache.pulsar.broker.delayed.bucket.ImmutableBucket:setSnapshotCreateFuture() #178

Error messages


Reproducing the issue

nohup bin/pulsar-perf produce --auth_plugin org.apache.pulsar.client.impl.auth.AuthenticationToken --auth-params xxxxxxxxxxxxxx -threads 5 -u pulsar://xxxx:6650 -n 4 -s 2048 -r 100000 -dr 1,180 persistent://qlm-tn/delay/topic-delay-1 >producer.log 2>&1 &

Additional information

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