Skip to content

[improve][broker] Implementing delayed message cancellation in pulsar #23907

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.NavigableSet;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.mledger.Position;
import org.apache.pulsar.broker.delayed.bucket.DelayedOperationType;

/**
* Represent the tracker for the delayed delivery of messages for a particular subscription.
Expand Down Expand Up @@ -74,6 +75,23 @@ public interface DelayedDeliveryTracker extends AutoCloseable {
*/
void resetTickTime(long tickTime);

/**
* Apply a delayed operation for a specific message. {@link DelayedOperationType}
* <p>
* For {@link DelayedOperationType#DELAY} type:
* - Adds the message to the tracker with its scheduled delivery time.
* <p>
* For {@link DelayedOperationType#CANCEL} type:
* - Removes the message from the tracker, preventing it from being delivered later.
*
* @param ledgerId the ledger ID of the message
* @param entryId the entry ID of the message
* @param deliverAt the scheduled delivery time (in milliseconds) for DELAY type, or ignored for CANCEL
* @param operationType the type of operation (DELAY/CANCEL)
* @return true if the operation was successfully applied, false if the message was already canceled or not found
*/
boolean applyDelayOperation(long ledgerId, long entryId, long deliverAt, DelayedOperationType operationType);

/**
* Clear all delayed messages from the tracker.
*
Expand Down Expand Up @@ -122,6 +140,11 @@ public void resetTickTime(long tickTime) {

}

@Override
public boolean applyDelayOperation(long ledgerId, long entryId, long deliverAt, DelayedOperationType opType) {
return false;
}

@Override
public CompletableFuture<Void> clear() {
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,14 @@
import it.unimi.dsi.fastutil.longs.LongSet;
import java.time.Clock;
import java.util.NavigableSet;
import java.util.Objects;
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.PositionFactory;
import org.apache.pulsar.broker.delayed.bucket.DelayedOperationType;
import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers;
import org.roaringbitmap.longlong.Roaring64Bitmap;

Expand Down Expand Up @@ -258,6 +260,14 @@ && getNumberOfDelayedMessages() >= fixedDelayDetectionLookahead
&& !hasMessageAvailable();
}

@Override
public boolean applyDelayOperation(long ledgerId, long entryId, long deliverAt, DelayedOperationType opType) {
if (Objects.requireNonNull(opType) == DelayedOperationType.DELAY) {
return addMessage(ledgerId, entryId, deliverAt);
}
return false;
}

protected long nextDeliveryTime() {
return delayedMessageMap.firstLongKey();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers;
import org.apache.pulsar.common.policies.data.stats.TopicMetricBean;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet;
import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
import org.roaringbitmap.RoaringBitmap;

Expand Down Expand Up @@ -105,6 +106,8 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker

private CompletableFuture<Void> pendingLoad = null;

private final ConcurrentLongPairSet canceledMessages;

public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher,
Timer timer, long tickTimeMillis,
boolean isDelayedDeliveryDeliverAtTimeStrict,
Expand Down Expand Up @@ -137,6 +140,7 @@ public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumer
bucketSnapshotStorage);
this.stats = new BucketDelayedMessageIndexStats();

this.canceledMessages = ConcurrentLongPairSet.newBuilder().autoShrink(true).build();
// Close the tracker if failed to recover.
try {
this.numberDelayedMessages = recoverBucketSnapshot();
Expand Down Expand Up @@ -206,8 +210,14 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT
this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(),
lastDelayedIndex.getEntryId(), immutableBucket);
for (DelayedIndex index : indexList) {
this.sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(),
index.getEntryId());
long ledgerId = index.getLedgerId();
long entryId = index.getEntryId();
if (index.hasDelayedOperationType()
&& index.getDelayedOperationType() == DelayedIndex.DelayedOperationType.CANCEL) {
this.canceledMessages.add(ledgerId, entryId);
} else if (!canceledMessages.contains(ledgerId, entryId)) {
this.sharedBucketPriorityQueue.add(index.getTimestamp(), ledgerId, entryId);
}
}
}
}
Expand Down Expand Up @@ -315,7 +325,7 @@ private void afterCreateImmutableBucket(Pair<ImmutableBucket, DelayedIndex> immu
immutableBucket.asyncUpdateSnapshotLength();
log.info("[{}] Create bucket snapshot finish, bucketKey: {}", dispatcher.getName(),
immutableBucket.bucketKey());

lastMutableBucket.clearCanceledOperations();
stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.create,
System.currentTimeMillis() - startTime);

Expand Down Expand Up @@ -610,6 +620,11 @@ public synchronized NavigableSet<Position> getScheduledMessages(int maxMessages)

long ledgerId = sharedBucketPriorityQueue.peekN2();
long entryId = sharedBucketPriorityQueue.peekN3();
if (canceledMessages.contains(ledgerId, entryId)) {
sharedBucketPriorityQueue.pop();
removeIndexBit(ledgerId, entryId);
continue;
}

ImmutableBucket bucket = snapshotSegmentLastIndexTable.get(ledgerId, entryId);
if (bucket != null && immutableBuckets.asMapOfRanges().containsValue(bucket)) {
Expand Down Expand Up @@ -710,6 +725,39 @@ public boolean shouldPauseAllDeliveries() {
return false;
}

public synchronized boolean applyDelayOperation(long ledgerId, long entryId, long deliverAt,
DelayedOperationType opType) {
switch (opType) {
case DELAY -> {
return addMessage(ledgerId, entryId, deliverAt);
}
case CANCEL -> {
return doCancelOperation(ledgerId, entryId, deliverAt);
}
default -> {
return false;
}
}
}

private synchronized boolean doCancelOperation(long ledgerId, long entryId, long deliverAt) {
if (containsMessage(ledgerId, entryId)) {
removeIndexBit(ledgerId, entryId);
--numberDelayedMessages;
return true;
}

if (deliverAt < 0 || deliverAt <= getCutoffTime()) {
return false;
}

long cancelAheadTime = 2 * tickTimeMillis;
long cancelTime = Math.max(clock.millis(), deliverAt - cancelAheadTime);

lastMutableBucket.addMessage(ledgerId, entryId, cancelTime, DelayedOperationType.CANCEL);
return true;
}

@Override
public synchronized CompletableFuture<Void> clear() {
CompletableFuture<Void> future = cleanImmutableBuckets();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.delayed.bucket;

import org.apache.pulsar.common.classification.InterfaceAudience;
import org.apache.pulsar.common.classification.InterfaceStability;

@InterfaceAudience.Public
@InterfaceStability.Stable
public enum DelayedOperationType {
DELAY(0),
CANCEL(1);

final int value;

DelayedOperationType(int value) {
this.value = value;
}

public int getValue() {
return this.value;
}

public static DelayedOperationType valueOf(int value) {
return switch (value) {
case 0 -> DELAY;
case 1 -> CANCEL;
default -> throw new IllegalArgumentException("Invalid value for DelayedOperationType: " + value);
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,18 +35,21 @@
import org.apache.pulsar.broker.delayed.proto.SnapshotSegment;
import org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.collections.ConcurrentLongPairSet;
import org.apache.pulsar.common.util.collections.TripleLongPriorityQueue;
import org.roaringbitmap.RoaringBitmap;

@Slf4j
class MutableBucket extends Bucket implements AutoCloseable {

private final TripleLongPriorityQueue priorityQueue;
private final ConcurrentLongPairSet canceledOperations;

MutableBucket(String dispatcherName, ManagedCursor cursor, FutureUtil.Sequencer<Void> sequencer,
BucketSnapshotStorage bucketSnapshotStorage) {
super(dispatcherName, cursor, sequencer, bucketSnapshotStorage, -1L, -1L);
this.priorityQueue = new TripleLongPriorityQueue();
this.canceledOperations = ConcurrentLongPairSet.newBuilder().autoShrink(true).build();
}

Pair<ImmutableBucket, DelayedIndex> sealBucketAndAsyncPersistent(
Expand Down Expand Up @@ -97,6 +100,12 @@ Pair<ImmutableBucket, DelayedIndex> createImmutableBucketAndAsyncPersistent(
final long ledgerId = delayedIndex.getLedgerId();
final long entryId = delayedIndex.getEntryId();

if (canceledOperations.contains(delayedIndex.getLedgerId(), delayedIndex.getEntryId())) {
delayedIndex.setDelayedOperationType(DelayedIndex.DelayedOperationType.CANCEL);
} else {
delayedIndex.setDelayedOperationType(DelayedIndex.DelayedOperationType.DELAY);
}

removeIndexBit(ledgerId, entryId);

checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId);
Expand Down Expand Up @@ -189,7 +198,9 @@ void moveScheduledMessageToSharedQueue(long cutoffTime, TripleLongPriorityQueue

long ledgerId = priorityQueue.peekN2();
long entryId = priorityQueue.peekN3();
sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
if (!canceledOperations.contains(ledgerId, entryId)) {
sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId);
}

priorityQueue.pop();
}
Expand Down Expand Up @@ -234,4 +245,20 @@ void addMessage(long ledgerId, long entryId, long deliverAt) {
this.endLedgerId = ledgerId;
putIndexBit(ledgerId, entryId);
}

void addMessage(long ledgerId, long entryId, long deliverAt, DelayedOperationType operationType) {
switch (operationType) {
case CANCEL -> {
priorityQueue.add(deliverAt, ledgerId, entryId);
canceledOperations.add(ledgerId, entryId);
putIndexBit(ledgerId, entryId);
}
case DELAY -> addMessage(ledgerId, entryId, deliverAt);
default -> throw new IllegalArgumentException("Unknown operation type: " + operationType);
}
}

void clearCanceledOperations() {
canceledOperations.clear();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,14 @@ option optimize_for = SPEED;
option java_multiple_files = true;

message DelayedIndex {
enum DelayedOperationType {
DELAY = 0;
CANCEL = 1;
}
required uint64 timestamp = 1;
required uint64 ledger_id = 2;
required uint64 entry_id = 3;
optional DelayedOperationType delayed_operation_type = 4 [default = DELAY];
}

message SnapshotSegment {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
*/
package org.apache.pulsar.broker.service.persistent;

