pulsar
pulsar copied to clipboard
PIP-195: New bucket based delayed message tracker
PIP-195: New bucket based delayed message tracker
Discussion Mailing list thread: https://lists.apache.org/thread/1krdhrvs803kb6rqzdh17q0f199nroz4 Vote Mailing list thread: https://lists.apache.org/thread/51n8kp64d16vxwh9h6klvyh1zo0owf91
Motivation
Scheduled and delayed message delivery is a widespread feature in messaging systems. Pulsar has supported delayed messages[0] in 2.4.0, which uses an in-memory delayed message tracker to track all the delayed message indexes with a priority queue. The blog "Apache pulsar delay message delivery analysis"[1] provided details of delayed messages and shared some challenges for the current implementation.
The memory limitation of the priority queue
A broker's memory is not infinite. For scenarios where users need to store many delayed messages, the in-memory priority queue might be a bottleneck for maintaining an extensive delayed index.
Suppose you want to scale the delayed message capacity. In that case, you can add more partitions so that the delayed index can be distributed to multiple brokers, but this does not change the fact that a lot of broker memory is used.
A topic might have several subscriptions, and the in-memory delayed indexes can't be used across the subscriptions; this also affects the broker's memory overhead.
Expensive delayed index rebuilding
To rebuild the delayed index, the broker needs to read all delayed messages of a topic. If there are too many delayed messages on a topic, the index rebuilding might take a long time, a few minutes to a few hours. As long as the subscription is under the delay index rebuilding situation, the consumers can't consume messages from the topic; this will bring more consumer unavailable time.
This proposal focuses on the following two major problems of delayed message implementation.
Goal
- Support delayed message index snapshot to avoid the high index rebuild costs
- Reduce the memory usage for maintaining the delayed message index
Approach
The solution is to introduce a new delayed message tracker which splits the whole delayed message index into multiple buckets based on the ledgers and an immutable snapshot for each bucket. Each bucket provides the ability to get the scheduled messages. Go through all the buckets to get scheduled messages on a topic.
Share the delayed message index across subscriptions
A topic can have several subscriptions. The current implementation is building the delayed message indexes for each subscription which will increase broker memory overhead and the overhead of replaying the log build index multiple times.
Instead, we can use a separate cursor to build the shared delayed message indexes. So that all the subscriptions under the topic can reuse the same delayed message indexes.
Any subscription that triggers the delayed message checking will poll the message IDs from the delayed message tracker. But unlike the current implementation, the scheduled message IDs need to add to the replay queue of all subscriptions. The dispatcher of the subscription will take care of the newly added message IDs and perform the message delivery.
The subscriptions have different mark delete positions. If the scheduled messages are before the mark delete position, the cursor read operation will filter out them.
A risk here is the scheduled messages will remove from the delayed message tracker. If the broker crashes before delivering the scheduled messages to consumers, the messages will not add back to the delayed tracker again. So the broker will not redeliver the messages to consumers. It’s related to the delayed message filtering and will introduce in the following sections.
Delayed message index bucket
The delayed message index bucket contains the indexes of a couple of Ledgers. Each bucket mainly includes two parts, the Bitset for each Ledger for checking if a message ID is a delayed message (contained by the delayed message index) and the priority queue for getting the scheduled messages.
A topic can have multiple delayed message index buckets, and the maximum number of buckets is configurable. The delayed message tracker will load the first segment(This part will be introduced later, one segment will map to an entry of the bucket snapshot) of each bucket to a shared priority queue. To get the topic’s scheduled messages by poll messages from the shared priority queue. After all the messages of a bucket segment have been processed, load the next segment of this bucket.
The delayed message tracker contains a special bucket (LastMutableBucket), it records the current last ledger range delayed message index by using an extra priority queue(last mutable delayed message priority queue) in fact, that priority queue reused the previous queue of InMemoryDelayedDeliveryTracker
. When the tracker receives a message ID of ledgerId > LastMutableBucket.endLegerId
tracker will create an immutable bucket and clear LastMutableDelayedMessagePriorityQueue
. The delayed message tracker will move scheduled messages from LastMutableDelayedMessagePriorityQueue
to the shared delayed message queue when regular task is triggered or poll the message IDs from the delayed message tracker.
The dispatcher of a subscription reads messages by the cursor to dispatch messages to consumers. For the delayed messages, the cursor needs to filter out them based on the delayed message index bucket. For example, if we have ten messages [0, 9], messages [1,8] are delayed. The cursor should only read messages 0 and 9 from the bookies. Note that the current implementation reads all ten messages and filters [1,8] out in the broker, which we need to improve.
So if the messages are not in the delayed message tracker and reach the delivery time, the broker can dispatch the messages to the consumer directly. If the messages are not in the delayed tracker but do not reach the delivery time, the subscription just needs to skip them because they will be added back to the delayed message tracker.
Delayed message index bucket snapshot
The bucket snapshot can reduce the costs(replay all the original messages) of rebuilding the delayed message index. We can use a Ledger to store the bucket snapshot data and maintain the bucket snapshot list by the cursor(build delayed message index cursor) properties. We can know how many delayed index buckets the topic has and read the snapshot from the persistent Ledger.
The delayed index bucket snapshot data write to multiple entries according to the delivery time. We can only load the first valid entry in the memory. After all the delayed messages are scheduled, load the delayed messages from the next entry. Here we will not make any changes to the snapshot data.
The delayed index bucket snapshot data will be stored starting from Entry1, because Entry0 recorded the metadata for snapshots, then introduce that metadata.
The maxScheduleTimestamps is used to find the first snapshot entry(which has messages not reach the delivery time). The bucket will skip the snapshot entry if all the messages in the snapshot entry reach the delivery time when rebuilding the delayed message index (because the broker can dispatch the messages to the consumer directly).
The delayedMessageMaps is used to check if the message ID is present in the bucket or not. It records all BitSet pair key for the delay message indexes per snapshot. When load a snapshot in the memory, the delayed message tracker will merge the BitSet pair from the current snapshot to the last snapshot.
The number of the delayed message in the persistence bucket.
Merge message index buckets
We can configure the max buckets of a topic. If the number of buckets reaches the max buckets limitation, the newly added Ledger into the buckets will trigger the buckets merging. The delayed message tracker will merge two adjacent buckets with the least delayed messages.
Delete message index bucket
After merging buckets, the delayed message tracker will delete the old bucket. Also, when all the delayed messages of all snapshots in the bucket are scheduled then tracker will delete that bucket.
Config Changes
borker.conf
# Enable bucket based delayed message index tracker
enbaleBucketDelayedDeliveryTracker=false
# The delayed message index bucket ledgerId range step
delayedDeliveryLedgerStepPerBucket=5
# The delayed message index bucket time step in per snapshot
delayedDeliveryTimeStepPerBucketSnapshotSeconds=300
# The max number of delayed message index bucket
delayedDeliveryMaxNumBuckets=20
# Enable share delayed message index
enbaleShareDelayedMessageIndex=false
API Changes
Admin-API changes
- Add clear all delayed message bucket index snapshots of subscription level admin-api
Prerequisite
enbaleBucketDelayedDeliveryTracker
must be set totrue
.
pulsar-admin topics clear-delayed-bucket-snapshots persistent://tenant/namespace/topic --subscription my-subscription
- Add clear all delayed message bucket index snapshots of shared subscription admin-api
Prerequisite
enbaleBucketDelayedDeliveryTracker
andenbaleShareDelayedMessageIndex
must be set totrue
.
pulsar-admin topics clear-delayed-bucket-snapshots persistent://tenant/namespace/topic
Metrics Changes
We need to add some metrics to the delayed index buckets and snapshots will help users to tune the configurations.
The new metrics and change metrics:
Name | Type | Description |
---|---|---|
pulsar_delayed_delivery_tracker_bucket_count | Gauge | The number of delayed message index buckets (immutable buckets + LastMutableBucket ) |
pulsar_delayed_delivery_tracker_memory_usage | GAUGE | The total memory size allocated by InMemoryDelayedDeliveryTracker /BucketDelayedDeliveryTracker of the namespace owned by this broker (in bytes). |
pulsar_subscription_delayed | GAUGE | The total message batches (entries) are delayed for dispatching. |
pulsar_subscription_delayed_in_memory | GAUGE | The total message batches (entries) in the memory |
pulsar_subscription_delayed_in_last_mutable_queue | GAUGE | The total message batches (entries) in the last mutable delayed message index queue. |
pulsar_delayed_delivery_tracker_bucket_merge_count | Counter | The total number of merge bucket operations. |
pulsar_delayed_delivery_tracker_bucket_delete_count | Counter | The total number of delete bucket operations. |
Note: If enabled share the delayed message index across subscriptions will can't get precise metrics of subscriptions level
Implementation
- Add a new Prototbuf for bucket snapshot BucketSnapshotFormat.proto
message BucketMessage {
required uint64 timestamp = 1;
required int64 ledger_id = 2;
required int64 entry_id = 3;
}
message DelayMessageBitMap {
map<uint64, bytes> bit_map = 1;
}
message BucketSnapshot {
repeated DelayMessageBitMap delay_message_bit_maps = 1;
repeated uint64 max_schedule_timestamps = 2;
repeated BucketMessage messages = 3;
option uint64 numDelayedMessage = 4;
}
- Add a interface
BucketSnapshotStorage
to store BucketSnapshot
public interface BucketSnapshotStorage {
CompletableFuture<Long> createBucket();
CompletableFuture<Void> saveBucketAllSnapshots(Long bucketId, List<BucketSnapshotFormat.BucketSnapshot> bucketSnapshots);
CompletableFuture<BucketSnapshotFormat.BucketSnapshot> getBucketSnapshot(Long bucketId, Long snapshotEntryId);
CompletableFuture<Void> deleteBucket(Long bucketId);
void start() throws Exception;
void close() throws Exception;
}
-
Add a new delayed message tracker
BucketDelayedDeliveryTracker
and extends from InMemoryDelayedDeliveryTracker. -
Add a method
existDelayedMessage
in theDelayedDeliveryTracker
interface to filter out the delayed messages.
public interface DelayedDeliveryTracker {
//......
boolean existDelayedMessage(long ledgerId, long entryId);
}
- The cursor will filter out all delayed messages based on the
existDelayedMessage
and skip them when reading messages from bookies. The change will include make cursor and ManagedLedger support discontinuous read entries. (the change seems to be relatively large). - Use
existDelayedMessage
to avoid tracker record repeated message index, when add a message to the delayed message tracker. - Use a separate cursor to build the delayed message tracker and add the scheduled message to the replay queue of all subscriptions when any subscription that triggers the delayed message checking.
- Add delayed index bucket Policies Admin API. And add clear delayed message bucket index snapshots Admin API
Compatibility
Upgrade
We can possible to enable the bucket based delayed message tracker by doing a rolling upgrade of the brokers, because the delayed message index in the old tracker only exists in the memory.
- Waiting for all broker nodes have been upgraded finish.
- Rolling enable the bucket based delayed message tracker feature of all brokers.
And we also can possible to enable share delayed message index by doing a rolling upgrade of the brokers, but the delayed message bucket index will be rebuilt and we should clean up all snapshots of subscriptions level.
- Rolling enable share delayed message index feature of all brokers.
- Clear all delayed message bucket index snapshots of subscriptions level.
Downgrade
We can possible to disable the bucket based delayed message tracker and disable share delayed message index by doing a rolling downgrade of the brokers, because the previous memory tracker can rebuild the delayed message index.
- Rolling diabale share delayed message index feature and diable share delayed message index feature of all brokers.
- Waiting for all broker nodes have been downgrade finish.
- Clear all delayed message bucket index snapshots.
References
[0] delayed message delivery [1] Apache pulsar delay message delivery analysis
I would like to know when this feature will be submitted
I would like to know when this feature will be submitted
The partial submission of PR will begin once the community vote is completed.
Is there any recycle or compaction policies of Buckets? Bucket may hold a Delayed Message which is scheduled very long time later, this message can block bucket deletion, if we are unfortunate, every bucket have one such message, finally it can cuase too many buckets.
Is there any recycle or compaction policies of Buckets? Bucket may hold a Delayed Message which is scheduled very long time later, this message can block bucket deletion, if we are unfortunate, every bucket have one such message, finally it can cuase too many buckets.
We can limit the number of buckets by merging buckets. More please see: Merge message index buckets and Delete message index bucket sections.
@codelipenghui @gaoran10 @liudezhi2098 @eolivelli I have sent a vote email, please take a look. Thanks! Vote Mailing list thread: https://lists.apache.org/thread/51n8kp64d16vxwh9h6klvyh1zo0owf91
Delete message index bucket After merging buckets, the delayed message tracker will delete the old bucket. Also, when all the delayed messages of all snapshots in the bucket are scheduled then tracker will delete that bucket.
We need to explain how snapshot can be guaranteed to be deleted without being missed.
Admin-API changes for clear snapshot
Could you please add more context about why we need this API? to rebuild the index? It should be a risk if only clear the index without any cursor rewind or index rebuilding. The consumer will not be able to receive delayed messages, no?
For the metrics we are using pulsar_delayed_message_index_*
https://github.com/apache/pulsar/pull/15867/files#diff-b3f64ff76fdabb9e471435147298c3c707cbecae09023f0d86cf06f6ad78cbdaR352. It's better to keep the metrics name
consistent.
pulsar_delayed_message_index_bucket_total | Gauge
pulsar_delayed_message_index_snapshot_size_bytes | Gauge
pulsar_delayed_message_index_bucket_op_latency_ms {type="load/merge/del"} | Histogram
pulsar_delayed_message_index_bucket_op_failed_count | Counter
For the BucketSnapshotFormat.proto
Is it better to split SnapshotMetadata and SnapshotData into 2 commands? It's easier to read. The metadata message have bitset and scheduleTime, the data message only has the sorted list. And we'd better change BucketSnapshotFormat.proto
to DelayedMessageIndexBucketSnapshotFormat.proto
CompletableFuture<Void> saveBucketAllSnapshots(long bucketId, List<BucketSnapshotFormat.BucketSnapshot> bucketSnapshots);
It's a little confusing for this one. For each bucket, we should only have one snapshot, for each entry in the snapshot, we should use a different name, snapshot record or something.
CompletableFuture<Long> createBucket();
I think this one is used to create a snapshot?
CompletableFuture<Void> deleteBucket(long bucketId);
is this one used to delete a snapshot?
void start() throws Exception;
What does this method really do? Does it look like initialize()
?
BTW, It is hard to comment on the issue, it looks like we need to find a new approach to create and review the proposal.
Could you please add more context about why we need this API? to rebuild the index? It should be a risk if only clear the index without any cursor rewind or index rebuilding. The consumer will not be able to receive delayed messages, no?
Oh, It only is to clear residual data after operation downgrade, I think we can remove it and discuss the cleanup of residual data in the future.
For the metrics we are using pulsar_delayed_message_index_* https://github.com/apache/pulsar/pull/15867/files#diff-b3f64ff76fdabb9e471435147298c3c707cbecae09023f0d86cf06f6ad78cbdaR352. It's better to keep the metrics name consistent.
Ok, I improve it.
Is it better to split SnapshotMetadata and SnapshotData into 2 commands? It's easier to read. The metadata message have bitset and scheduleTime, the data message only has the sorted list. And we'd better change BucketSnapshotFormat.proto to DelayedMessageIndexBucketSnapshotFormat.proto
Ok, I agree with it.
It's a little confusing for this one. For each bucket, we should only have one snapshot, for each entry in the snapshot, we should use a different name, snapshot record or something.
CompletableFuture createBucket(); I think this one is used to create a snapshot? CompletableFuture deleteBucket(long bucketId); is this one used to delete a snapshot?
Yes, I improve method name.
What does this method really do? Does it look like initialize()?
Yes, It is a initialized function.
@codelipenghui I already improve this proposal, please review it again.
@merlimat @gaoran10 @liudezhi2098 @eolivelli @mattisonchao @Technoboy- @poorbarcode Please take a look at the vote email and help complete the vote, Thanks! Vote Mailing list thread: https://lists.apache.org/thread/51n8kp64d16vxwh9h6klvyh1zo0owf91
The issue had no activity for 30 days, mark with Stale label.
Confirmed with @coderzc . This PIP has been implemented. Close this PIP as completed
It should be noted, I haven't implemented this part of Share the delayed message index across subscriptions
yet, but this PIP still works well, although this part is not implemented, and I will consider implementing this part in the future.
Hi @coderzc thanks for introducing this great feature!
I see some PRs related to this PIP were labeled with doc-not-need
, so I want to double-check: for users, does this feature not pose any influence on usage? If it affects users, we need to add docs of “what is it” and “how to use it” at least. Thanks!
Hi @coderzc thanks for introducing this great feature!
I see some PRs related to this PIP were labeled with
doc-not-need
, so I want to double-check: for users, does this feature not pose any influence on usage? If it affects users, we need to add docs of “what is it” and “how to use it” at least. Thanks!
@Anonymitaet Thanks for your reminder, This feature is a performance improvement and does not change the API, I can submit some descriptions of this PIP to the site.