import static org.apache.pulsar.common.naming.Constants.DELAY_CANCELED_MESSAGE_POSITION;
import static org.apache.pulsar.common.naming.Constants.IS_MARK_DELETE_DELAY_MESSAGE;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotNull;
Expand All @@ -32,6 +34,8 @@
import java.util.Set;
import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import org.apache.bookkeeper.client.BKException;
Expand All @@ -43,6 +47,7 @@
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageIdAdv;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConsumerBase;
import org.apache.pulsar.client.api.PulsarClientException;
Expand Down Expand Up @@ -686,4 +691,65 @@ public void testDelayedDeliveryExceedsMaxDelay() throws Exception {
+ maxDeliveryDelayInMillis + " milliseconds");
}
}

@Test
public void testDelayedMessageCancel() throws Exception {
String topic = BrokerTestUtil.newUniqueName("testDelayedMessageCancel");
CountDownLatch latch = new CountDownLatch(9);
Set<String> receivedMessages = ConcurrentHashMap.newKeySet();

@Cleanup
Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
.topic(topic)
.subscriptionName("shared-sub")
.subscriptionType(SubscriptionType.Shared)
.messageListener((Consumer<String> c, Message<String> msg) -> {
receivedMessages.add(msg.getValue());
c.acknowledgeAsync(msg);
latch.countDown();
})
.subscribe();

final long tickTime = 1000L;

admin.topicPolicies().setDelayedDeliveryPolicy(topic,
DelayedDeliveryPolicies.builder()
.active(true)
.tickTime(tickTime)
.maxDeliveryDelayInMillis(10000)
.build());

@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.topic(topic)
.create();

for (int i = 0; i < 10; i++) {
final int n = i;
final long currentTime = System.currentTimeMillis();
final long deliverAtTime = currentTime + 5000L;
producer.newMessage()
.key(String.valueOf(i))
.value("msg-" + i)
.deliverAt(deliverAtTime)
.sendAsync().whenComplete((id, ex) -> {
if (n == 0) {
MessageIdAdv messageIdAdv = (MessageIdAdv) id;
String deleteDelayedMessageId = messageIdAdv.getLedgerId() + ":" + messageIdAdv.getEntryId();
producer.newMessage()
.key(String.valueOf(n))
.value("msg-0-mark")
.deliverAt(deliverAtTime - 2 * tickTime)
.property(IS_MARK_DELETE_DELAY_MESSAGE, "true")
.property(DELAY_CANCELED_MESSAGE_POSITION, deleteDelayedMessageId)
.sendAsync();
}
});
}
producer.flush();

assertTrue(latch.await(15, TimeUnit.SECONDS), "Not all messages were received in time");
assertFalse(receivedMessages.contains("msg-0") || receivedMessages.contains("msg-0-mark"),
"msg-0 and msg-0-mark should have been cancelled but was received");
}
}