From 7afe161d4e9919b5c5f237a71f0d984a077a6c3f Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Wed, 29 Jan 2025 11:48:35 +0800 Subject: [PATCH 01/15] Implementing delayed message cancellation in pulsar --- .../service/AbstractBaseDispatcher.java | 49 ++++++++++++++ .../persistent/DelayedDeliveryTest.java | 66 +++++++++++++++++++ .../pulsar/common/naming/Constants.java | 2 + 3 files changed, 117 insertions(+) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index fb5c457fcc874..919b811f002c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -20,6 +20,8 @@ import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.andAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetEmpty; +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 io.netty.buffer.ByteBuf; import io.prometheus.client.Gauge; import java.util.ArrayList; @@ -28,6 +30,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAdder; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -46,10 +49,12 @@ import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.api.proto.CommandAck.AckType; +import org.apache.pulsar.common.api.proto.KeyValue; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; import org.apache.pulsar.compaction.Compactor; import org.checkerframework.checker.nullness.qual.Nullable; @@ -69,10 +74,13 @@ public abstract class AbstractBaseDispatcher extends EntryFilterSupport implemen private final LongAdder filterRejectedMsgs = new LongAdder(); private final LongAdder filterRescheduledMsgs = new LongAdder(); + protected final ConcurrentLongLongPairHashMap delayedMessageMarkDeleteMap; + protected AbstractBaseDispatcher(Subscription subscription, ServiceConfiguration serviceConfig) { super(subscription); this.serviceConfig = serviceConfig; this.dispatchThrottlingOnBatchMessageEnabled = serviceConfig.isDispatchThrottlingOnBatchMessageEnabled(); + this.delayedMessageMarkDeleteMap = ConcurrentLongLongPairHashMap.newBuilder().autoShrink(true).build(); } @@ -221,6 +229,47 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i entries.set(i, null); entry.release(); continue; + } else if (delayedMessageMarkDeleteMap.containsKey(entry.getLedgerId(), entry.getEntryId())) { + // The delayed message is marked for delete. + ConcurrentLongLongPairHashMap.LongPair markMessageId = delayedMessageMarkDeleteMap + .get(entry.getLedgerId(), entry.getEntryId()); + List deleteDelayedMessageList = new ArrayList<>(); + deleteDelayedMessageList.add(entry.getPosition()); + deleteDelayedMessageList.add(PositionFactory.create(markMessageId.first, markMessageId.second)); + + delayedMessageMarkDeleteMap.remove(entry.getLedgerId(), entry.getEntryId()); + individualAcknowledgeMessageIfNeeded(deleteDelayedMessageList, Collections.emptyMap()); + entries.set(i, null); + entry.release(); + continue; + } + + List propertiesList = msgMetadata.getPropertiesList(); + if (!propertiesList.isEmpty()) { + Map propertiesMap = propertiesList.stream() + .filter(p -> p.getKey().equals(DELAY_CANCELED_MESSAGE_POSITION) + || p.getKey().equals(IS_MARK_DELETE_DELAY_MESSAGE)) + .collect(Collectors.toMap(KeyValue::getKey, KeyValue::getValue, + (oldValue, newValue) -> newValue)); + + if (propertiesMap.containsKey(IS_MARK_DELETE_DELAY_MESSAGE)) { + if (propertiesMap.containsKey(DELAY_CANCELED_MESSAGE_POSITION)) { + String[] data = propertiesMap.get(DELAY_CANCELED_MESSAGE_POSITION).split(":"); + long ledgerId = Long.parseLong(data[0]); + long entryId = Long.parseLong(data[1]); + delayedMessageMarkDeleteMap.put(ledgerId, entryId, + entry.getLedgerId(), entry.getEntryId()); + entries.set(i, null); + entry.release(); + continue; + } else { + individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), + Collections.emptyMap()); + entries.set(i, null); + entry.release(); + continue; + } + } } if (hasFilter) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index e47857e8ec60f..df676a79e11d7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -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; @@ -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; @@ -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; @@ -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 receivedMessages = ConcurrentHashMap.newKeySet(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("shared-sub") + .subscriptionType(SubscriptionType.Shared) + .messageListener((Consumer c, Message 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 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"); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java index ab71f2a43e5d4..944a35a779566 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java @@ -24,6 +24,8 @@ public class Constants { public static final String GLOBAL_CLUSTER = "global"; + public static final String DELAY_CANCELED_MESSAGE_POSITION = "delayCanceledMsgPosition"; + public static final String IS_MARK_DELETE_DELAY_MESSAGE = "isMarkDeleteDelayMessage"; private Constants() {} } From 88ffdf9dc9d5536760d28d4f68bf87ad0941e790 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Feb 2025 00:43:06 +0800 Subject: [PATCH 02/15] feat(broker): Add support for canceling delayed messages in DelayedDeliveryTracker. --- .../delayed/DelayedDeliveryTracker.java | 23 ++++++++ .../InMemoryDelayedDeliveryTracker.java | 10 ++++ .../bucket/BucketDelayedDeliveryTracker.java | 54 +++++++++++++++++-- .../delayed/bucket/DelayedOperationType.java | 47 ++++++++++++++++ .../broker/delayed/bucket/MutableBucket.java | 29 +++++++++- .../service/AbstractBaseDispatcher.java | 49 ----------------- .../DelayedMessageIndexBucketSegment.proto | 5 ++ .../pulsar/common/naming/Constants.java | 2 - 8 files changed, 164 insertions(+), 55 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index 7c954879fe845..e0ca9252691bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -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. @@ -74,6 +75,23 @@ public interface DelayedDeliveryTracker extends AutoCloseable { */ void resetTickTime(long tickTime); + /** + * Apply a delayed operation for a specific message. {@link DelayedOperationType} + *

+ * For {@link DelayedOperationType#DELAY} type: + * - Adds the message to the tracker with its scheduled delivery time. + *

+ * 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. * @@ -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 clear() { return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 5796fcbd78550..fe5a682f3abfd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -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; @@ -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(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 08f3ae1fa6e8a..c1a390138c25e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -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; @@ -105,6 +106,8 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; + private final ConcurrentLongPairSet canceledMessages; + public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, @@ -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(); @@ -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); + } } } } @@ -315,7 +325,7 @@ private void afterCreateImmutableBucket(Pair immu immutableBucket.asyncUpdateSnapshotLength(); log.info("[{}] Create bucket snapshot finish, bucketKey: {}", dispatcher.getName(), immutableBucket.bucketKey()); - + lastMutableBucket.clearCanceledOperations(); stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.create, System.currentTimeMillis() - startTime); @@ -610,6 +620,11 @@ public synchronized NavigableSet 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)) { @@ -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 clear() { CompletableFuture future = cleanImmutableBuckets(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java new file mode 100644 index 0000000000000..242adea8cd4b1 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java @@ -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); + }; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java index 1173a401a8903..7464547d247fb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java @@ -35,6 +35,7 @@ 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; @@ -42,11 +43,13 @@ class MutableBucket extends Bucket implements AutoCloseable { private final TripleLongPriorityQueue priorityQueue; + private final ConcurrentLongPairSet canceledOperations; MutableBucket(String dispatcherName, ManagedCursor cursor, FutureUtil.Sequencer sequencer, BucketSnapshotStorage bucketSnapshotStorage) { super(dispatcherName, cursor, sequencer, bucketSnapshotStorage, -1L, -1L); this.priorityQueue = new TripleLongPriorityQueue(); + this.canceledOperations = ConcurrentLongPairSet.newBuilder().autoShrink(true).build(); } Pair sealBucketAndAsyncPersistent( @@ -97,6 +100,12 @@ Pair 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); @@ -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(); } @@ -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(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index 919b811f002c8..fb5c457fcc874 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -20,8 +20,6 @@ import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.andAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetEmpty; -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 io.netty.buffer.ByteBuf; import io.prometheus.client.Gauge; import java.util.ArrayList; @@ -30,7 +28,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAdder; -import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -49,12 +46,10 @@ import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.api.proto.CommandAck.AckType; -import org.apache.pulsar.common.api.proto.KeyValue; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; -import org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap; import org.apache.pulsar.compaction.Compactor; import org.checkerframework.checker.nullness.qual.Nullable; @@ -74,13 +69,10 @@ public abstract class AbstractBaseDispatcher extends EntryFilterSupport implemen private final LongAdder filterRejectedMsgs = new LongAdder(); private final LongAdder filterRescheduledMsgs = new LongAdder(); - protected final ConcurrentLongLongPairHashMap delayedMessageMarkDeleteMap; - protected AbstractBaseDispatcher(Subscription subscription, ServiceConfiguration serviceConfig) { super(subscription); this.serviceConfig = serviceConfig; this.dispatchThrottlingOnBatchMessageEnabled = serviceConfig.isDispatchThrottlingOnBatchMessageEnabled(); - this.delayedMessageMarkDeleteMap = ConcurrentLongLongPairHashMap.newBuilder().autoShrink(true).build(); } @@ -229,47 +221,6 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i entries.set(i, null); entry.release(); continue; - } else if (delayedMessageMarkDeleteMap.containsKey(entry.getLedgerId(), entry.getEntryId())) { - // The delayed message is marked for delete. - ConcurrentLongLongPairHashMap.LongPair markMessageId = delayedMessageMarkDeleteMap - .get(entry.getLedgerId(), entry.getEntryId()); - List deleteDelayedMessageList = new ArrayList<>(); - deleteDelayedMessageList.add(entry.getPosition()); - deleteDelayedMessageList.add(PositionFactory.create(markMessageId.first, markMessageId.second)); - - delayedMessageMarkDeleteMap.remove(entry.getLedgerId(), entry.getEntryId()); - individualAcknowledgeMessageIfNeeded(deleteDelayedMessageList, Collections.emptyMap()); - entries.set(i, null); - entry.release(); - continue; - } - - List propertiesList = msgMetadata.getPropertiesList(); - if (!propertiesList.isEmpty()) { - Map propertiesMap = propertiesList.stream() - .filter(p -> p.getKey().equals(DELAY_CANCELED_MESSAGE_POSITION) - || p.getKey().equals(IS_MARK_DELETE_DELAY_MESSAGE)) - .collect(Collectors.toMap(KeyValue::getKey, KeyValue::getValue, - (oldValue, newValue) -> newValue)); - - if (propertiesMap.containsKey(IS_MARK_DELETE_DELAY_MESSAGE)) { - if (propertiesMap.containsKey(DELAY_CANCELED_MESSAGE_POSITION)) { - String[] data = propertiesMap.get(DELAY_CANCELED_MESSAGE_POSITION).split(":"); - long ledgerId = Long.parseLong(data[0]); - long entryId = Long.parseLong(data[1]); - delayedMessageMarkDeleteMap.put(ledgerId, entryId, - entry.getLedgerId(), entry.getEntryId()); - entries.set(i, null); - entry.release(); - continue; - } else { - individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), - Collections.emptyMap()); - entries.set(i, null); - entry.release(); - continue; - } - } } if (hasFilter) { diff --git a/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto b/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto index a6ed30cfe8cd4..4f2b5af40c1ec 100644 --- a/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto +++ b/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto @@ -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 { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java index 944a35a779566..ab71f2a43e5d4 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/Constants.java @@ -24,8 +24,6 @@ public class Constants { public static final String GLOBAL_CLUSTER = "global"; - public static final String DELAY_CANCELED_MESSAGE_POSITION = "delayCanceledMsgPosition"; - public static final String IS_MARK_DELETE_DELAY_MESSAGE = "isMarkDeleteDelayMessage"; private Constants() {} } From 212f1741211a0e77b5f9ce8650fd15b8a1106d6b Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sat, 31 May 2025 19:08:27 +0800 Subject: [PATCH 03/15] add cancelDelayedMessage admin api. --- .../admin/impl/PersistentTopicsBase.java | 159 ++++++++++++++++++ .../broker/admin/v1/PersistentTopics.java | 54 ++++++ .../broker/admin/v2/PersistentTopics.java | 49 ++++++ .../delayed/DelayedDeliveryTracker.java | 2 +- .../InMemoryDelayedDeliveryTracker.java | 2 +- .../bucket/BucketDelayedDeliveryTracker.java | 3 +- .../delayed/bucket/DelayedOperationType.java | 47 ------ .../broker/delayed/bucket/MutableBucket.java | 5 +- .../pulsar/broker/service/Dispatcher.java | 15 ++ ...PersistentDispatcherMultipleConsumers.java | 25 +++ .../DelayedMessageIndexBucketSegment.proto | 9 +- .../persistent/BucketDelayedDeliveryTest.java | 62 ++++++- .../persistent/DelayedDeliveryTest.java | 65 ------- .../apache/pulsar/client/admin/Topics.java | 28 +++ .../client/admin/internal/TopicsImpl.java | 40 +++++ .../apache/pulsar/admin/cli/CmdTopics.java | 34 ++++ .../pulsar/admin/cli/TestCmdTopics.java | 40 +++++ .../common/policies/data/TopicOperation.java | 1 + 18 files changed, 516 insertions(+), 124 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index e88b1110d0ad7..6f2971ce141a0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -80,6 +80,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.AlreadyRunningException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionInvalidCursorPosition; +import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.GetStatsOptions; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.broker.service.Subscription; @@ -5478,4 +5479,162 @@ protected CompletableFuture internalGetAutoSub return null; })); } + + protected void internalCancelDelayedMessage(AsyncResponse asyncResponse, + long ledgerId, long entryId, long deliverAt, + List subscriptionNames, boolean authoritative) { + CompletableFuture validationFuture = validateTopicOperationAsync(topicName, + TopicOperation.CANCEL_DELAYED_MESSAGE); + validationFuture = validationFuture.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }); + validationFuture.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (topicName.isPartitioned()) { + internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, deliverAt, + subscriptionNames, authoritative); + } else { + if (partitionMetadata.partitions > 0) { + internalCancelDelayedMessageForPartitionedTopic(asyncResponse, partitionMetadata, + ledgerId, entryId, deliverAt, subscriptionNames); + } else { + internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, + deliverAt, subscriptionNames, authoritative); + } + } + }).exceptionally(ex -> { + if (isNot307And404Exception(ex)) { + log.error("[{}] Failed to cancel delayed message {}-{} (deliverAt: {}) on topic {}: {}", + clientAppId(), ledgerId, entryId, deliverAt, topicName, ex.getMessage(), ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); + } + + private void internalCancelDelayedMessageForPartitionedTopic(AsyncResponse asyncResponse, + PartitionedTopicMetadata partitionMetadata, + long ledgerId, long entryId, + long deliverAt, List subscriptionNames) { + final List> futures = new ArrayList<>(partitionMetadata.partitions); + PulsarAdmin admin; + try { + admin = pulsar().getAdminClient(); + } catch (PulsarServerException e) { + asyncResponse.resume(new RestException(e)); + return; + } + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + futures.add(admin + .topics() + .cancelDelayedMessageAsync(topicNamePartition.toString(), + ledgerId, entryId, deliverAt, subscriptionNames)); + } + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = FutureUtil.unwrapCompletionException(exception); + log.warn("[{}] Failed to cancel delayed message {}-{} (deliverAt: {}) on some partitions of {}: {}", + clientAppId(), ledgerId, entryId, deliverAt, topicName, t.getMessage()); + resumeAsyncResponseExceptionally(asyncResponse, t); + } else { + log.info("[{}] Successfully requested cancellation for delayed message {}-{} (deliverAt: {}) on" + + " all partitions of topic {}", + clientAppId(), ledgerId, entryId, deliverAt, topicName); + asyncResponse.resume(Response.noContent().build()); + } + return null; + }); + } + + private void internalCancelDelayedMessageForNonPartitionedTopic(AsyncResponse asyncResponse, + long ledgerId, long entryId, + long deliverAt, + List subscriptionNames, + boolean authoritative) { + validateTopicOwnershipAsync(topicName, authoritative) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(optTopic -> { + if (!(optTopic instanceof PersistentTopic)) { + throw new RestException(Status.METHOD_NOT_ALLOWED, "Cancel delayed message on a non-persistent" + + " topic is not allowed"); + } + PersistentTopic persistentTopic = (PersistentTopic) optTopic; + List subsToProcess; + if (subscriptionNames == null || subscriptionNames.isEmpty()) { + subsToProcess = persistentTopic.getSubscriptions().keySet().stream() + .filter(subName -> !subName.equals(Compactor.COMPACTION_SUBSCRIPTION) + && !SystemTopicNames.isSystemTopic(TopicName. + get(persistentTopic.getName() + "/" + subName))) + .collect(Collectors.toList()); + if (subsToProcess.isEmpty()) { + log.info("[{}] No user subscriptions found to process for cancelling delayed message on" + + " topic {}.", clientAppId(), topicName); + return CompletableFuture.completedFuture(null); + } + log.info("[{}] Cancelling delayed message {}-{} (deliverAt: {}) for all non-system" + + " subscriptions on topic {}", + clientAppId(), ledgerId, entryId, deliverAt, topicName); + } else { + subsToProcess = new ArrayList<>(subscriptionNames); + log.info("[{}] Cancelling delayed message {}-{} (deliverAt: {}) for subscriptions {} on" + + " topic {}", clientAppId(), ledgerId, entryId, deliverAt, subsToProcess, topicName); + } + List> cancelFutures = subsToProcess.stream() + .map(subName -> internalCancelDelayedMessageForSubscriptionAsync( + persistentTopic, subName, ledgerId, entryId, deliverAt)) + .collect(Collectors.toList()); + return FutureUtil.waitForAll(cancelFutures); + }) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + Throwable t = FutureUtil.unwrapCompletionException(ex); + if (isNot307And404Exception(t)) { + log.error("[{}] Error in internalCancelDelayedMessageForNonPartitionedTopic for {}: {}", + clientAppId(), topicName, t.getMessage(), t); + } + resumeAsyncResponseExceptionally(asyncResponse, t); + return null; + }); + } + + private CompletableFuture internalCancelDelayedMessageForSubscriptionAsync( + PersistentTopic topic, String subName, long ledgerId, long entryId, long deliverAt) { + Subscription sub = topic.getSubscription(subName); + if (sub == null) { + return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topic.getName(), subName))); + } + Dispatcher dispatcher = sub.getDispatcher(); + if (dispatcher == null) { + String errorMsg = String.format( + "No active dispatcher for subscription %s on topic %s. Cannot cancel delayed message %d-%d." + + " Active consumers might be required to ensure the tracker is live.", + subName, topic.getName(), ledgerId, entryId); + log.warn("[{}] {}", clientAppId(), errorMsg); + return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, errorMsg)); + } + return dispatcher.cancelDelayedMessage(ledgerId, entryId, deliverAt) + .thenCompose(cancelled -> { + if (cancelled) { + log.info("[{}] Successfully requested cancellation for delayed message {}-{} (deliverAt: {})" + + " on subscription {} of topic {}", + clientAppId(), ledgerId, entryId, deliverAt, subName, topic.getName()); + return CompletableFuture.completedFuture(null); + } else { + String errorMsg = String.format( + "Failed to cancel delayed message %d-%d with original deliverAt %d on subscription %s of" + + " topic %s. Message may not exist in tracker, already delivered/cancelled, or " + + "tracker not available/initialized.", + ledgerId, entryId, deliverAt, subName, topic.getName()); + log.warn("[{}] {}", clientAppId(), errorMsg); + return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, errorMsg)); + } + }); + } + } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index 43224248fdca0..5f2bf131c0f28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -24,6 +24,7 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import java.util.List; import java.util.Optional; import java.util.Set; import javax.ws.rs.DELETE; @@ -1134,4 +1135,57 @@ public void getReplicatedSubscriptionStatus( internalGetReplicatedSubscriptionStatus(asyncResponse, decode(encodedSubName), authoritative); } + @POST + @Path("/{property}/{cluster}/{namespace}/{topic}/cancelDelayedMessage") + @ApiOperation(hidden = true, value = "Cancel a delayed message on specified subscriptions" + + " (or all if none specified).") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace or topic or subscription does not exist"), + @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), + @ApiResponse(code = 412, message = "Failed to cancel delayed message or invalid parameters"), + @ApiResponse(code = 500, message = "Internal server error")}) + public void cancelDelayedMessage( + @Suspended final AsyncResponse asyncResponse, + @ApiParam(value = "Specify the property (tenant)", required = true) + @PathParam("property") String property, + @ApiParam(value = "Specify the cluster", required = true) + @PathParam("cluster") String cluster, + @ApiParam(value = "Specify the namespace", required = true) + @PathParam("namespace") String namespace, + @ApiParam(value = "Specify topic name", required = true) + @PathParam("topic") @Encoded String encodedTopic, + @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "Ledger ID of the target delayed message", required = true) + @QueryParam("ledgerId") long ledgerId, + @ApiParam(value = "Entry ID of the target delayed message", required = true) + @QueryParam("entryId") long entryId, + @ApiParam(value = "Original deliverAt time of the target delayed message (in milliseconds from epoch)", + required = true) + @QueryParam("deliverAt") long deliverAt, + @ApiParam(value = "List of subscription names to cancel on (comma-separated, empty or null for" + + " all subscriptions)") + @QueryParam("subscriptionNames") List subscriptionNames) { + try { + validateTopicName(property, cluster, namespace, encodedTopic); + if (ledgerId < 0 || entryId < 0 || deliverAt <= 0) { + asyncResponse.resume(new RestException(Response.Status.PRECONDITION_FAILED, + "ledgerId, entryId must be non-negative, and deliverAt must be positive.")); + return; + } + List finalSubscriptionNames = (subscriptionNames == null || subscriptionNames.isEmpty()) + ? null : subscriptionNames; + internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, deliverAt, + finalSubscriptionNames, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + } + } + } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index a8e5e7a3ce77b..de1b7a515b735 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -5015,5 +5015,54 @@ public void removeAutoSubscriptionCreation( }); } + @POST + @Path("/{tenant}/{namespace}/{topic}/cancelDelayedMessage") + @ApiOperation(value = "Cancel a delayed message on specified subscriptions (or all if none specified).") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace or topic or subscription does not exist"), + @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), + @ApiResponse(code = 412, message = "Failed to cancel delayed message or invalid parameters"), + @ApiResponse(code = 500, message = "Internal server error")}) + public void cancelDelayedMessage( + @Suspended final AsyncResponse asyncResponse, + @ApiParam(value = "Specify the tenant", required = true) + @PathParam("tenant") String tenant, + @ApiParam(value = "Specify the namespace", required = true) + @PathParam("namespace") String namespace, + @ApiParam(value = "Specify topic name", required = true) + @PathParam("topic") @Encoded String encodedTopic, + @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "Ledger ID of the target delayed message", required = true) + @QueryParam("ledgerId") long ledgerId, + @ApiParam(value = "Entry ID of the target delayed message", required = true) + @QueryParam("entryId") long entryId, + @ApiParam(value = "Original deliverAt time of the target delayed message (in milliseconds from epoch)", + required = true) + @QueryParam("deliverAt") long deliverAt, + @ApiParam(value = "List of subscription names to cancel on (empty or null for all subscriptions)") + @QueryParam("subscriptionNames") List subscriptionNames) { + try { + validateTopicName(tenant, namespace, encodedTopic); + if (ledgerId < 0 || entryId < 0 || deliverAt <= 0) { + asyncResponse.resume(new RestException(Response.Status.PRECONDITION_FAILED, + "ledgerId, entryId, and deliverAt must be positive.")); + return; + } + List finalSubscriptionNames = (subscriptionNames == null || subscriptionNames.isEmpty()) + ? null : subscriptionNames; + internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, deliverAt, + finalSubscriptionNames, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + } + } + private static final Logger log = LoggerFactory.getLogger(PersistentTopics.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index e0ca9252691bd..076c7a9cff132 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -22,7 +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; +import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; /** * Represent the tracker for the delayed delivery of messages for a particular subscription. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index fe5a682f3abfd..366d9efa87536 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -35,7 +35,7 @@ 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.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.roaringbitmap.longlong.Roaring64Bitmap; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index c1a390138c25e..d6ff04bc04c14 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -56,6 +56,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; +import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; @@ -213,7 +214,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT long ledgerId = index.getLedgerId(); long entryId = index.getEntryId(); if (index.hasDelayedOperationType() - && index.getDelayedOperationType() == DelayedIndex.DelayedOperationType.CANCEL) { + && index.getDelayedOperationType() == DelayedOperationType.CANCEL) { this.canceledMessages.add(ledgerId, entryId); } else if (!canceledMessages.contains(ledgerId, entryId)) { this.sharedBucketPriorityQueue.add(index.getTimestamp(), ledgerId, entryId); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java deleted file mode 100644 index 242adea8cd4b1..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/DelayedOperationType.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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); - }; - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java index 7464547d247fb..d74da0a8419b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; +import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.delayed.proto.SnapshotMetadata; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; import org.apache.pulsar.broker.delayed.proto.SnapshotSegmentMetadata; @@ -101,9 +102,9 @@ Pair createImmutableBucketAndAsyncPersistent( final long entryId = delayedIndex.getEntryId(); if (canceledOperations.contains(delayedIndex.getLedgerId(), delayedIndex.getEntryId())) { - delayedIndex.setDelayedOperationType(DelayedIndex.DelayedOperationType.CANCEL); + delayedIndex.setDelayedOperationType(DelayedOperationType.CANCEL); } else { - delayedIndex.setDelayedOperationType(DelayedIndex.DelayedOperationType.DELAY); + delayedIndex.setDelayedOperationType(DelayedOperationType.DELAY); } removeIndexBit(ledgerId, entryId); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index f68a9a0986b84..c01b130a217fd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -177,4 +177,19 @@ default long getFilterRescheduledMsgCount() { return 0; } + /** + * Cancels a specific delayed message for this dispatcher's subscriptions. + * + * @param ledgerId The ledger ID of the message to cancel. + * @param entryId The entry ID of the message to cancel. + * @param deliverAt The original scheduled delivery time of the message. + * @return A CompletableFuture that resolves to true if the cancellation was successfully + * requested from the tracker, false otherwise (e.g., message not found in tracker, tracker not present, + * or already delivered/cancelled). + */ + default CompletableFuture cancelDelayedMessage(long ledgerId, long entryId, long deliverAt) { + // Default for dispatchers not supporting this + return CompletableFuture.completedFuture(false); + } + } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 82b96c365072f..4b9be653a23f3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -54,6 +54,7 @@ import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; +import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -1495,6 +1496,30 @@ public boolean isClassic() { return false; } + @Override + public CompletableFuture cancelDelayedMessage(long ledgerId, long entryId, long deliverAt) { + synchronized (this) { + if (delayedDeliveryTracker.isPresent()) { + boolean result = delayedDeliveryTracker.get() + .applyDelayOperation(ledgerId, entryId, deliverAt, DelayedOperationType.CANCEL); + if (result) { + log.info("[{}] Successfully applied CANCEL operation for message {}:{} with deliverAt {} to " + + "tracker", getName(), ledgerId, entryId, deliverAt); + } else { + log.warn("[{}] Failed to apply CANCEL operation for message {}:{} with deliverAt {} to tracker. " + + "Message might not be in tracker or already processed.", + getName(), ledgerId, entryId, deliverAt); + } + return CompletableFuture.completedFuture(result); + } else { + log.warn("[{}] No delayed delivery tracker present for dispatcher. Cannot cancel message {}:{} for" + + " deliverAt {}", + getName(), ledgerId, entryId, deliverAt); + return CompletableFuture.completedFuture(false); + } + } + } + public ManagedCursor getCursor() { return cursor; } diff --git a/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto b/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto index 4f2b5af40c1ec..c6a97154f7dee 100644 --- a/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto +++ b/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto @@ -24,10 +24,6 @@ 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; @@ -37,3 +33,8 @@ message DelayedIndex { message SnapshotSegment { repeated DelayedIndex indexes = 1; } + +enum DelayedOperationType { + DELAY = 0; + CANCEL = 1; +} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 4c8e6897df3fc..7a9817bd3f6e9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -21,17 +21,19 @@ import static org.apache.bookkeeper.mledger.ManagedCursor.CURSOR_INTERNAL_PROPERTY_PREFIX; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; +import static org.testng.Assert.*; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.client.BKException; @@ -44,7 +46,9 @@ import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; @@ -462,6 +466,58 @@ public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType sub admin.topics().deletePartitionedTopic(topic); } + @Test + public void testDelayedMessageCancel() throws Exception { + String topic = BrokerTestUtil.newUniqueName("testDelayedMessageCancel"); + final String subName = "shared-sub"; + CountDownLatch latch = new CountDownLatch(9); + Set receivedMessages = ConcurrentHashMap.newKeySet(); + + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .messageListener((Consumer c, Message msg) -> { + receivedMessages.add(msg.getValue()); + c.acknowledgeAsync(msg); + latch.countDown(); + }) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + Map messageIds = new HashMap<>(); + Map delayedTimes = new HashMap<>(); + + for (int i = 0; i < 10; i++) { + final long deliverAtTime = System.currentTimeMillis() + 5000L; + MessageId messageId = producer.newMessage() + .key(String.valueOf(i)) + .value("msg-" + i) + .deliverAt(deliverAtTime) + .send(); + messageIds.put(i, messageId); + delayedTimes.put(i, deliverAtTime); + } + + final int cancelMessage = 5; + + admin.topics().cancelDelayedMessage( + topic, + ((MessageIdAdv) messageIds.get(cancelMessage)).getLedgerId(), + ((MessageIdAdv) messageIds.get(cancelMessage)).getEntryId(), + delayedTimes.get(cancelMessage), + Collections.singletonList(subName) + ); + + assertTrue(latch.await(20, TimeUnit.SECONDS), "Not all messages were received in time"); + assertFalse(receivedMessages.contains("msg-" + cancelMessage) + || receivedMessages.contains("msg-" + cancelMessage + "-mark"), + "msg-0" + cancelMessage + " and msg-" + cancelMessage + "-mark should have been cancelled but was received"); + consumer.close(); + } private ManagedCursor findCursor(String topic, String subscriptionName) { PersistentTopic persistentTopic = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index df676a79e11d7..ae52c9b5d9fd4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -18,8 +18,6 @@ */ 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; @@ -34,8 +32,6 @@ 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; @@ -47,7 +43,6 @@ 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; @@ -692,64 +687,4 @@ public void testDelayedDeliveryExceedsMaxDelay() throws Exception { } } - @Test - public void testDelayedMessageCancel() throws Exception { - String topic = BrokerTestUtil.newUniqueName("testDelayedMessageCancel"); - CountDownLatch latch = new CountDownLatch(9); - Set receivedMessages = ConcurrentHashMap.newKeySet(); - - @Cleanup - Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(topic) - .subscriptionName("shared-sub") - .subscriptionType(SubscriptionType.Shared) - .messageListener((Consumer c, Message 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 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"); - } } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index c681bd1a7bca1..5bdef56ce806a 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -4558,4 +4558,32 @@ default void createShadowTopic(String shadowTopic, String sourceTopic) throws Pu default CompletableFuture createShadowTopicAsync(String shadowTopic, String sourceTopic) { return createShadowTopicAsync(shadowTopic, sourceTopic, null); } + + /** + * Cancel a delayed message on specified subscriptions (or all if none specified). + * + * @param topic The topic name + * @param ledgerId The ledger ID of the target delayed message + * @param entryId The entry ID of the target delayed message + * @param deliverAt The original deliverAt time of the target delayed message (in milliseconds from epoch) + * @param subscriptionNames List of subscription names to cancel on. + * If null or empty, applies to all subscriptions. + * @throws PulsarAdminException if the request fails + */ + void cancelDelayedMessage(String topic, long ledgerId, long entryId, long deliverAt, + List subscriptionNames) throws PulsarAdminException; + + /** + * Cancel a delayed message on specified subscriptions (or all if none specified) asynchronously. + * + * @param topic The topic name + * @param ledgerId The ledger ID of the target delayed message + * @param entryId The entry ID of the target delayed message + * @param deliverAt The original deliverAt time of the target delayed message (in milliseconds from epoch) + * @param subscriptionNames List of subscription names to cancel on. + * If null or empty, applies to all subscriptions. + * @return a future that can be used to track the asynchronous operation + */ + CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, long deliverAt, + List subscriptionNames); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index 9c4a6eef753de..b686c9dcab796 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -2819,5 +2819,45 @@ public CompletableFuture createShadowTopicAsync(String shadowTopic, String }); } + @Override + public void cancelDelayedMessage(String topic, long ledgerId, long entryId, long deliverAt, + List subscriptionNames) throws PulsarAdminException { + sync(() -> cancelDelayedMessageAsync(topic, ledgerId, entryId, deliverAt, subscriptionNames)); + } + + @Override + public CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, long deliverAt, + List subscriptionNames) { + TopicName tn = TopicName.get(topic); + WebTarget path = topicPath(tn, "cancelDelayedMessage"); + path = path.queryParam("ledgerId", ledgerId) + .queryParam("entryId", entryId) + .queryParam("deliverAt", deliverAt); + if (subscriptionNames != null && !subscriptionNames.isEmpty()) { + path = path.queryParam("subscriptionNames", subscriptionNames.toArray()); + } + final CompletableFuture future = new CompletableFuture<>(); + try { + request(path).async().post(Entity.entity("", MediaType.APPLICATION_JSON), + new InvocationCallback() { + @Override + public void completed(Response response) { + if (response.getStatusInfo().getFamily() == Response.Status.Family.SUCCESSFUL) { + future.complete(null); + } else { + future.completeExceptionally(getApiException(response)); + } + } + @Override + public void failed(Throwable throwable) { + future.completeExceptionally(getApiException(throwable)); + } + }); + } catch (PulsarAdminException e) { + future.completeExceptionally(e); + } + return future; + } + private static final Logger log = LoggerFactory.getLogger(TopicsImpl.class); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 22073b1a89dc9..7e18db532fe2f 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -275,6 +275,7 @@ public CmdTopics(Supplier admin) { addCommand("set-schema-validation-enforce", new SetSchemaValidationEnforced()); addCommand("trim-topic", new TrimTopic()); + addCommand("cancel-delayed-message", new CancelDelayedMessage()); } @Command(description = "Get the list of topics under a namespace.") @@ -3052,4 +3053,37 @@ void run() throws PulsarAdminException { getAdmin().topics().trimTopic(topic); } } + + @Command(description = "Cancel a delayed message") + private class CancelDelayedMessage extends CliCommand { + @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") + private String topicName; + + @Option(names = {"-l", "--ledgerId"}, description = "Ledger ID of the message to cancel", required = true) + private long ledgerId = -1L; + + @Option(names = {"-e", "--entryId"}, description = "Entry ID of the message to cancel", required = true) + private long entryId = -1L; + + @Option(names = {"-t", "--deliverAt"}, description = "Original scheduled delivery time" + + " (timestamp in ms from epoch) of the message to cancel", required = true) + private long deliverAt = -1L; + + @Option(names = {"-s", "--subscriptionNames"}, description = "Comma-separated list of subscription names to" + + " target. If not specified, applies to all subscriptions.", split = ",") + private List subscriptionNames = new ArrayList<>(); + + @Override + void run() throws Exception { + String topic = validateTopicName(topicName); + if (ledgerId < 0 || entryId < 0 || deliverAt <= 0) { + throw new PulsarAdminException("ledgerId, entryId must be non-negative," + + " and deliverAt must be positive."); + } + getAdmin().topics().cancelDelayedMessage(topic, ledgerId, entryId, deliverAt, subscriptionNames); + print("Successfully requested cancellation for delayed message " + ledgerId + ":" + entryId + + " with deliverAt " + deliverAt + " on topic " + topic); + } + } + } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index bd926edc5a808..7e25bf0ee1db8 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -267,4 +267,44 @@ public void testSetRetentionCmd() throws Exception { verify(mockTopics, times(1)).setRetention("persistent://public/default/topic", new RetentionPolicies(200 * 24 * 60, 2 * 1024 * 1024)); } + + @Test + public void testCancelDelayedMessage() throws Exception { + String topic = "persistent://public/default/testCancelDelayed"; + long ledgerId = 123L; + long entryId = 45L; + long deliverAt = System.currentTimeMillis() + 10000; + + // Test case 1: No specific subscriptions (should apply to all) + cmdTopics.run(new String[]{ + "cancel-delayed-message", topic, + "-l", String.valueOf(ledgerId), + "-e", String.valueOf(entryId), + "-t", String.valueOf(deliverAt) + }); + verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), + eq(deliverAt), eq(new ArrayList<>())); + + // Test case 2: Specific subscriptions + List subs = Lists.newArrayList("sub1", "sub2"); + cmdTopics.run(new String[]{ + "cancel-delayed-message", topic, + "-l", String.valueOf(ledgerId), + "-e", String.valueOf(entryId), + "-t", String.valueOf(deliverAt), + "-s", "sub1", "-s", "sub2" + }); + verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(deliverAt), eq(subs)); + + // Test case 3: Single specific subscription + List singleSub = Lists.newArrayList("sub-single"); + cmdTopics.run(new String[]{ + "cancel-delayed-message", topic, + "-l", String.valueOf(ledgerId), + "-e", String.valueOf(entryId), + "-t", String.valueOf(deliverAt), + "-s", "sub-single" + }); + verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(deliverAt), eq(singleSub)); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java index 4c74ffcd8f085..b7b840cff2180 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java @@ -34,6 +34,7 @@ public enum TopicOperation { SKIP, TERMINATE, UNLOAD, + CANCEL_DELAYED_MESSAGE, GRANT_PERMISSION, GET_PERMISSION, From 6fe2b118173af4f7a5c4d682c222b17b61341517 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 1 Jun 2025 14:06:44 +0800 Subject: [PATCH 04/15] the cancel command does not need to be added to the sharedBucketPriorityQueue. --- .../bucket/BucketDelayedDeliveryTracker.java | 6 ++- .../broker/delayed/bucket/MutableBucket.java | 8 ++-- .../BucketDelayedDeliveryTrackerTest.java | 43 ++++++++++++++++++- .../persistent/BucketDelayedDeliveryTest.java | 43 ++++++++++++------- 4 files changed, 80 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 901a327934104..1bc0f354ddba2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -623,7 +623,9 @@ public synchronized NavigableSet getScheduledMessages(int maxMessages) long entryId = sharedBucketPriorityQueue.peekN3(); if (canceledMessages.contains(ledgerId, entryId)) { sharedBucketPriorityQueue.pop(); - removeIndexBit(ledgerId, entryId); + if (removeIndexBit(ledgerId, entryId)) { + --numberDelayedMessages; + } continue; } @@ -744,6 +746,7 @@ public synchronized boolean applyDelayOperation(long ledgerId, long entryId, lon private synchronized boolean doCancelOperation(long ledgerId, long entryId, long deliverAt) { if (containsMessage(ledgerId, entryId)) { removeIndexBit(ledgerId, entryId); + canceledMessages.add(ledgerId, entryId); --numberDelayedMessages; return true; } @@ -766,6 +769,7 @@ public synchronized CompletableFuture clear() { lastMutableBucket.clear(); snapshotSegmentLastIndexTable.clear(); numberDelayedMessages = 0; + canceledMessages.clear(); return future; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java index d74da0a8419b4..79c11fc4e046c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java @@ -112,8 +112,10 @@ Pair createImmutableBucketAndAsyncPersistent( checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId); // Move first segment of bucket snapshot to sharedBucketPriorityQueue - if (segmentMetadataList.size() == 0) { - sharedQueue.add(timestamp, ledgerId, entryId); + if (segmentMetadataList.isEmpty()) { + if (!canceledOperations.contains(ledgerId, entryId)) { + sharedQueue.add(timestamp, ledgerId, entryId); + } } bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1); @@ -216,6 +218,7 @@ void clear() { this.resetLastMutableBucketRange(); this.delayedIndexBitMap.clear(); this.priorityQueue.clear(); + this.canceledOperations.clear(); } public void close() { @@ -252,7 +255,6 @@ void addMessage(long ledgerId, long entryId, long deliverAt, DelayedOperationTyp 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); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index b41a3c6974109..b76a0ab15547c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -51,6 +51,7 @@ import org.apache.pulsar.broker.delayed.AbstractDeliveryTrackerTest; import org.apache.pulsar.broker.delayed.MockBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.MockManagedCursor; +import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.roaringbitmap.RoaringBitmap; @@ -87,7 +88,7 @@ public Object[][] provider(Method method) throws Exception { final String methodName = method.getName(); return switch (methodName) { - case "test" -> new Object[][]{{ + case "test", "testCancelExistingDelayedMessage" -> new Object[][]{{ new BucketDelayedDeliveryTracker(dispatcher, timer, 1, clock, false, bucketSnapshotStorage, 5, TimeUnit.MILLISECONDS.toMillis(10), -1, 50) }}; @@ -463,4 +464,44 @@ public void testClear(BucketDelayedDeliveryTracker tracker) tracker.close(); } + + @Test(dataProvider = "delayedTracker") + public void testCancelExistingDelayedMessage(BucketDelayedDeliveryTracker tracker) { + // tickTimeMillis is 1 for the "test" case in the dataProvider, which this test method will use. + long tickTimeMillis = 1; + long targetLedgerId = 1L; + long targetEntryId = 1L; + long now = clockTime.get(); // Initial time, likely 0 if clockTime is fresh for this test run. + long originalDeliverAt = now + 200; // Target message delivers in 200ms relative to current clock + // 1. Add the target message + assertTrue(tracker.addMessage(targetLedgerId, targetEntryId, originalDeliverAt), "Failed to add target message."); + assertEquals(tracker.getNumberOfDelayedMessages(), 1, "Should have 1 message after adding target."); + assertTrue(tracker.containsMessage(targetLedgerId, targetEntryId), "Tracker should contain the target message before cancel."); + // 2. Cancel the existing target message. + // The 'deliverAt' parameter for CANCEL operation is the original delivery time of the target message. + // The tracker will internally calculate when the cancel command itself should be processed. + boolean cancelResult = tracker.applyDelayOperation(targetLedgerId, targetEntryId, originalDeliverAt, DelayedOperationType.CANCEL); + assertTrue(cancelResult, "applyDelayOperation for CANCEL should return true for an existing message."); + // Since the message existed (in mutable or immutable bucket), it should be directly removed by doCancelOperation. + // This means its bit in the corresponding RoaringBitmap is removed, and numberOfDelayedMessages is decremented. + assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Should have 0 messages after cancellation of an existing message."); + assertFalse("Tracker should not contain the target message after cancellation.", tracker.containsMessage(targetLedgerId, targetEntryId)); + // 3. Advance time and verify it's not delivered. + // Advance time to just before the original delivery time. + // Calling getScheduledMessages() can trigger internal processing, including moving messages + // from mutable to shared queue, and loading from immutable buckets if necessary. + clockTime.set(originalDeliverAt - (2 * tickTimeMillis)); // A time well before original delivery, but after any cancel command would typically be scheduled. + Set messagesJustBefore = tracker.getScheduledMessages(10); + assertTrue(messagesJustBefore.isEmpty(), "No messages should be scheduled just before original delivery time if cancelled."); + assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Number of messages should still be 0 if cancellation was effective."); + // Advance time past the original delivery time + clockTime.set(originalDeliverAt + tickTimeMillis); + Set messagesAfter = tracker.getScheduledMessages(10); + assertTrue(messagesAfter.isEmpty(), "Target message should not be delivered after its original_deliver_at time if cancelled."); + // Final state check + assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Number of messages should remain 0 well after cancellation and original delivery time."); + assertFalse("Tracker should definitely not contain the cancelled message later.", tracker.containsMessage(targetLedgerId, targetEntryId)); + tracker.close(); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 7a9817bd3f6e9..c4506ed8a0d16 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -48,10 +49,11 @@ import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -468,11 +470,12 @@ public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType sub @Test public void testDelayedMessageCancel() throws Exception { - String topic = BrokerTestUtil.newUniqueName("testDelayedMessageCancel"); + String topic = BrokerTestUtil.newUniqueName("persistent://public/default/testDelayedMessageCancel"); final String subName = "shared-sub"; CountDownLatch latch = new CountDownLatch(9); Set receivedMessages = ConcurrentHashMap.newKeySet(); + @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topic) .subscriptionName(subName) @@ -484,39 +487,49 @@ public void testDelayedMessageCancel() throws Exception { }) .subscribe(); + admin.topicPolicies().setDelayedDeliveryPolicy(topic, + DelayedDeliveryPolicies.builder() + .active(true) + .tickTime(1000L) + .maxDeliveryDelayInMillis(10000) + .build()); + + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .topic(topic) .create(); - Map messageIds = new HashMap<>(); - Map delayedTimes = new HashMap<>(); + List messageIds = new ArrayList<>(); + List delayedTimes = new ArrayList<>(); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 100; i++) { final long deliverAtTime = System.currentTimeMillis() + 5000L; MessageId messageId = producer.newMessage() .key(String.valueOf(i)) .value("msg-" + i) .deliverAt(deliverAtTime) .send(); - messageIds.put(i, messageId); - delayedTimes.put(i, deliverAtTime); + messageIds.add(i, messageId); + delayedTimes.add(i, deliverAtTime); } - final int cancelMessage = 5; + final int cancelMessage = 50; + MessageIdImpl messageId = (MessageIdImpl) messageIds.get(cancelMessage); + + Thread.sleep(1000L); admin.topics().cancelDelayedMessage( topic, - ((MessageIdAdv) messageIds.get(cancelMessage)).getLedgerId(), - ((MessageIdAdv) messageIds.get(cancelMessage)).getEntryId(), + messageId.getLedgerId(), + messageId.getEntryId(), delayedTimes.get(cancelMessage), - Collections.singletonList(subName) + // Collections.singletonList(subName) + Collections.emptyList() ); assertTrue(latch.await(20, TimeUnit.SECONDS), "Not all messages were received in time"); - assertFalse(receivedMessages.contains("msg-" + cancelMessage) - || receivedMessages.contains("msg-" + cancelMessage + "-mark"), - "msg-0" + cancelMessage + " and msg-" + cancelMessage + "-mark should have been cancelled but was received"); - consumer.close(); + assertFalse(receivedMessages.contains("msg-" + cancelMessage), + "msg-" + cancelMessage + " should have been cancelled but was received"); } private ManagedCursor findCursor(String topic, String subscriptionName) { From dc079a675185e34abeb31fbea2eaec7778e7f7a4 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 1 Jun 2025 17:04:38 +0800 Subject: [PATCH 05/15] fix test. --- .../BucketDelayedDeliveryTrackerTest.java | 25 ++++++----------- .../persistent/BucketDelayedDeliveryTest.java | 28 +++++++++++++++---- 2 files changed, 31 insertions(+), 22 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index b76a0ab15547c..e6d17ae87ac09 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -467,38 +467,31 @@ public void testClear(BucketDelayedDeliveryTracker tracker) @Test(dataProvider = "delayedTracker") public void testCancelExistingDelayedMessage(BucketDelayedDeliveryTracker tracker) { - // tickTimeMillis is 1 for the "test" case in the dataProvider, which this test method will use. long tickTimeMillis = 1; long targetLedgerId = 1L; long targetEntryId = 1L; - long now = clockTime.get(); // Initial time, likely 0 if clockTime is fresh for this test run. - long originalDeliverAt = now + 200; // Target message delivers in 200ms relative to current clock - // 1. Add the target message + long now = clockTime.get(); + long originalDeliverAt = now + 200; + assertTrue(tracker.addMessage(targetLedgerId, targetEntryId, originalDeliverAt), "Failed to add target message."); assertEquals(tracker.getNumberOfDelayedMessages(), 1, "Should have 1 message after adding target."); assertTrue(tracker.containsMessage(targetLedgerId, targetEntryId), "Tracker should contain the target message before cancel."); - // 2. Cancel the existing target message. - // The 'deliverAt' parameter for CANCEL operation is the original delivery time of the target message. - // The tracker will internally calculate when the cancel command itself should be processed. + boolean cancelResult = tracker.applyDelayOperation(targetLedgerId, targetEntryId, originalDeliverAt, DelayedOperationType.CANCEL); assertTrue(cancelResult, "applyDelayOperation for CANCEL should return true for an existing message."); - // Since the message existed (in mutable or immutable bucket), it should be directly removed by doCancelOperation. - // This means its bit in the corresponding RoaringBitmap is removed, and numberOfDelayedMessages is decremented. + assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Should have 0 messages after cancellation of an existing message."); assertFalse("Tracker should not contain the target message after cancellation.", tracker.containsMessage(targetLedgerId, targetEntryId)); - // 3. Advance time and verify it's not delivered. - // Advance time to just before the original delivery time. - // Calling getScheduledMessages() can trigger internal processing, including moving messages - // from mutable to shared queue, and loading from immutable buckets if necessary. - clockTime.set(originalDeliverAt - (2 * tickTimeMillis)); // A time well before original delivery, but after any cancel command would typically be scheduled. + + clockTime.set(originalDeliverAt - (2 * tickTimeMillis)); Set messagesJustBefore = tracker.getScheduledMessages(10); assertTrue(messagesJustBefore.isEmpty(), "No messages should be scheduled just before original delivery time if cancelled."); assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Number of messages should still be 0 if cancellation was effective."); - // Advance time past the original delivery time + clockTime.set(originalDeliverAt + tickTimeMillis); Set messagesAfter = tracker.getScheduledMessages(10); assertTrue(messagesAfter.isEmpty(), "Target message should not be delivered after its original_deliver_at time if cancelled."); - // Final state check + assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Number of messages should remain 0 well after cancellation and original delivery time."); assertFalse("Tracker should definitely not contain the cancelled message later.", tracker.containsMessage(targetLedgerId, targetEntryId)); tracker.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index c4506ed8a0d16..c963fa0e23347 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -472,16 +472,30 @@ public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType sub public void testDelayedMessageCancel() throws Exception { String topic = BrokerTestUtil.newUniqueName("persistent://public/default/testDelayedMessageCancel"); final String subName = "shared-sub"; - CountDownLatch latch = new CountDownLatch(9); - Set receivedMessages = ConcurrentHashMap.newKeySet(); + CountDownLatch latch = new CountDownLatch(99); + admin.topics().createPartitionedTopic(topic,2); + Set receivedMessages1 = ConcurrentHashMap.newKeySet(); + Set receivedMessages2 = ConcurrentHashMap.newKeySet(); @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(topic) + .topic(topic + "-partition-0") + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .messageListener((Consumer c, Message msg) -> { + receivedMessages1.add(msg.getValue()); + c.acknowledgeAsync(msg); + latch.countDown(); + }) + .subscribe(); + + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer(Schema.STRING) + .topic(topic + "-partition-1") .subscriptionName(subName) .subscriptionType(SubscriptionType.Shared) .messageListener((Consumer c, Message msg) -> { - receivedMessages.add(msg.getValue()); + receivedMessages2.add(msg.getValue()); c.acknowledgeAsync(msg); latch.countDown(); }) @@ -527,8 +541,10 @@ public void testDelayedMessageCancel() throws Exception { Collections.emptyList() ); - assertTrue(latch.await(20, TimeUnit.SECONDS), "Not all messages were received in time"); - assertFalse(receivedMessages.contains("msg-" + cancelMessage), + assertTrue(latch.await(15, TimeUnit.SECONDS), "Not all messages were received in time"); + assertFalse((receivedMessages1.contains("msg-" + cancelMessage) + || receivedMessages2.contains("msg-" + cancelMessage)) + && (receivedMessages1.size() + receivedMessages2.size() == 99), "msg-" + cancelMessage + " should have been cancelled but was received"); } From f66f1b5439f0a9809e84a1720855468ccfedc24c Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 1 Jun 2025 17:06:35 +0800 Subject: [PATCH 06/15] clean up useless canceledMessages. --- .../bucket/BucketDelayedDeliveryTracker.java | 42 ++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index 1bc0f354ddba2..f413dac1f2c87 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -108,6 +108,7 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; private final ConcurrentLongPairSet canceledMessages; + private final TripleLongPriorityQueue canceledMessagesExpiryQueue; public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, @@ -142,6 +143,7 @@ public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumer this.stats = new BucketDelayedMessageIndexStats(); this.canceledMessages = ConcurrentLongPairSet.newBuilder().autoShrink(true).build(); + this.canceledMessagesExpiryQueue = new TripleLongPriorityQueue(); // Close the tracker if failed to recover. try { this.numberDelayedMessages = recoverBucketSnapshot(); @@ -215,7 +217,7 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT long entryId = index.getEntryId(); if (index.hasDelayedOperationType() && index.getDelayedOperationType() == DelayedOperationType.CANCEL) { - this.canceledMessages.add(ledgerId, entryId); + addCanceledMessage(ledgerId, entryId, index.getTimestamp()); } else if (!canceledMessages.contains(ledgerId, entryId)) { this.sharedBucketPriorityQueue.add(index.getTimestamp(), ledgerId, entryId); } @@ -242,6 +244,12 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT return numberDelayedMessages.getValue(); } + private void addCanceledMessage(long ledgerId, long entryId, long timestamp) { + if (canceledMessages.add(ledgerId, entryId)) { + canceledMessagesExpiryQueue.add(timestamp + 2 * tickTimeMillis, ledgerId, entryId); + } + } + /** * Handle the BucketNotExistException when recover bucket snapshot entry. * The non exist bucket will be added to `toBeDeletedBucketMap` and deleted from `immutableBuckets` @@ -296,6 +304,7 @@ public void run(Timeout timeout) throws Exception { return; } lastMutableBucket.moveScheduledMessageToSharedQueue(getCutoffTime(), sharedBucketPriorityQueue); + canceledMessagesCleanup(); } super.run(timeout); } @@ -746,7 +755,7 @@ public synchronized boolean applyDelayOperation(long ledgerId, long entryId, lon private synchronized boolean doCancelOperation(long ledgerId, long entryId, long deliverAt) { if (containsMessage(ledgerId, entryId)) { removeIndexBit(ledgerId, entryId); - canceledMessages.add(ledgerId, entryId); + addCanceledMessage(ledgerId, entryId, deliverAt); --numberDelayedMessages; return true; } @@ -770,6 +779,7 @@ public synchronized CompletableFuture clear() { snapshotSegmentLastIndexTable.clear(); numberDelayedMessages = 0; canceledMessages.clear(); + canceledMessagesExpiryQueue.clear(); return future; } @@ -778,6 +788,7 @@ public synchronized void close() { super.close(); lastMutableBucket.close(); sharedBucketPriorityQueue.close(); + canceledMessagesExpiryQueue.close(); try { List> completableFutures = immutableBuckets.asMapOfRanges().values().stream() .map(bucket -> bucket.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE)).toList(); @@ -827,4 +838,31 @@ public Map genTopicMetricMap() { stats.recordBucketSnapshotSizeBytes(totalSnapshotLength.longValue()); return stats.genTopicMetricMap(); } + + private void canceledMessagesCleanup() { + try { + long now = clock.millis(); + int cleanedCount = 0; + while (!canceledMessagesExpiryQueue.isEmpty() && canceledMessagesExpiryQueue.peekN1() <= now) { + long ledgerId = canceledMessagesExpiryQueue.peekN2(); + long entryId = canceledMessagesExpiryQueue.peekN3(); + canceledMessagesExpiryQueue.pop(); + + if (canceledMessages.remove(ledgerId, entryId)) { + cleanedCount++; + if (log.isDebugEnabled()) { + log.debug("[{}] Cleaned up expired CANCEL marker for {}:{}", + dispatcher.getName(), ledgerId, entryId); + } + } + } + if (cleanedCount > 0) { + log.info("[{}] Cleaned up {} expired CANCEL command markers during regular run.", dispatcher.getName(), + cleanedCount); + } + } catch (Exception e) { + log.error("[{}] Error during CANCEL command marker cleanup in run method.", dispatcher.getName(), e); + } + } + } From 391a4262260f781f5fff54ab6c583220b14ff8c1 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 8 Jun 2025 12:48:40 +0800 Subject: [PATCH 07/15] Implement the delayed message cancellation function through acknowledge message. --- .../admin/impl/PersistentTopicsBase.java | 84 +++++++--------- .../broker/admin/v1/PersistentTopics.java | 10 +- .../broker/admin/v2/PersistentTopics.java | 9 +- .../delayed/DelayedDeliveryTracker.java | 23 ----- .../InMemoryDelayedDeliveryTracker.java | 10 -- .../bucket/BucketDelayedDeliveryTracker.java | 97 +------------------ .../broker/delayed/bucket/MutableBucket.java | 36 +------ .../pulsar/broker/service/Dispatcher.java | 15 --- .../pulsar/broker/service/Subscription.java | 13 +++ ...PersistentDispatcherMultipleConsumers.java | 25 ----- .../persistent/PersistentSubscription.java | 13 +++ .../DelayedMessageIndexBucketSegment.proto | 6 -- .../BucketDelayedDeliveryTrackerTest.java | 36 +------ .../persistent/BucketDelayedDeliveryTest.java | 29 ++---- .../persistent/DelayedDeliveryTest.java | 1 - .../apache/pulsar/client/admin/Topics.java | 6 +- .../client/admin/internal/TopicsImpl.java | 33 ++----- .../apache/pulsar/admin/cli/CmdTopics.java | 13 +-- .../pulsar/admin/cli/TestCmdTopics.java | 11 +-- 19 files changed, 98 insertions(+), 372 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 849af1dcaa39b..562c18fa11754 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -79,7 +79,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException.AlreadyRunningException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionInvalidCursorPosition; -import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.GetStatsOptions; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.broker.service.Subscription; @@ -5495,8 +5494,7 @@ protected CompletableFuture internalGetAutoSub })); } - protected void internalCancelDelayedMessage(AsyncResponse asyncResponse, - long ledgerId, long entryId, long deliverAt, + protected void internalCancelDelayedMessage(AsyncResponse asyncResponse, long ledgerId, long entryId, List subscriptionNames, boolean authoritative) { CompletableFuture validationFuture = validateTopicOperationAsync(topicName, TopicOperation.CANCEL_DELAYED_MESSAGE); @@ -5510,21 +5508,21 @@ protected void internalCancelDelayedMessage(AsyncResponse asyncResponse, validationFuture.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) .thenAccept(partitionMetadata -> { if (topicName.isPartitioned()) { - internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, deliverAt, + internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, subscriptionNames, authoritative); } else { if (partitionMetadata.partitions > 0) { internalCancelDelayedMessageForPartitionedTopic(asyncResponse, partitionMetadata, - ledgerId, entryId, deliverAt, subscriptionNames); + ledgerId, entryId, subscriptionNames); } else { internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, - deliverAt, subscriptionNames, authoritative); + subscriptionNames, authoritative); } } }).exceptionally(ex -> { if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to cancel delayed message {}-{} (deliverAt: {}) on topic {}: {}", - clientAppId(), ledgerId, entryId, deliverAt, topicName, ex.getMessage(), ex); + log.error("[{}] Failed to cancel delayed message {}-{} on topic {}: {}", + clientAppId(), ledgerId, entryId, topicName, ex.getMessage(), ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); return null; @@ -5534,7 +5532,7 @@ protected void internalCancelDelayedMessage(AsyncResponse asyncResponse, private void internalCancelDelayedMessageForPartitionedTopic(AsyncResponse asyncResponse, PartitionedTopicMetadata partitionMetadata, long ledgerId, long entryId, - long deliverAt, List subscriptionNames) { + List subscriptionNames) { final List> futures = new ArrayList<>(partitionMetadata.partitions); PulsarAdmin admin; try { @@ -5548,18 +5546,18 @@ private void internalCancelDelayedMessageForPartitionedTopic(AsyncResponse async futures.add(admin .topics() .cancelDelayedMessageAsync(topicNamePartition.toString(), - ledgerId, entryId, deliverAt, subscriptionNames)); + ledgerId, entryId, subscriptionNames)); } FutureUtil.waitForAll(futures).handle((result, exception) -> { if (exception != null) { Throwable t = FutureUtil.unwrapCompletionException(exception); - log.warn("[{}] Failed to cancel delayed message {}-{} (deliverAt: {}) on some partitions of {}: {}", - clientAppId(), ledgerId, entryId, deliverAt, topicName, t.getMessage()); + log.warn("[{}] Failed to cancel delayed message {}-{} on some partitions of {}: {}", + clientAppId(), ledgerId, entryId, topicName, t.getMessage()); resumeAsyncResponseExceptionally(asyncResponse, t); } else { - log.info("[{}] Successfully requested cancellation for delayed message {}-{} (deliverAt: {}) on" + log.info("[{}] Successfully requested cancellation for delayed message {}-{} on" + " all partitions of topic {}", - clientAppId(), ledgerId, entryId, deliverAt, topicName); + clientAppId(), ledgerId, entryId, topicName); asyncResponse.resume(Response.noContent().build()); } return null; @@ -5568,7 +5566,6 @@ private void internalCancelDelayedMessageForPartitionedTopic(AsyncResponse async private void internalCancelDelayedMessageForNonPartitionedTopic(AsyncResponse asyncResponse, long ledgerId, long entryId, - long deliverAt, List subscriptionNames, boolean authoritative) { validateTopicOwnershipAsync(topicName, authoritative) @@ -5591,17 +5588,17 @@ private void internalCancelDelayedMessageForNonPartitionedTopic(AsyncResponse as + " topic {}.", clientAppId(), topicName); return CompletableFuture.completedFuture(null); } - log.info("[{}] Cancelling delayed message {}-{} (deliverAt: {}) for all non-system" + log.info("[{}] Cancelling delayed message {}-{} for all non-system" + " subscriptions on topic {}", - clientAppId(), ledgerId, entryId, deliverAt, topicName); + clientAppId(), ledgerId, entryId, topicName); } else { - subsToProcess = new ArrayList<>(subscriptionNames); - log.info("[{}] Cancelling delayed message {}-{} (deliverAt: {}) for subscriptions {} on" - + " topic {}", clientAppId(), ledgerId, entryId, deliverAt, subsToProcess, topicName); + subsToProcess = new ArrayList<>(subscriptionNames); + log.info("[{}] Cancelling delayed message {}-{} for subscriptions {} on" + + " topic {}", clientAppId(), ledgerId, entryId, subsToProcess, topicName); } List> cancelFutures = subsToProcess.stream() .map(subName -> internalCancelDelayedMessageForSubscriptionAsync( - persistentTopic, subName, ledgerId, entryId, deliverAt)) + persistentTopic, subName, ledgerId, entryId)) .collect(Collectors.toList()); return FutureUtil.waitForAll(cancelFutures); }) @@ -5618,38 +5615,29 @@ private void internalCancelDelayedMessageForNonPartitionedTopic(AsyncResponse as } private CompletableFuture internalCancelDelayedMessageForSubscriptionAsync( - PersistentTopic topic, String subName, long ledgerId, long entryId, long deliverAt) { + PersistentTopic topic, String subName, long ledgerId, long entryId) { Subscription sub = topic.getSubscription(subName); if (sub == null) { return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, getSubNotFoundErrorMessage(topic.getName(), subName))); } - Dispatcher dispatcher = sub.getDispatcher(); - if (dispatcher == null) { - String errorMsg = String.format( - "No active dispatcher for subscription %s on topic %s. Cannot cancel delayed message %d-%d." - + " Active consumers might be required to ensure the tracker is live.", - subName, topic.getName(), ledgerId, entryId); - log.warn("[{}] {}", clientAppId(), errorMsg); - return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, errorMsg)); - } - return dispatcher.cancelDelayedMessage(ledgerId, entryId, deliverAt) - .thenCompose(cancelled -> { - if (cancelled) { - log.info("[{}] Successfully requested cancellation for delayed message {}-{} (deliverAt: {})" - + " on subscription {} of topic {}", - clientAppId(), ledgerId, entryId, deliverAt, subName, topic.getName()); - return CompletableFuture.completedFuture(null); - } else { - String errorMsg = String.format( - "Failed to cancel delayed message %d-%d with original deliverAt %d on subscription %s of" - + " topic %s. Message may not exist in tracker, already delivered/cancelled, or " - + "tracker not available/initialized.", - ledgerId, entryId, deliverAt, subName, topic.getName()); - log.warn("[{}] {}", clientAppId(), errorMsg); - return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, errorMsg)); - } - }); + return sub.cancelDelayedMessage(ledgerId, entryId) + .thenCompose(cancelled -> { + if (cancelled) { + log.info("[{}] Successfully requested cancellation for delayed message {}-{}" + + " on subscription {} of topic {}", + clientAppId(), ledgerId, entryId, subName, topic.getName()); + return CompletableFuture.completedFuture(null); + } else { + String errorMsg = String.format( + "Failed to cancel delayed message %d-%d on subscription %s" + + " of topic %s. Message may not exist in tracker, already delivered/cancelled," + + " or tracker not available/initialized.", + ledgerId, entryId, subName, topic.getName()); + log.warn("[{}] {}", clientAppId(), errorMsg); + return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, errorMsg)); + } + }); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index 5f2bf131c0f28..e4b2c1ec2d8d2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -1164,23 +1164,19 @@ public void cancelDelayedMessage( @QueryParam("ledgerId") long ledgerId, @ApiParam(value = "Entry ID of the target delayed message", required = true) @QueryParam("entryId") long entryId, - @ApiParam(value = "Original deliverAt time of the target delayed message (in milliseconds from epoch)", - required = true) - @QueryParam("deliverAt") long deliverAt, @ApiParam(value = "List of subscription names to cancel on (comma-separated, empty or null for" + " all subscriptions)") @QueryParam("subscriptionNames") List subscriptionNames) { try { validateTopicName(property, cluster, namespace, encodedTopic); - if (ledgerId < 0 || entryId < 0 || deliverAt <= 0) { + if (ledgerId < 0 || entryId < 0) { asyncResponse.resume(new RestException(Response.Status.PRECONDITION_FAILED, - "ledgerId, entryId must be non-negative, and deliverAt must be positive.")); + "ledgerId, entryId must be non-negative.")); return; } List finalSubscriptionNames = (subscriptionNames == null || subscriptionNames.isEmpty()) ? null : subscriptionNames; - internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, deliverAt, - finalSubscriptionNames, authoritative); + internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, finalSubscriptionNames, authoritative); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 338761e065dda..8ac67f63e1964 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -5041,21 +5041,18 @@ public void cancelDelayedMessage( @QueryParam("ledgerId") long ledgerId, @ApiParam(value = "Entry ID of the target delayed message", required = true) @QueryParam("entryId") long entryId, - @ApiParam(value = "Original deliverAt time of the target delayed message (in milliseconds from epoch)", - required = true) - @QueryParam("deliverAt") long deliverAt, @ApiParam(value = "List of subscription names to cancel on (empty or null for all subscriptions)") @QueryParam("subscriptionNames") List subscriptionNames) { try { validateTopicName(tenant, namespace, encodedTopic); - if (ledgerId < 0 || entryId < 0 || deliverAt <= 0) { + if (ledgerId < 0 || entryId < 0) { asyncResponse.resume(new RestException(Response.Status.PRECONDITION_FAILED, - "ledgerId, entryId, and deliverAt must be positive.")); + "ledgerId, entryId must be positive.")); return; } List finalSubscriptionNames = (subscriptionNames == null || subscriptionNames.isEmpty()) ? null : subscriptionNames; - internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, deliverAt, + internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, finalSubscriptionNames, authoritative); } catch (WebApplicationException wae) { asyncResponse.resume(wae); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index 076c7a9cff132..7c954879fe845 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -22,7 +22,6 @@ import java.util.NavigableSet; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.Position; -import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; /** * Represent the tracker for the delayed delivery of messages for a particular subscription. @@ -75,23 +74,6 @@ public interface DelayedDeliveryTracker extends AutoCloseable { */ void resetTickTime(long tickTime); - /** - * Apply a delayed operation for a specific message. {@link DelayedOperationType} - *

- * For {@link DelayedOperationType#DELAY} type: - * - Adds the message to the tracker with its scheduled delivery time. - *

- * 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. * @@ -140,11 +122,6 @@ public void resetTickTime(long tickTime) { } - @Override - public boolean applyDelayOperation(long ledgerId, long entryId, long deliverAt, DelayedOperationType opType) { - return false; - } - @Override public CompletableFuture clear() { return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java index 6da517a72a422..a48ed4161386d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java @@ -28,14 +28,12 @@ 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.proto.DelayedOperationType; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.roaringbitmap.longlong.Roaring64Bitmap; @@ -260,14 +258,6 @@ && 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(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java index f413dac1f2c87..686daa92d1311 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTracker.java @@ -56,12 +56,10 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; -import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.delayed.proto.SnapshotSegment; 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; @@ -107,9 +105,6 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; - private final ConcurrentLongPairSet canceledMessages; - private final TripleLongPriorityQueue canceledMessagesExpiryQueue; - public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, long tickTimeMillis, boolean isDelayedDeliveryDeliverAtTimeStrict, @@ -142,8 +137,6 @@ public BucketDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumer bucketSnapshotStorage); this.stats = new BucketDelayedMessageIndexStats(); - this.canceledMessages = ConcurrentLongPairSet.newBuilder().autoShrink(true).build(); - this.canceledMessagesExpiryQueue = new TripleLongPriorityQueue(); // Close the tracker if failed to recover. try { this.numberDelayedMessages = recoverBucketSnapshot(); @@ -213,14 +206,8 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT this.snapshotSegmentLastIndexTable.put(lastDelayedIndex.getLedgerId(), lastDelayedIndex.getEntryId(), immutableBucket); for (DelayedIndex index : indexList) { - long ledgerId = index.getLedgerId(); - long entryId = index.getEntryId(); - if (index.hasDelayedOperationType() - && index.getDelayedOperationType() == DelayedOperationType.CANCEL) { - addCanceledMessage(ledgerId, entryId, index.getTimestamp()); - } else if (!canceledMessages.contains(ledgerId, entryId)) { - this.sharedBucketPriorityQueue.add(index.getTimestamp(), ledgerId, entryId); - } + this.sharedBucketPriorityQueue.add(index.getTimestamp(), index.getLedgerId(), + index.getEntryId()); } } } @@ -244,12 +231,6 @@ private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryT return numberDelayedMessages.getValue(); } - private void addCanceledMessage(long ledgerId, long entryId, long timestamp) { - if (canceledMessages.add(ledgerId, entryId)) { - canceledMessagesExpiryQueue.add(timestamp + 2 * tickTimeMillis, ledgerId, entryId); - } - } - /** * Handle the BucketNotExistException when recover bucket snapshot entry. * The non exist bucket will be added to `toBeDeletedBucketMap` and deleted from `immutableBuckets` @@ -304,7 +285,6 @@ public void run(Timeout timeout) throws Exception { return; } lastMutableBucket.moveScheduledMessageToSharedQueue(getCutoffTime(), sharedBucketPriorityQueue); - canceledMessagesCleanup(); } super.run(timeout); } @@ -335,7 +315,7 @@ private void afterCreateImmutableBucket(Pair immu immutableBucket.asyncUpdateSnapshotLength(); log.info("[{}] Create bucket snapshot finish, bucketKey: {}", dispatcher.getName(), immutableBucket.bucketKey()); - lastMutableBucket.clearCanceledOperations(); + stats.recordSuccessEvent(BucketDelayedMessageIndexStats.Type.create, System.currentTimeMillis() - startTime); @@ -630,13 +610,6 @@ public synchronized NavigableSet getScheduledMessages(int maxMessages) long ledgerId = sharedBucketPriorityQueue.peekN2(); long entryId = sharedBucketPriorityQueue.peekN3(); - if (canceledMessages.contains(ledgerId, entryId)) { - sharedBucketPriorityQueue.pop(); - if (removeIndexBit(ledgerId, entryId)) { - --numberDelayedMessages; - } - continue; - } ImmutableBucket bucket = snapshotSegmentLastIndexTable.get(ledgerId, entryId); if (bucket != null && immutableBuckets.asMapOfRanges().containsValue(bucket)) { @@ -737,40 +710,6 @@ 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); - addCanceledMessage(ledgerId, entryId, deliverAt); - --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 clear() { CompletableFuture future = cleanImmutableBuckets(); @@ -778,8 +717,6 @@ public synchronized CompletableFuture clear() { lastMutableBucket.clear(); snapshotSegmentLastIndexTable.clear(); numberDelayedMessages = 0; - canceledMessages.clear(); - canceledMessagesExpiryQueue.clear(); return future; } @@ -788,7 +725,6 @@ public synchronized void close() { super.close(); lastMutableBucket.close(); sharedBucketPriorityQueue.close(); - canceledMessagesExpiryQueue.close(); try { List> completableFutures = immutableBuckets.asMapOfRanges().values().stream() .map(bucket -> bucket.getSnapshotCreateFuture().orElse(NULL_LONG_PROMISE)).toList(); @@ -838,31 +774,4 @@ public Map genTopicMetricMap() { stats.recordBucketSnapshotSizeBytes(totalSnapshotLength.longValue()); return stats.genTopicMetricMap(); } - - private void canceledMessagesCleanup() { - try { - long now = clock.millis(); - int cleanedCount = 0; - while (!canceledMessagesExpiryQueue.isEmpty() && canceledMessagesExpiryQueue.peekN1() <= now) { - long ledgerId = canceledMessagesExpiryQueue.peekN2(); - long entryId = canceledMessagesExpiryQueue.peekN3(); - canceledMessagesExpiryQueue.pop(); - - if (canceledMessages.remove(ledgerId, entryId)) { - cleanedCount++; - if (log.isDebugEnabled()) { - log.debug("[{}] Cleaned up expired CANCEL marker for {}:{}", - dispatcher.getName(), ledgerId, entryId); - } - } - } - if (cleanedCount > 0) { - log.info("[{}] Cleaned up {} expired CANCEL command markers during regular run.", dispatcher.getName(), - cleanedCount); - } - } catch (Exception e) { - log.error("[{}] Error during CANCEL command marker cleanup in run method.", dispatcher.getName(), e); - } - } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java index 79c11fc4e046c..1173a401a8903 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/MutableBucket.java @@ -31,12 +31,10 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.proto.DelayedIndex; -import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.delayed.proto.SnapshotMetadata; 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; @@ -44,13 +42,11 @@ class MutableBucket extends Bucket implements AutoCloseable { private final TripleLongPriorityQueue priorityQueue; - private final ConcurrentLongPairSet canceledOperations; MutableBucket(String dispatcherName, ManagedCursor cursor, FutureUtil.Sequencer sequencer, BucketSnapshotStorage bucketSnapshotStorage) { super(dispatcherName, cursor, sequencer, bucketSnapshotStorage, -1L, -1L); this.priorityQueue = new TripleLongPriorityQueue(); - this.canceledOperations = ConcurrentLongPairSet.newBuilder().autoShrink(true).build(); } Pair sealBucketAndAsyncPersistent( @@ -101,21 +97,13 @@ Pair createImmutableBucketAndAsyncPersistent( final long ledgerId = delayedIndex.getLedgerId(); final long entryId = delayedIndex.getEntryId(); - if (canceledOperations.contains(delayedIndex.getLedgerId(), delayedIndex.getEntryId())) { - delayedIndex.setDelayedOperationType(DelayedOperationType.CANCEL); - } else { - delayedIndex.setDelayedOperationType(DelayedOperationType.DELAY); - } - removeIndexBit(ledgerId, entryId); checkArgument(ledgerId >= startLedgerId && ledgerId <= endLedgerId); // Move first segment of bucket snapshot to sharedBucketPriorityQueue - if (segmentMetadataList.isEmpty()) { - if (!canceledOperations.contains(ledgerId, entryId)) { - sharedQueue.add(timestamp, ledgerId, entryId); - } + if (segmentMetadataList.size() == 0) { + sharedQueue.add(timestamp, ledgerId, entryId); } bitMap.computeIfAbsent(ledgerId, k -> new RoaringBitmap()).add(entryId, entryId + 1); @@ -201,9 +189,7 @@ void moveScheduledMessageToSharedQueue(long cutoffTime, TripleLongPriorityQueue long ledgerId = priorityQueue.peekN2(); long entryId = priorityQueue.peekN3(); - if (!canceledOperations.contains(ledgerId, entryId)) { - sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId); - } + sharedBucketPriorityQueue.add(timestamp, ledgerId, entryId); priorityQueue.pop(); } @@ -218,7 +204,6 @@ void clear() { this.resetLastMutableBucketRange(); this.delayedIndexBitMap.clear(); this.priorityQueue.clear(); - this.canceledOperations.clear(); } public void close() { @@ -249,19 +234,4 @@ 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); - } - case DELAY -> addMessage(ledgerId, entryId, deliverAt); - default -> throw new IllegalArgumentException("Unknown operation type: " + operationType); - } - } - - void clearCanceledOperations() { - canceledOperations.clear(); - } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java index 59924dbd5cb5c..e19deb34e31b9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Dispatcher.java @@ -225,19 +225,4 @@ default long getDispatchThrottledBytesEventsByBrokerLimit() { return 0; } - /** - * Cancels a specific delayed message for this dispatcher's subscriptions. - * - * @param ledgerId The ledger ID of the message to cancel. - * @param entryId The entry ID of the message to cancel. - * @param deliverAt The original scheduled delivery time of the message. - * @return A CompletableFuture that resolves to true if the cancellation was successfully - * requested from the tracker, false otherwise (e.g., message not found in tracker, tracker not present, - * or already delivered/cancelled). - */ - default CompletableFuture cancelDelayedMessage(long ledgerId, long entryId, long deliverAt) { - // Default for dispatchers not supporting this - return CompletableFuture.completedFuture(false); - } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 452c30b45febb..4a6eea35ec10d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -114,6 +114,19 @@ default void processReplicatedSubscriptionSnapshot(ReplicatedSubscriptionsSnapsh CompletableFuture analyzeBacklog(Optional position); + /** + * Cancels a specific delayed message for subscriptions. + * + * @param ledgerId The ledger ID of the message to cancel. + * @param entryId The entry ID of the message to cancel. + * @return A CompletableFuture that resolves to true if the cancellation was successfully + * requested from the tracker, false otherwise (e.g., message not found in tracker, tracker not present, + * or already delivered/cancelled). + */ + default CompletableFuture cancelDelayedMessage(long ledgerId, long entryId) { + return CompletableFuture.completedFuture(false); + } + default int getNumberOfSameAddressConsumers(final String clientAddress) { int count = 0; if (clientAddress != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 717829d4a2c1e..c35d802f43d54 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -54,7 +54,6 @@ import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker; import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; -import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -1473,30 +1472,6 @@ public boolean isClassic() { return false; } - @Override - public CompletableFuture cancelDelayedMessage(long ledgerId, long entryId, long deliverAt) { - synchronized (this) { - if (delayedDeliveryTracker.isPresent()) { - boolean result = delayedDeliveryTracker.get() - .applyDelayOperation(ledgerId, entryId, deliverAt, DelayedOperationType.CANCEL); - if (result) { - log.info("[{}] Successfully applied CANCEL operation for message {}:{} with deliverAt {} to " - + "tracker", getName(), ledgerId, entryId, deliverAt); - } else { - log.warn("[{}] Failed to apply CANCEL operation for message {}:{} with deliverAt {} to tracker. " - + "Message might not be in tracker or already processed.", - getName(), ledgerId, entryId, deliverAt); - } - return CompletableFuture.completedFuture(result); - } else { - log.warn("[{}] No delayed delivery tracker present for dispatcher. Cannot cancel message {}:{} for" - + " deliverAt {}", - getName(), ledgerId, entryId, deliverAt); - return CompletableFuture.completedFuture(false); - } - } - } - public ManagedCursor getCursor() { return cursor; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 97b4dc06d0837..001159a62d656 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -51,6 +51,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.ConcurrentFindCursorPositionException; import org.apache.bookkeeper.mledger.ManagedLedgerException.InvalidCursorPositionException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.tuple.MutablePair; @@ -1547,6 +1548,18 @@ public CompletableFuture endTxn(long txnidMostBits, long txnidLeastBits, i } } + @Override + public CompletableFuture cancelDelayedMessage(long ledgerId, long entryId) { + if (Subscription.isCumulativeAckMode(getType())) { + return CompletableFuture.completedFuture(false); + } + Position position = PositionFactory.create(ledgerId, entryId); + List positions = Collections.singletonList(position); + Map properties = Collections.emptyMap(); + acknowledgeMessage(positions, AckType.Individual, properties); + return CompletableFuture.completedFuture(true); + } + @VisibleForTesting public ManagedCursor getCursor() { return cursor; diff --git a/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto b/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto index c6a97154f7dee..a6ed30cfe8cd4 100644 --- a/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto +++ b/pulsar-broker/src/main/proto/DelayedMessageIndexBucketSegment.proto @@ -27,14 +27,8 @@ message DelayedIndex { required uint64 timestamp = 1; required uint64 ledger_id = 2; required uint64 entry_id = 3; - optional DelayedOperationType delayed_operation_type = 4 [default = DELAY]; } message SnapshotSegment { repeated DelayedIndex indexes = 1; } - -enum DelayedOperationType { - DELAY = 0; - CANCEL = 1; -} \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java index e6d17ae87ac09..b41a3c6974109 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/bucket/BucketDelayedDeliveryTrackerTest.java @@ -51,7 +51,6 @@ import org.apache.pulsar.broker.delayed.AbstractDeliveryTrackerTest; import org.apache.pulsar.broker.delayed.MockBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.MockManagedCursor; -import org.apache.pulsar.broker.delayed.proto.DelayedOperationType; import org.apache.pulsar.broker.service.persistent.AbstractPersistentDispatcherMultipleConsumers; import org.awaitility.Awaitility; import org.roaringbitmap.RoaringBitmap; @@ -88,7 +87,7 @@ public Object[][] provider(Method method) throws Exception { final String methodName = method.getName(); return switch (methodName) { - case "test", "testCancelExistingDelayedMessage" -> new Object[][]{{ + case "test" -> new Object[][]{{ new BucketDelayedDeliveryTracker(dispatcher, timer, 1, clock, false, bucketSnapshotStorage, 5, TimeUnit.MILLISECONDS.toMillis(10), -1, 50) }}; @@ -464,37 +463,4 @@ public void testClear(BucketDelayedDeliveryTracker tracker) tracker.close(); } - - @Test(dataProvider = "delayedTracker") - public void testCancelExistingDelayedMessage(BucketDelayedDeliveryTracker tracker) { - long tickTimeMillis = 1; - long targetLedgerId = 1L; - long targetEntryId = 1L; - long now = clockTime.get(); - long originalDeliverAt = now + 200; - - assertTrue(tracker.addMessage(targetLedgerId, targetEntryId, originalDeliverAt), "Failed to add target message."); - assertEquals(tracker.getNumberOfDelayedMessages(), 1, "Should have 1 message after adding target."); - assertTrue(tracker.containsMessage(targetLedgerId, targetEntryId), "Tracker should contain the target message before cancel."); - - boolean cancelResult = tracker.applyDelayOperation(targetLedgerId, targetEntryId, originalDeliverAt, DelayedOperationType.CANCEL); - assertTrue(cancelResult, "applyDelayOperation for CANCEL should return true for an existing message."); - - assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Should have 0 messages after cancellation of an existing message."); - assertFalse("Tracker should not contain the target message after cancellation.", tracker.containsMessage(targetLedgerId, targetEntryId)); - - clockTime.set(originalDeliverAt - (2 * tickTimeMillis)); - Set messagesJustBefore = tracker.getScheduledMessages(10); - assertTrue(messagesJustBefore.isEmpty(), "No messages should be scheduled just before original delivery time if cancelled."); - assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Number of messages should still be 0 if cancellation was effective."); - - clockTime.set(originalDeliverAt + tickTimeMillis); - Set messagesAfter = tracker.getScheduledMessages(10); - assertTrue(messagesAfter.isEmpty(), "Target message should not be delivered after its original_deliver_at time if cancelled."); - - assertEquals(tracker.getNumberOfDelayedMessages(), 0, "Number of messages should remain 0 well after cancellation and original delivery time."); - assertFalse("Tracker should definitely not contain the cancelled message later.", tracker.containsMessage(targetLedgerId, targetEntryId)); - tracker.close(); - } - } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index c963fa0e23347..2628da91f3981 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -53,7 +53,6 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -473,14 +472,14 @@ public void testDelayedMessageCancel() throws Exception { String topic = BrokerTestUtil.newUniqueName("persistent://public/default/testDelayedMessageCancel"); final String subName = "shared-sub"; CountDownLatch latch = new CountDownLatch(99); - admin.topics().createPartitionedTopic(topic,2); + admin.topics().createPartitionedTopic(topic, 2); Set receivedMessages1 = ConcurrentHashMap.newKeySet(); Set receivedMessages2 = ConcurrentHashMap.newKeySet(); @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(topic + "-partition-0") - .subscriptionName(subName) + .topic(topic) + .subscriptionName(subName + "-1") .subscriptionType(SubscriptionType.Shared) .messageListener((Consumer c, Message msg) -> { receivedMessages1.add(msg.getValue()); @@ -491,8 +490,8 @@ public void testDelayedMessageCancel() throws Exception { @Cleanup Consumer consumer2 = pulsarClient.newConsumer(Schema.STRING) - .topic(topic + "-partition-1") - .subscriptionName(subName) + .topic(topic) + .subscriptionName(subName + "-2") .subscriptionType(SubscriptionType.Shared) .messageListener((Consumer c, Message msg) -> { receivedMessages2.add(msg.getValue()); @@ -501,44 +500,32 @@ public void testDelayedMessageCancel() throws Exception { }) .subscribe(); - admin.topicPolicies().setDelayedDeliveryPolicy(topic, - DelayedDeliveryPolicies.builder() - .active(true) - .tickTime(1000L) - .maxDeliveryDelayInMillis(10000) - .build()); - @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .topic(topic) .create(); List messageIds = new ArrayList<>(); - List delayedTimes = new ArrayList<>(); for (int i = 0; i < 100; i++) { - final long deliverAtTime = System.currentTimeMillis() + 5000L; + final long deliverAtTime = System.currentTimeMillis() + 3000L; MessageId messageId = producer.newMessage() .key(String.valueOf(i)) .value("msg-" + i) .deliverAt(deliverAtTime) .send(); messageIds.add(i, messageId); - delayedTimes.add(i, deliverAtTime); } final int cancelMessage = 50; MessageIdImpl messageId = (MessageIdImpl) messageIds.get(cancelMessage); - Thread.sleep(1000L); - admin.topics().cancelDelayedMessage( topic, messageId.getLedgerId(), messageId.getEntryId(), - delayedTimes.get(cancelMessage), - // Collections.singletonList(subName) - Collections.emptyList() + Collections.singletonList(subName + "-1") + // Collections.emptyList() ); assertTrue(latch.await(15, TimeUnit.SECONDS), "Not all messages were received in time"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java index 21903f8dfc6e5..793a1767fd8c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/DelayedDeliveryTest.java @@ -686,5 +686,4 @@ public void testDelayedDeliveryExceedsMaxDelay() throws Exception { + maxDeliveryDelayInMillis + " milliseconds"); } } - } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index 5bdef56ce806a..4f5c2f5b520d3 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -4565,12 +4565,11 @@ default CompletableFuture createShadowTopicAsync(String shadowTopic, Strin * @param topic The topic name * @param ledgerId The ledger ID of the target delayed message * @param entryId The entry ID of the target delayed message - * @param deliverAt The original deliverAt time of the target delayed message (in milliseconds from epoch) * @param subscriptionNames List of subscription names to cancel on. * If null or empty, applies to all subscriptions. * @throws PulsarAdminException if the request fails */ - void cancelDelayedMessage(String topic, long ledgerId, long entryId, long deliverAt, + void cancelDelayedMessage(String topic, long ledgerId, long entryId, List subscriptionNames) throws PulsarAdminException; /** @@ -4579,11 +4578,10 @@ void cancelDelayedMessage(String topic, long ledgerId, long entryId, long delive * @param topic The topic name * @param ledgerId The ledger ID of the target delayed message * @param entryId The entry ID of the target delayed message - * @param deliverAt The original deliverAt time of the target delayed message (in milliseconds from epoch) * @param subscriptionNames List of subscription names to cancel on. * If null or empty, applies to all subscriptions. * @return a future that can be used to track the asynchronous operation */ - CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, long deliverAt, + CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, List subscriptionNames); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index b686c9dcab796..3db8cbd333e86 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -2820,43 +2820,22 @@ public CompletableFuture createShadowTopicAsync(String shadowTopic, String } @Override - public void cancelDelayedMessage(String topic, long ledgerId, long entryId, long deliverAt, + public void cancelDelayedMessage(String topic, long ledgerId, long entryId, List subscriptionNames) throws PulsarAdminException { - sync(() -> cancelDelayedMessageAsync(topic, ledgerId, entryId, deliverAt, subscriptionNames)); + sync(() -> cancelDelayedMessageAsync(topic, ledgerId, entryId, subscriptionNames)); } @Override - public CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, long deliverAt, + public CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, List subscriptionNames) { - TopicName tn = TopicName.get(topic); + TopicName tn = validateTopic(topic); WebTarget path = topicPath(tn, "cancelDelayedMessage"); path = path.queryParam("ledgerId", ledgerId) - .queryParam("entryId", entryId) - .queryParam("deliverAt", deliverAt); + .queryParam("entryId", entryId); if (subscriptionNames != null && !subscriptionNames.isEmpty()) { path = path.queryParam("subscriptionNames", subscriptionNames.toArray()); } - final CompletableFuture future = new CompletableFuture<>(); - try { - request(path).async().post(Entity.entity("", MediaType.APPLICATION_JSON), - new InvocationCallback() { - @Override - public void completed(Response response) { - if (response.getStatusInfo().getFamily() == Response.Status.Family.SUCCESSFUL) { - future.complete(null); - } else { - future.completeExceptionally(getApiException(response)); - } - } - @Override - public void failed(Throwable throwable) { - future.completeExceptionally(getApiException(throwable)); - } - }); - } catch (PulsarAdminException e) { - future.completeExceptionally(e); - } - return future; + return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); } private static final Logger log = LoggerFactory.getLogger(TopicsImpl.class); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 8c8fa4eacfb83..0611f932484cd 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -3071,10 +3071,6 @@ private class CancelDelayedMessage extends CliCommand { @Option(names = {"-e", "--entryId"}, description = "Entry ID of the message to cancel", required = true) private long entryId = -1L; - @Option(names = {"-t", "--deliverAt"}, description = "Original scheduled delivery time" - + " (timestamp in ms from epoch) of the message to cancel", required = true) - private long deliverAt = -1L; - @Option(names = {"-s", "--subscriptionNames"}, description = "Comma-separated list of subscription names to" + " target. If not specified, applies to all subscriptions.", split = ",") private List subscriptionNames = new ArrayList<>(); @@ -3082,13 +3078,12 @@ private class CancelDelayedMessage extends CliCommand { @Override void run() throws Exception { String topic = validateTopicName(topicName); - if (ledgerId < 0 || entryId < 0 || deliverAt <= 0) { - throw new PulsarAdminException("ledgerId, entryId must be non-negative," - + " and deliverAt must be positive."); + if (ledgerId < 0 || entryId < 0) { + throw new PulsarAdminException("ledgerId, entryId must be non-negative."); } - getAdmin().topics().cancelDelayedMessage(topic, ledgerId, entryId, deliverAt, subscriptionNames); + getAdmin().topics().cancelDelayedMessage(topic, ledgerId, entryId, subscriptionNames); print("Successfully requested cancellation for delayed message " + ledgerId + ":" + entryId - + " with deliverAt " + deliverAt + " on topic " + topic); + + " on topic " + topic); } } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index 7e25bf0ee1db8..a588b44a2317a 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -273,17 +273,14 @@ public void testCancelDelayedMessage() throws Exception { String topic = "persistent://public/default/testCancelDelayed"; long ledgerId = 123L; long entryId = 45L; - long deliverAt = System.currentTimeMillis() + 10000; // Test case 1: No specific subscriptions (should apply to all) cmdTopics.run(new String[]{ "cancel-delayed-message", topic, "-l", String.valueOf(ledgerId), "-e", String.valueOf(entryId), - "-t", String.valueOf(deliverAt) }); - verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), - eq(deliverAt), eq(new ArrayList<>())); + verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(new ArrayList<>())); // Test case 2: Specific subscriptions List subs = Lists.newArrayList("sub1", "sub2"); @@ -291,10 +288,9 @@ public void testCancelDelayedMessage() throws Exception { "cancel-delayed-message", topic, "-l", String.valueOf(ledgerId), "-e", String.valueOf(entryId), - "-t", String.valueOf(deliverAt), "-s", "sub1", "-s", "sub2" }); - verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(deliverAt), eq(subs)); + verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(subs)); // Test case 3: Single specific subscription List singleSub = Lists.newArrayList("sub-single"); @@ -302,9 +298,8 @@ public void testCancelDelayedMessage() throws Exception { "cancel-delayed-message", topic, "-l", String.valueOf(ledgerId), "-e", String.valueOf(entryId), - "-t", String.valueOf(deliverAt), "-s", "sub-single" }); - verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(deliverAt), eq(singleSub)); + verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(singleSub)); } } From d7f025c4931a294beef7721f83e335460100ac02 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Thu, 17 Jul 2025 21:54:40 +0800 Subject: [PATCH 08/15] feat(admin): add skipMessages by message IDs and remove cancelDelayedMessage --- .../admin/impl/PersistentTopicsBase.java | 161 ++---------------- .../broker/admin/v1/PersistentTopics.java | 56 +----- .../broker/admin/v2/PersistentTopics.java | 51 +----- .../pulsar/broker/service/Subscription.java | 15 +- .../NonPersistentSubscription.java | 6 + .../persistent/PersistentSubscription.java | 42 +++-- .../persistent/BucketDelayedDeliveryTest.java | 13 +- .../apache/pulsar/client/admin/Topics.java | 29 +--- .../client/admin/internal/TopicsImpl.java | 33 ++-- .../apache/pulsar/admin/cli/CmdTopics.java | 49 +++--- .../pulsar/admin/cli/TestCmdTopics.java | 38 ++--- .../common/policies/data/TopicOperation.java | 1 - 12 files changed, 110 insertions(+), 384 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 562c18fa11754..276db4c6542bf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1901,7 +1901,7 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy } protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, int numMessages, - boolean authoritative) { + boolean authoritative, Map messageIds) { CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); future.thenCompose(__ -> { if (topicName.isGlobal()) { @@ -1915,7 +1915,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, if (partitionMetadata.partitions > 0) { String msg = "Skip messages on a partitioned topic is not allowed"; log.warn("[{}] {} {} {}", clientAppId(), msg, topicName, subName); - throw new RestException(Status.METHOD_NOT_ALLOWED, msg); + throw new RestException(Status.METHOD_NOT_ALLOWED, msg); } return getTopicReferenceAsync(topicName).thenCompose(t -> { PersistentTopic topic = (PersistentTopic) t; @@ -1923,7 +1923,8 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, throw new RestException(new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString()))); } - if (subName.startsWith(topic.getReplicatorPrefix())) { + if (subName.startsWith(topic.getReplicatorPrefix()) + && (messageIds.isEmpty() || numMessages == 0)) { String remoteCluster = PersistentReplicator.getRemoteCluster(subName); PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); @@ -1944,6 +1945,13 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, new RestException(Status.NOT_FOUND, getSubNotFoundErrorMessage(topicName.toString(), subName))); } + if (!messageIds.isEmpty() && numMessages == 0) { + return sub.skipMessages(messageIds).thenAccept(unused -> { + log.info("[{}] Skipped messages on {} {}", clientAppId(), topicName, subName); + asyncResponse.resume(Response.noContent().build()); + } + ); + } return sub.skipMessages(numMessages).thenAccept(unused -> { log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, topicName, subName); @@ -5493,151 +5501,4 @@ protected CompletableFuture internalGetAutoSub return null; })); } - - protected void internalCancelDelayedMessage(AsyncResponse asyncResponse, long ledgerId, long entryId, - List subscriptionNames, boolean authoritative) { - CompletableFuture validationFuture = validateTopicOperationAsync(topicName, - TopicOperation.CANCEL_DELAYED_MESSAGE); - validationFuture = validationFuture.thenCompose(__ -> { - if (topicName.isGlobal()) { - return validateGlobalNamespaceOwnershipAsync(namespaceName); - } else { - return CompletableFuture.completedFuture(null); - } - }); - validationFuture.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) - .thenAccept(partitionMetadata -> { - if (topicName.isPartitioned()) { - internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, - subscriptionNames, authoritative); - } else { - if (partitionMetadata.partitions > 0) { - internalCancelDelayedMessageForPartitionedTopic(asyncResponse, partitionMetadata, - ledgerId, entryId, subscriptionNames); - } else { - internalCancelDelayedMessageForNonPartitionedTopic(asyncResponse, ledgerId, entryId, - subscriptionNames, authoritative); - } - } - }).exceptionally(ex -> { - if (isNot307And404Exception(ex)) { - log.error("[{}] Failed to cancel delayed message {}-{} on topic {}: {}", - clientAppId(), ledgerId, entryId, topicName, ex.getMessage(), ex); - } - resumeAsyncResponseExceptionally(asyncResponse, ex); - return null; - }); - } - - private void internalCancelDelayedMessageForPartitionedTopic(AsyncResponse asyncResponse, - PartitionedTopicMetadata partitionMetadata, - long ledgerId, long entryId, - List subscriptionNames) { - final List> futures = new ArrayList<>(partitionMetadata.partitions); - PulsarAdmin admin; - try { - admin = pulsar().getAdminClient(); - } catch (PulsarServerException e) { - asyncResponse.resume(new RestException(e)); - return; - } - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - futures.add(admin - .topics() - .cancelDelayedMessageAsync(topicNamePartition.toString(), - ledgerId, entryId, subscriptionNames)); - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = FutureUtil.unwrapCompletionException(exception); - log.warn("[{}] Failed to cancel delayed message {}-{} on some partitions of {}: {}", - clientAppId(), ledgerId, entryId, topicName, t.getMessage()); - resumeAsyncResponseExceptionally(asyncResponse, t); - } else { - log.info("[{}] Successfully requested cancellation for delayed message {}-{} on" - + " all partitions of topic {}", - clientAppId(), ledgerId, entryId, topicName); - asyncResponse.resume(Response.noContent().build()); - } - return null; - }); - } - - private void internalCancelDelayedMessageForNonPartitionedTopic(AsyncResponse asyncResponse, - long ledgerId, long entryId, - List subscriptionNames, - boolean authoritative) { - validateTopicOwnershipAsync(topicName, authoritative) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(optTopic -> { - if (!(optTopic instanceof PersistentTopic)) { - throw new RestException(Status.METHOD_NOT_ALLOWED, "Cancel delayed message on a non-persistent" - + " topic is not allowed"); - } - PersistentTopic persistentTopic = (PersistentTopic) optTopic; - List subsToProcess; - if (subscriptionNames == null || subscriptionNames.isEmpty()) { - subsToProcess = persistentTopic.getSubscriptions().keySet().stream() - .filter(subName -> !subName.equals(Compactor.COMPACTION_SUBSCRIPTION) - && !SystemTopicNames.isSystemTopic(TopicName. - get(persistentTopic.getName() + "/" + subName))) - .collect(Collectors.toList()); - if (subsToProcess.isEmpty()) { - log.info("[{}] No user subscriptions found to process for cancelling delayed message on" - + " topic {}.", clientAppId(), topicName); - return CompletableFuture.completedFuture(null); - } - log.info("[{}] Cancelling delayed message {}-{} for all non-system" - + " subscriptions on topic {}", - clientAppId(), ledgerId, entryId, topicName); - } else { - subsToProcess = new ArrayList<>(subscriptionNames); - log.info("[{}] Cancelling delayed message {}-{} for subscriptions {} on" - + " topic {}", clientAppId(), ledgerId, entryId, subsToProcess, topicName); - } - List> cancelFutures = subsToProcess.stream() - .map(subName -> internalCancelDelayedMessageForSubscriptionAsync( - persistentTopic, subName, ledgerId, entryId)) - .collect(Collectors.toList()); - return FutureUtil.waitForAll(cancelFutures); - }) - .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) - .exceptionally(ex -> { - Throwable t = FutureUtil.unwrapCompletionException(ex); - if (isNot307And404Exception(t)) { - log.error("[{}] Error in internalCancelDelayedMessageForNonPartitionedTopic for {}: {}", - clientAppId(), topicName, t.getMessage(), t); - } - resumeAsyncResponseExceptionally(asyncResponse, t); - return null; - }); - } - - private CompletableFuture internalCancelDelayedMessageForSubscriptionAsync( - PersistentTopic topic, String subName, long ledgerId, long entryId) { - Subscription sub = topic.getSubscription(subName); - if (sub == null) { - return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, - getSubNotFoundErrorMessage(topic.getName(), subName))); - } - return sub.cancelDelayedMessage(ledgerId, entryId) - .thenCompose(cancelled -> { - if (cancelled) { - log.info("[{}] Successfully requested cancellation for delayed message {}-{}" - + " on subscription {} of topic {}", - clientAppId(), ledgerId, entryId, subName, topic.getName()); - return CompletableFuture.completedFuture(null); - } else { - String errorMsg = String.format( - "Failed to cancel delayed message %d-%d on subscription %s" - + " of topic %s. Message may not exist in tracker, already delivered/cancelled," - + " or tracker not available/initialized.", - ledgerId, entryId, subName, topic.getName()); - log.warn("[{}] {}", clientAppId(), errorMsg); - return FutureUtil.failedFuture(new RestException(Status.PRECONDITION_FAILED, errorMsg)); - } - }); - } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index e4b2c1ec2d8d2..9aa7e93cf5265 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -24,7 +24,7 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; -import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import javax.ws.rs.DELETE; @@ -623,10 +623,11 @@ public void skipMessages(@Suspended final AsyncResponse asyncResponse, @PathPara @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String encodedSubName, @PathParam("numMessages") int numMessages, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "The message ID to skip") Map messageIds) { try { validateTopicName(property, cluster, namespace, encodedTopic); - internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative); + internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative, messageIds); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { @@ -1135,53 +1136,4 @@ public void getReplicatedSubscriptionStatus( internalGetReplicatedSubscriptionStatus(asyncResponse, decode(encodedSubName), authoritative); } - @POST - @Path("/{property}/{cluster}/{namespace}/{topic}/cancelDelayedMessage") - @ApiOperation(hidden = true, value = "Cancel a delayed message on specified subscriptions" - + " (or all if none specified).") - @ApiResponses(value = { - @ApiResponse(code = 204, message = "Operation successful"), - @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), - @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), - @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace or topic or subscription does not exist"), - @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), - @ApiResponse(code = 412, message = "Failed to cancel delayed message or invalid parameters"), - @ApiResponse(code = 500, message = "Internal server error")}) - public void cancelDelayedMessage( - @Suspended final AsyncResponse asyncResponse, - @ApiParam(value = "Specify the property (tenant)", required = true) - @PathParam("property") String property, - @ApiParam(value = "Specify the cluster", required = true) - @PathParam("cluster") String cluster, - @ApiParam(value = "Specify the namespace", required = true) - @PathParam("namespace") String namespace, - @ApiParam(value = "Specify topic name", required = true) - @PathParam("topic") @Encoded String encodedTopic, - @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, - @ApiParam(value = "Ledger ID of the target delayed message", required = true) - @QueryParam("ledgerId") long ledgerId, - @ApiParam(value = "Entry ID of the target delayed message", required = true) - @QueryParam("entryId") long entryId, - @ApiParam(value = "List of subscription names to cancel on (comma-separated, empty or null for" - + " all subscriptions)") - @QueryParam("subscriptionNames") List subscriptionNames) { - try { - validateTopicName(property, cluster, namespace, encodedTopic); - if (ledgerId < 0 || entryId < 0) { - asyncResponse.resume(new RestException(Response.Status.PRECONDITION_FAILED, - "ledgerId, entryId must be non-negative.")); - return; - } - List finalSubscriptionNames = (subscriptionNames == null || subscriptionNames.isEmpty()) - ? null : subscriptionNames; - internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, finalSubscriptionNames, authoritative); - } catch (WebApplicationException wae) { - asyncResponse.resume(wae); - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - } - } - } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 8ac67f63e1964..fa0d85bff5292 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -1545,10 +1545,11 @@ public void skipMessages( @ApiParam(value = "The number of messages to skip", defaultValue = "0") @PathParam("numMessages") int numMessages, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "The message ID to skip") Map messageIds) { try { validateTopicName(tenant, namespace, encodedTopic); - internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative); + internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative, messageIds); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { @@ -5015,51 +5016,5 @@ public void removeAutoSubscriptionCreation( }); } - @POST - @Path("/{tenant}/{namespace}/{topic}/cancelDelayedMessage") - @ApiOperation(value = "Cancel a delayed message on specified subscriptions (or all if none specified).") - @ApiResponses(value = { - @ApiResponse(code = 204, message = "Operation successful"), - @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), - @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), - @ApiResponse(code = 403, message = "Don't have admin permission"), - @ApiResponse(code = 404, message = "Namespace or topic or subscription does not exist"), - @ApiResponse(code = 405, message = "Operation not allowed on non-persistent topic"), - @ApiResponse(code = 412, message = "Failed to cancel delayed message or invalid parameters"), - @ApiResponse(code = 500, message = "Internal server error")}) - public void cancelDelayedMessage( - @Suspended final AsyncResponse asyncResponse, - @ApiParam(value = "Specify the tenant", required = true) - @PathParam("tenant") String tenant, - @ApiParam(value = "Specify the namespace", required = true) - @PathParam("namespace") String namespace, - @ApiParam(value = "Specify topic name", required = true) - @PathParam("topic") @Encoded String encodedTopic, - @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, - @ApiParam(value = "Ledger ID of the target delayed message", required = true) - @QueryParam("ledgerId") long ledgerId, - @ApiParam(value = "Entry ID of the target delayed message", required = true) - @QueryParam("entryId") long entryId, - @ApiParam(value = "List of subscription names to cancel on (empty or null for all subscriptions)") - @QueryParam("subscriptionNames") List subscriptionNames) { - try { - validateTopicName(tenant, namespace, encodedTopic); - if (ledgerId < 0 || entryId < 0) { - asyncResponse.resume(new RestException(Response.Status.PRECONDITION_FAILED, - "ledgerId, entryId must be positive.")); - return; - } - List finalSubscriptionNames = (subscriptionNames == null || subscriptionNames.isEmpty()) - ? null : subscriptionNames; - internalCancelDelayedMessage(asyncResponse, ledgerId, entryId, - finalSubscriptionNames, authoritative); - } catch (WebApplicationException wae) { - asyncResponse.resume(wae); - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - } - } - private static final Logger log = LoggerFactory.getLogger(PersistentTopics.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 4a6eea35ec10d..c9ff8226b0d56 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -80,6 +80,8 @@ default long getNumberOfEntriesDelayed() { CompletableFuture skipMessages(int numMessagesToSkip); + CompletableFuture skipMessages(Map messageIds); + CompletableFuture resetCursor(long timestamp); CompletableFuture resetCursor(Position position); @@ -114,19 +116,6 @@ default void processReplicatedSubscriptionSnapshot(ReplicatedSubscriptionsSnapsh CompletableFuture analyzeBacklog(Optional position); - /** - * Cancels a specific delayed message for subscriptions. - * - * @param ledgerId The ledger ID of the message to cancel. - * @param entryId The entry ID of the message to cancel. - * @return A CompletableFuture that resolves to true if the cancellation was successfully - * requested from the tracker, false otherwise (e.g., message not found in tracker, tracker not present, - * or already delivered/cancelled). - */ - default CompletableFuture cancelDelayedMessage(long ledgerId, long entryId) { - return CompletableFuture.completedFuture(false); - } - default int getNumberOfSameAddressConsumers(final String clientAddress) { int count = 0; if (clientAddress != null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index d469ce1daa1a8..c5472415db4a5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -249,6 +249,12 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { return CompletableFuture.completedFuture(null); } + @Override + public CompletableFuture skipMessages(Map messageIds) { + // No-op + return CompletableFuture.completedFuture(null); + } + @Override public CompletableFuture resetCursor(long timestamp) { // No-op diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 001159a62d656..c8221b337b6d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -24,6 +24,7 @@ import com.google.common.base.MoreObjects; import io.netty.buffer.ByteBuf; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; @@ -789,6 +790,35 @@ public void skipEntriesFailed(ManagedLedgerException exception, Object ctx) { return future; } + @Override + public CompletableFuture skipMessages(Map messageIds) { + if (log.isDebugEnabled()) { + log.debug("[{}][{}] Skipping messages by messageIds, current backlog {}", topicName, subName, + cursor.getNumberOfEntriesInBacklog(false)); + } + + if (Subscription.isCumulativeAckMode(getType())) { + return CompletableFuture.failedFuture(new NotAllowedException("Unsupported subscription type.")); + } + + List positions = new ArrayList<>(); + for (Map.Entry entry : messageIds.entrySet()) { + try { + long ledgerId = Long.parseLong(entry.getKey()); + long entryId = Long.parseLong(entry.getValue()); + Position position = PositionFactory.create(ledgerId, entryId); + positions.add(position); + } catch (Exception e) { + return CompletableFuture.failedFuture(new NotAllowedException("Invalid message ID.")); + } + } + + Map properties = Collections.emptyMap(); + acknowledgeMessage(positions, AckType.Individual, properties); + + return CompletableFuture.completedFuture(null); + } + @Override public CompletableFuture resetCursor(long timestamp) { CompletableFuture future = new CompletableFuture<>(); @@ -1548,18 +1578,6 @@ public CompletableFuture endTxn(long txnidMostBits, long txnidLeastBits, i } } - @Override - public CompletableFuture cancelDelayedMessage(long ledgerId, long entryId) { - if (Subscription.isCumulativeAckMode(getType())) { - return CompletableFuture.completedFuture(false); - } - Position position = PositionFactory.create(ledgerId, entryId); - List positions = Collections.singletonList(position); - Map properties = Collections.emptyMap(); - acknowledgeMessage(positions, AckType.Individual, properties); - return CompletableFuture.completedFuture(true); - } - @VisibleForTesting public ManagedCursor getCursor() { return cursor; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 2628da91f3981..31652ee033f17 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -26,7 +26,6 @@ import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -520,13 +519,11 @@ public void testDelayedMessageCancel() throws Exception { final int cancelMessage = 50; MessageIdImpl messageId = (MessageIdImpl) messageIds.get(cancelMessage); - admin.topics().cancelDelayedMessage( - topic, - messageId.getLedgerId(), - messageId.getEntryId(), - Collections.singletonList(subName + "-1") - // Collections.emptyList() - ); + Map ackMessageIds = new HashMap<>(); + ackMessageIds.put(String.valueOf(messageId.getLedgerId()), String.valueOf(messageId.getEntryId())); + + admin.topics().skipMessages(topic + "-partition-0", subName + "-1", ackMessageIds); + admin.topics().skipMessages(topic + "-partition-1", subName + "-1", ackMessageIds); assertTrue(latch.await(15, TimeUnit.SECONDS), "Not all messages were received in time"); assertFalse((receivedMessages1.contains("msg-" + cancelMessage) diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index 4f5c2f5b520d3..e911f4ea18d31 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -1551,6 +1551,10 @@ PartitionedTopicInternalStats getPartitionedInternalStats(String topic) */ CompletableFuture skipMessagesAsync(String topic, String subName, long numMessages); + void skipMessages(String topic, String subName, Map messageIds) throws PulsarAdminException; + + CompletableFuture skipMessagesAsync(String topic, String subName, Map messageIds); + /** * Expire all messages older than given N (expireTimeInSeconds) seconds for a given subscription. * @@ -4559,29 +4563,4 @@ default CompletableFuture createShadowTopicAsync(String shadowTopic, Strin return createShadowTopicAsync(shadowTopic, sourceTopic, null); } - /** - * Cancel a delayed message on specified subscriptions (or all if none specified). - * - * @param topic The topic name - * @param ledgerId The ledger ID of the target delayed message - * @param entryId The entry ID of the target delayed message - * @param subscriptionNames List of subscription names to cancel on. - * If null or empty, applies to all subscriptions. - * @throws PulsarAdminException if the request fails - */ - void cancelDelayedMessage(String topic, long ledgerId, long entryId, - List subscriptionNames) throws PulsarAdminException; - - /** - * Cancel a delayed message on specified subscriptions (or all if none specified) asynchronously. - * - * @param topic The topic name - * @param ledgerId The ledger ID of the target delayed message - * @param entryId The entry ID of the target delayed message - * @param subscriptionNames List of subscription names to cancel on. - * If null or empty, applies to all subscriptions. - * @return a future that can be used to track the asynchronous operation - */ - CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, - List subscriptionNames); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index 3db8cbd333e86..04a1ed9444f13 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -826,6 +826,20 @@ public CompletableFuture skipMessagesAsync(String topic, String subName, l return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); } + @Override + public void skipMessages(String topic, String subName, Map messageIds) throws PulsarAdminException { + sync(() -> skipMessagesAsync(topic, subName, messageIds)); + } + + @Override + public CompletableFuture skipMessagesAsync(String topic, String subName, Map messageIds) { + TopicName tn = validateTopic(topic); + String encodedSubName = Codec.encode(subName); + WebTarget path = topicPath(tn, "subscription", encodedSubName, "skip", "0"); + messageIds = messageIds == null ? new HashMap<>() : messageIds; + return asyncPostRequest(path, Entity.entity(messageIds, MediaType.APPLICATION_JSON)); + } + @Override public void expireMessages(String topic, String subName, long expireTimeInSeconds) throws PulsarAdminException { sync(() -> expireMessagesAsync(topic, subName, expireTimeInSeconds)); @@ -2819,24 +2833,5 @@ public CompletableFuture createShadowTopicAsync(String shadowTopic, String }); } - @Override - public void cancelDelayedMessage(String topic, long ledgerId, long entryId, - List subscriptionNames) throws PulsarAdminException { - sync(() -> cancelDelayedMessageAsync(topic, ledgerId, entryId, subscriptionNames)); - } - - @Override - public CompletableFuture cancelDelayedMessageAsync(String topic, long ledgerId, long entryId, - List subscriptionNames) { - TopicName tn = validateTopic(topic); - WebTarget path = topicPath(tn, "cancelDelayedMessage"); - path = path.queryParam("ledgerId", ledgerId) - .queryParam("entryId", entryId); - if (subscriptionNames != null && !subscriptionNames.isEmpty()) { - path = path.queryParam("subscriptionNames", subscriptionNames.toArray()); - } - return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); - } - private static final Logger log = LoggerFactory.getLogger(TopicsImpl.class); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 0611f932484cd..ad3389da15843 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -124,6 +124,7 @@ public CmdTopics(Supplier admin) { addCommand("partitioned-stats-internal", new GetPartitionedStatsInternal()); addCommand("skip", new Skip()); + addCommand("skip-messages", new SkipMessages()); addCommand("clear-backlog", new ClearBacklog()); addCommand("expire-messages", new ExpireMessages()); @@ -275,7 +276,6 @@ public CmdTopics(Supplier admin) { addCommand("set-schema-validation-enforce", new SetSchemaValidationEnforced()); addCommand("trim-topic", new TrimTopic()); - addCommand("cancel-delayed-message", new CancelDelayedMessage()); } @Command(description = "Get the list of topics under a namespace.") @@ -854,6 +854,25 @@ void run() throws PulsarAdminException { } } + @Command(description = "Skip some messages for the subscription") + private class SkipMessages extends CliCommand { + @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") + private String topicName; + + @Option(names = { "-s", + "--subscription" }, description = "Subscription to be skip messages on", required = true) + private String subName; + + @Option(names = { "-m", "--messageId" }, description = "The message ID to skip", required = true) + private Map messageIds; + + @Override + void run() throws PulsarAdminException { + String topic = validateTopicName(topicName); + getTopics().skipMessages(topic, subName, messageIds); + } + } + @Command(description = "Expire messages that older than given expiry time (in seconds) " + "for the subscription") private class ExpireMessages extends CliCommand { @@ -3059,32 +3078,4 @@ void run() throws PulsarAdminException { getAdmin().topics().trimTopic(topic); } } - - @Command(description = "Cancel a delayed message") - private class CancelDelayedMessage extends CliCommand { - @Parameters(description = "persistent://tenant/namespace/topic", arity = "1") - private String topicName; - - @Option(names = {"-l", "--ledgerId"}, description = "Ledger ID of the message to cancel", required = true) - private long ledgerId = -1L; - - @Option(names = {"-e", "--entryId"}, description = "Entry ID of the message to cancel", required = true) - private long entryId = -1L; - - @Option(names = {"-s", "--subscriptionNames"}, description = "Comma-separated list of subscription names to" - + " target. If not specified, applies to all subscriptions.", split = ",") - private List subscriptionNames = new ArrayList<>(); - - @Override - void run() throws Exception { - String topic = validateTopicName(topicName); - if (ledgerId < 0 || entryId < 0) { - throw new PulsarAdminException("ledgerId, entryId must be non-negative."); - } - getAdmin().topics().cancelDelayedMessage(topic, ledgerId, entryId, subscriptionNames); - print("Successfully requested cancellation for delayed message " + ledgerId + ":" + entryId - + " on topic " + topic); - } - } - } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index a588b44a2317a..3dcb27cb0e3e9 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -39,7 +39,9 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import lombok.Cleanup; import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.client.admin.Lookup; @@ -269,37 +271,19 @@ public void testSetRetentionCmd() throws Exception { } @Test - public void testCancelDelayedMessage() throws Exception { + public void testSkipMessages() throws Exception { String topic = "persistent://public/default/testCancelDelayed"; - long ledgerId = 123L; - long entryId = 45L; + String ledgerId = "123"; + String entryId = "45"; + Map messageIds = new HashMap<>(); + messageIds.put(ledgerId, entryId); - // Test case 1: No specific subscriptions (should apply to all) cmdTopics.run(new String[]{ - "cancel-delayed-message", topic, - "-l", String.valueOf(ledgerId), - "-e", String.valueOf(entryId), + "skip-messages", topic, + "-s", "test-sub", + "-m", ledgerId + "=" + entryId }); - verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(new ArrayList<>())); - // Test case 2: Specific subscriptions - List subs = Lists.newArrayList("sub1", "sub2"); - cmdTopics.run(new String[]{ - "cancel-delayed-message", topic, - "-l", String.valueOf(ledgerId), - "-e", String.valueOf(entryId), - "-s", "sub1", "-s", "sub2" - }); - verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(subs)); - - // Test case 3: Single specific subscription - List singleSub = Lists.newArrayList("sub-single"); - cmdTopics.run(new String[]{ - "cancel-delayed-message", topic, - "-l", String.valueOf(ledgerId), - "-e", String.valueOf(entryId), - "-s", "sub-single" - }); - verify(mockTopics).cancelDelayedMessage(eq(topic), eq(ledgerId), eq(entryId), eq(singleSub)); + verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), eq(messageIds)); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java index b7b840cff2180..4c74ffcd8f085 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/TopicOperation.java @@ -34,7 +34,6 @@ public enum TopicOperation { SKIP, TERMINATE, UNLOAD, - CANCEL_DELAYED_MESSAGE, GRANT_PERMISSION, GET_PERMISSION, From 5f170cccaaa89f2b8dc2740c2acc0bd2f1a164a6 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Thu, 31 Jul 2025 21:26:20 +0800 Subject: [PATCH 09/15] use skipByMessageIds as the new path --- .../admin/impl/PersistentTopicsBase.java | 144 ++++++++++++++---- .../broker/admin/v1/PersistentTopics.java | 28 +++- .../broker/admin/v2/PersistentTopics.java | 37 ++++- .../client/admin/internal/TopicsImpl.java | 2 +- 4 files changed, 175 insertions(+), 36 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 276db4c6542bf..2b0339d547fca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1277,19 +1277,15 @@ private void resumeAsyncResponse(AsyncResponse asyncResponse, Set subscr if (ex != null) { log.warn("[{}] Failed to get list of subscriptions for {}: {}", clientAppId(), topicName, ex.getMessage()); - if (ex instanceof PulsarAdminException) { - PulsarAdminException pae = (PulsarAdminException) ex; + if (ex instanceof PulsarAdminException pae) { if (pae.getStatusCode() == Status.NOT_FOUND.getStatusCode()) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Internal topics have not been generated yet")); - return; } else { asyncResponse.resume(new RestException(pae)); - return; } } else { asyncResponse.resume(new RestException(ex)); - return; } } else { asyncResponse.resume(subscriptions); @@ -1901,7 +1897,7 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy } protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, int numMessages, - boolean authoritative, Map messageIds) { + boolean authoritative) { CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); future.thenCompose(__ -> { if (topicName.isGlobal()) { @@ -1923,8 +1919,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, throw new RestException(new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString()))); } - if (subName.startsWith(topic.getReplicatorPrefix()) - && (messageIds.isEmpty() || numMessages == 0)) { + if (subName.startsWith(topic.getReplicatorPrefix())) { String remoteCluster = PersistentReplicator.getRemoteCluster(subName); PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); @@ -1945,13 +1940,6 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, new RestException(Status.NOT_FOUND, getSubNotFoundErrorMessage(topicName.toString(), subName))); } - if (!messageIds.isEmpty() && numMessages == 0) { - return sub.skipMessages(messageIds).thenAccept(unused -> { - log.info("[{}] Skipped messages on {} {}", clientAppId(), topicName, subName); - asyncResponse.resume(Response.noContent().build()); - } - ); - } return sub.skipMessages(numMessages).thenAccept(unused -> { log.info("[{}] Skipped {} messages on {} {}", clientAppId(), numMessages, topicName, subName); @@ -1972,6 +1960,109 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, }); } + protected void internalSkipByMessageIds(AsyncResponse asyncResponse, String subName, boolean authoritative, + Map messageIds) { + CompletableFuture validationFuture = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); + validationFuture = validationFuture.thenCompose(__ -> { + if (topicName.isGlobal()) { + return validateGlobalNamespaceOwnershipAsync(namespaceName); + } else { + return CompletableFuture.completedFuture(null); + } + }); + validationFuture.thenCompose(__ -> getPartitionedTopicMetadataAsync(topicName, authoritative, false)) + .thenAccept(partitionMetadata -> { + if (topicName.isPartitioned()) { + internalSkipByMessageIdsForNonPartitionedTopic(asyncResponse, messageIds, + subName, authoritative); + } else { + if (partitionMetadata.partitions > 0) { + internalSkipByMessageIdsForPartitionedTopic(asyncResponse, partitionMetadata, + messageIds, subName); + } else { + internalSkipByMessageIdsForNonPartitionedTopic(asyncResponse, messageIds, + subName, authoritative); + } + } + }).exceptionally(ex -> { + if (isNot307And404Exception(ex)) { + log.error("[{}] Failed to ack messages on topic {}: {}", clientAppId(), topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); + } + + private void internalSkipByMessageIdsForPartitionedTopic(AsyncResponse asyncResponse, + PartitionedTopicMetadata partitionMetadata, + Map messageIds, + String subName) { + final List> futures = new ArrayList<>(partitionMetadata.partitions); + PulsarAdmin admin; + try { + admin = pulsar().getAdminClient(); + } catch (PulsarServerException e) { + asyncResponse.resume(new RestException(e)); + return; + } + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + futures.add(admin + .topics() + .skipMessagesAsync(topicNamePartition.toString(), subName, messageIds)); + } + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = FutureUtil.unwrapCompletionException(exception); + log.warn("[{}] Failed to ack messages on some partitions of {}: {}", + clientAppId(), topicName, t.getMessage()); + resumeAsyncResponseExceptionally(asyncResponse, t); + } else { + log.info("[{}] Successfully requested cancellation for delayed message on" + + " all partitions of topic {}", clientAppId(), topicName); + asyncResponse.resume(Response.noContent().build()); + } + return null; + }); + } + + private void internalSkipByMessageIdsForNonPartitionedTopic(AsyncResponse asyncResponse, + Map messageIds, + String subName, + boolean authoritative) { + validateTopicOwnershipAsync(topicName, authoritative) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(optTopic -> { + if (!(optTopic instanceof PersistentTopic persistentTopic)) { + throw new RestException(Status.METHOD_NOT_ALLOWED, "Cancel delayed message on a non-persistent" + + " topic is not allowed"); + } + log.info("[{}] Cancelling delayed message for subscription {} on topic {}", clientAppId(), + subName, topicName); + return internalSkipByMessageIdsForSubscriptionAsync(persistentTopic, subName, messageIds); + }) + .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) + .exceptionally(ex -> { + Throwable t = FutureUtil.unwrapCompletionException(ex); + if (isNot307And404Exception(t)) { + log.error("[{}] Error in internalSkipByMessageIdsForNonPartitionedTopic for {}: {}", + clientAppId(), topicName, t.getMessage(), t); + } + resumeAsyncResponseExceptionally(asyncResponse, t); + return null; + }); + } + + private CompletableFuture internalSkipByMessageIdsForSubscriptionAsync( + PersistentTopic topic, String subName, Map messageIds) { + Subscription sub = topic.getSubscription(subName); + if (sub == null) { + return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, + getSubNotFoundErrorMessage(topic.getName(), subName))); + } + return sub.skipMessages(messageIds); + } + protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResponse, int expireTimeInSeconds, boolean authoritative) { CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES); @@ -2055,16 +2146,15 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy getTopicNotFoundErrorMessage(topicName.toString()))); return; } - if (!(t instanceof PersistentTopic)) { + if (!(t instanceof PersistentTopic topic)) { resumeAsyncResponseExceptionally(asyncResponse, new RestException(Status.METHOD_NOT_ALLOWED, "Expire messages for all subscriptions on a non-persistent topic is not allowed")); return; } - PersistentTopic topic = (PersistentTopic) t; final List> futures = - new ArrayList<>((int) topic.getReplicators().size()); + new ArrayList<>(topic.getReplicators().size()); List subNames = - new ArrayList<>((int) topic.getSubscriptions().size()); + new ArrayList<>(topic.getSubscriptions().size()); subNames.addAll(topic.getSubscriptions().keySet().stream().filter( subName -> !subName.equals(Compactor.COMPACTION_SUBSCRIPTION)).toList()); for (int i = 0; i < subNames.size(); i++) { @@ -2837,12 +2927,11 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic)) { + if (!(topic instanceof PersistentTopic persistentTopic)) { log.error("[{}] Not supported operation of non-persistent topic {} ", clientAppId(), topicName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Get message ID by timestamp on a non-persistent topic is not allowed"); } - final PersistentTopic persistentTopic = (PersistentTopic) topic; return persistentTopic.getTopicCompactionService().readLastCompactedEntry().thenCompose(lastEntry -> { if (lastEntry == null) { @@ -2983,13 +3072,12 @@ protected CompletableFuture internalExamineMessageAsync(String initial return CompletableFuture.completedFuture(null); }).thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic)) { + if (!(topic instanceof PersistentTopic persistentTopic)) { log.error("[{}] Not supported operation of non-persistent topic {} ", clientAppId(), topicName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Examine messages on a non-persistent topic is not allowed"); } try { - PersistentTopic persistentTopic = (PersistentTopic) topic; long totalMessage = persistentTopic.getNumberOfEntries(); if (totalMessage <= 0) { throw new RestException(Status.PRECONDITION_FAILED, @@ -4013,12 +4101,11 @@ private CompletableFuture internalExpireMessagesByTimestampForSinglePartit getTopicNotFoundErrorMessage(topicName.toString()))); return; } - if (!(t instanceof PersistentTopic)) { + if (!(t instanceof PersistentTopic topic)) { resultFuture.completeExceptionally(new RestException(Status.METHOD_NOT_ALLOWED, "Expire messages on a non-persistent topic is not allowed")); return; } - PersistentTopic topic = (PersistentTopic) t; final MessageExpirer messageExpirer; if (subName.startsWith(topic.getReplicatorPrefix())) { @@ -4051,7 +4138,6 @@ private CompletableFuture internalExpireMessagesByTimestampForSinglePartit + "almost catch up. If it's performed on a partitioned topic operation might" + " succeeded on other partitions, please check stats of individual partition." )); - return; } }).exceptionally(e -> { resultFuture.completeExceptionally(FutureUtil.unwrapCompletionException(e)); @@ -4280,12 +4366,10 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } catch (AlreadyRunningException e) { resumeAsyncResponseExceptionally(asyncResponse, new RestException(Status.CONFLICT, e.getMessage())); - return; } catch (Exception e) { log.error("[{}] Failed to trigger compaction on topic {}", clientAppId(), topicName, e); resumeAsyncResponseExceptionally(asyncResponse, new RestException(e)); - return; } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. @@ -4317,11 +4401,9 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } catch (AlreadyRunningException e) { resumeAsyncResponseExceptionally(asyncResponse, new RestException(Status.CONFLICT, e.getMessage())); - return; } catch (Exception e) { log.warn("Unexpected error triggering offload", e); resumeAsyncResponseExceptionally(asyncResponse, new RestException(e)); - return; } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. @@ -4462,7 +4544,7 @@ private CompletableFuture topicNotFoundReasonAsync(TopicName topicName) { final String topicErrorType = partitionedTopicMetadata == null ? "has no metadata" : "has zero partitions"; throw new RestException(Status.NOT_FOUND, String.format( - "Partitioned Topic not found: %s %s", topicName.toString(), topicErrorType)); + "Partitioned Topic not found: %s %s", topicName, topicErrorType)); } }) .thenCompose(__ -> internalGetListAsync(Optional.empty())) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index 9aa7e93cf5265..f1b167d034348 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -623,11 +623,33 @@ public void skipMessages(@Suspended final AsyncResponse asyncResponse, @PathPara @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String encodedSubName, @PathParam("numMessages") int numMessages, - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, - @ApiParam(value = "The message ID to skip") Map messageIds) { + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + try { + validateTopicName(property, cluster, namespace, encodedTopic); + internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + } + } + + @POST + @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/skipByMessageIds") + @ApiOperation(hidden = true, value = "Skip messages on a topic subscription.") + @ApiResponses(value = { + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namesapce or topic or subscription does not exist") }) + public void skipByMessageIds(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("topic") @Encoded String encodedTopic, + @PathParam("subName") String encodedSubName, + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "The message ID to skip") Map messageIds) { try { validateTopicName(property, cluster, namespace, encodedTopic); - internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative, messageIds); + internalSkipByMessageIds(asyncResponse, decode(encodedSubName), authoritative, messageIds); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index fa0d85bff5292..9a0ded8a4bc1e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -1545,11 +1545,46 @@ public void skipMessages( @ApiParam(value = "The number of messages to skip", defaultValue = "0") @PathParam("numMessages") int numMessages, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + try { + validateTopicName(tenant, namespace, encodedTopic); + internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + } + } + + @POST + @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/skipByMessageIds") + @ApiOperation(value = "Skipping messages on a topic subscription.") + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), + @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Namespace or topic or subscription does not exist"), + @ApiResponse(code = 405, message = "Skipping messages on a partitioned topic is not allowed"), + @ApiResponse(code = 500, message = "Internal server error"), + @ApiResponse(code = 503, message = "Failed to validate global cluster configuration") + }) + public void skipByMessageIds( + @Suspended final AsyncResponse asyncResponse, + @ApiParam(value = "Specify the tenant", required = true) + @PathParam("tenant") String tenant, + @ApiParam(value = "Specify the namespace", required = true) + @PathParam("namespace") String namespace, + @ApiParam(value = "Specify topic name", required = true) + @PathParam("topic") @Encoded String encodedTopic, + @ApiParam(value = "Name of subscription") + @PathParam("subName") String encodedSubName, + @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "The message ID to skip") Map messageIds) { try { validateTopicName(tenant, namespace, encodedTopic); - internalSkipMessages(asyncResponse, decode(encodedSubName), numMessages, authoritative, messageIds); + internalSkipByMessageIds(asyncResponse, decode(encodedSubName), authoritative, messageIds); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index 04a1ed9444f13..a4a5c1620a209 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -835,7 +835,7 @@ public void skipMessages(String topic, String subName, Map messa public CompletableFuture skipMessagesAsync(String topic, String subName, Map messageIds) { TopicName tn = validateTopic(topic); String encodedSubName = Codec.encode(subName); - WebTarget path = topicPath(tn, "subscription", encodedSubName, "skip", "0"); + WebTarget path = topicPath(tn, "subscription", encodedSubName, "skipByMessageIds", "0"); messageIds = messageIds == null ? new HashMap<>() : messageIds; return asyncPostRequest(path, Entity.entity(messageIds, MediaType.APPLICATION_JSON)); } From 347e52f7b4ec31f5c6c459772ebcb9b616c15850 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Nov 2025 13:20:07 +0800 Subject: [PATCH 10/15] feat: enhance skipMessages functionality to support batch indices --- .../admin/impl/PersistentTopicsBase.java | 14 ++- .../persistent/PersistentSubscription.java | 115 ++++++++++++++++-- .../persistent/BucketDelayedDeliveryTest.java | 7 +- .../client/admin/internal/TopicsImpl.java | 2 +- .../apache/pulsar/admin/cli/CmdTopics.java | 4 +- .../pulsar/admin/cli/TestCmdTopics.java | 18 +++ 6 files changed, 141 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 4dd48156a9058..6fa1771a1f200 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1261,7 +1261,8 @@ private void resumeAsyncResponse(AsyncResponse asyncResponse, Set subscr if (ex != null) { log.warn("[{}] Failed to get list of subscriptions for {}: {}", clientAppId(), topicName, ex.getMessage()); - if (ex instanceof PulsarAdminException pae) { + if (ex instanceof PulsarAdminException) { + PulsarAdminException pae = (PulsarAdminException) ex; if (pae.getStatusCode() == Status.NOT_FOUND.getStatusCode()) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Internal topics have not been generated yet")); @@ -2133,11 +2134,12 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy getTopicNotFoundErrorMessage(topicName.toString()))); return; } - if (!(t instanceof PersistentTopic topic)) { + if (!(t instanceof PersistentTopic)) { resumeAsyncResponseExceptionally(asyncResponse, new RestException(Status.METHOD_NOT_ALLOWED, "Expire messages for all subscriptions on a non-persistent topic is not allowed")); return; } + PersistentTopic topic = (PersistentTopic) t; final List> futures = new ArrayList<>((int) topic.getReplicators().size()); List subNames = @@ -2916,7 +2918,7 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) .thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic persistentTopic)) { + if (!(topic instanceof PersistentTopic)) { log.error("[{}] Not supported operation of non-persistent topic {} ", clientAppId(), topicName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Get message ID by timestamp on a non-persistent topic is not allowed"); @@ -3047,12 +3049,13 @@ protected CompletableFuture internalExamineMessageAsync(String initial return CompletableFuture.completedFuture(null); }).thenCompose(__ -> getTopicReferenceAsync(topicName)) .thenCompose(topic -> { - if (!(topic instanceof PersistentTopic persistentTopic)) { + if (!(topic instanceof PersistentTopic)) { log.error("[{}] Not supported operation of non-persistent topic {} ", clientAppId(), topicName); throw new RestException(Status.METHOD_NOT_ALLOWED, "Examine messages on a non-persistent topic is not allowed"); } try { + PersistentTopic persistentTopic = (PersistentTopic) topic; long totalMessage = persistentTopic.getNumberOfEntries(); if (totalMessage <= 0) { throw new RestException(Status.PRECONDITION_FAILED, @@ -4057,11 +4060,12 @@ private CompletableFuture internalExpireMessagesByTimestampForSinglePartit getTopicNotFoundErrorMessage(topicName.toString()))); return; } - if (!(t instanceof PersistentTopic topic)) { + if (!(t instanceof PersistentTopic)) { resultFuture.completeExceptionally(new RestException(Status.METHOD_NOT_ALLOWED, "Expire messages on a non-persistent topic is not allowed")); return; } + PersistentTopic topic = (PersistentTopic) t; final MessageExpirer messageExpirer; if (subName.startsWith(topic.getReplicatorPrefix())) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index e26fe162ff87a..32d2fc67c2983 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -23,13 +23,16 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import io.netty.buffer.ByteBuf; -import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.BitSet; +import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -53,6 +56,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.InvalidCursorPositionException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.tuple.MutablePair; @@ -823,22 +827,113 @@ public CompletableFuture skipMessages(Map messageIds) { return CompletableFuture.failedFuture(new NotAllowedException("Unsupported subscription type.")); } - List positions = new ArrayList<>(); + // Collect full-entry acks and partial (batchIndex) acks + List fullEntryPositions = new ArrayList<>(); + Map> partialAckIndexByPos = new HashMap<>(); // key: ledgerId:entryId + for (Map.Entry entry : messageIds.entrySet()) { + final long ledgerId; + final String value = entry.getValue(); try { - long ledgerId = Long.parseLong(entry.getKey()); - long entryId = Long.parseLong(entry.getValue()); - Position position = PositionFactory.create(ledgerId, entryId); - positions.add(position); + ledgerId = Long.parseLong(entry.getKey()); + } catch (Exception e) { + return CompletableFuture.failedFuture(new NotAllowedException("Invalid message ID ledgerId.")); + } + + try { + if (value.contains(":")) { + int idx = value.indexOf(':'); + long entryId = Long.parseLong(value.substring(0, idx)); + int batchIndex = Integer.parseInt(value.substring(idx + 1)); + String key = ledgerId + ":" + entryId; + partialAckIndexByPos.computeIfAbsent(key, __ -> new ArrayList<>()).add(batchIndex); + } else { + long entryId = Long.parseLong(value); + fullEntryPositions.add(PositionFactory.create(ledgerId, entryId)); + } } catch (Exception e) { - return CompletableFuture.failedFuture(new NotAllowedException("Invalid message ID.")); + return CompletableFuture.failedFuture(new NotAllowedException("Invalid message ID value.")); } } - Map properties = Collections.emptyMap(); - acknowledgeMessage(positions, AckType.Individual, properties); + // If there are no partial ack requests, just ack full entries + if (partialAckIndexByPos.isEmpty()) { + Map properties = Collections.emptyMap(); + acknowledgeMessage(fullEntryPositions, AckType.Individual, properties); + return CompletableFuture.completedFuture(null); + } + + // We need to read entries corresponding to partial ack positions to determine batch sizes + Set positionsToLoad = new HashSet<>(); + for (String key : partialAckIndexByPos.keySet()) { + int sep = key.indexOf(':'); + long ledgerId = Long.parseLong(key.substring(0, sep)); + long entryId = Long.parseLong(key.substring(sep + 1)); + positionsToLoad.add(PositionFactory.create(ledgerId, entryId)); + } + + CompletableFuture result = new CompletableFuture<>(); + + cursor.asyncReplayEntries(positionsToLoad, new AsyncCallbacks.ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + try { + List positionsForAck = new ArrayList<>(fullEntryPositions.size() + entries.size()); + // include full-entry positions + positionsForAck.addAll(fullEntryPositions); + + for (Entry entry : entries) { + try { + final long ledgerId = entry.getLedgerId(); + final long entryId = entry.getEntryId(); + final String key = ledgerId + ":" + entryId; + List indexes = partialAckIndexByPos.get(key); + if (indexes == null || indexes.isEmpty()) { + // Nothing to ack for this entry + continue; + } + + MessageMetadata metadata = Commands.parseMessageMetadata(entry.getDataBuffer()); + int batchSize = metadata.hasNumMessagesInBatch() ? metadata.getNumMessagesInBatch() : 1; + if (batchSize <= 1) { + result.completeExceptionally(new NotAllowedException( + "batchIndex specified but entry is not a batch message")); + return; + } + // Validate and build ackSet bitset + BitSet bitSet = new BitSet(); + bitSet.set(0, batchSize); + for (int bi : indexes) { + if (bi < 0 || bi >= batchSize) { + result.completeExceptionally(new NotAllowedException( + "Invalid batchIndex: " + bi + ", batchSize=" + batchSize)); + return; + } + bitSet.clear(bi); + } + long[] ackSet = bitSet.toLongArray(); + Position posWithAckSet = AckSetStateUtil.createPositionWithAckSet(ledgerId, entryId, ackSet); + positionsForAck.add(posWithAckSet); + } finally { + entry.release(); + } + } + + Map properties = Collections.emptyMap(); + acknowledgeMessage(positionsForAck, AckType.Individual, properties); + result.complete(null); + } catch (Exception e) { + result.completeExceptionally(e); + } + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + result.completeExceptionally(exception); + } + }, null, true); - return CompletableFuture.completedFuture(null); + return result; } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index 31341102337e7..c2bedfcf1b2b9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -21,7 +21,10 @@ import static org.apache.bookkeeper.mledger.ManagedCursor.CURSOR_INTERNAL_PROPERTY_PREFIX; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; -import static org.testng.Assert.*; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; @@ -525,7 +528,7 @@ public void testDelayedMessageCancel() throws Exception { final int cancelMessage = 50; MessageIdImpl messageId = (MessageIdImpl) messageIds.get(cancelMessage); - Map ackMessageIds = new HashMap<>(); + Map ackMessageIds = new HashMap<>(); ackMessageIds.put(String.valueOf(messageId.getLedgerId()), String.valueOf(messageId.getEntryId())); admin.topics().skipMessages(topic + "-partition-0", subName + "-1", ackMessageIds); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index eb5c6abdd0d32..243ce508841db 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -835,7 +835,7 @@ public void skipMessages(String topic, String subName, Map messa public CompletableFuture skipMessagesAsync(String topic, String subName, Map messageIds) { TopicName tn = validateTopic(topic); String encodedSubName = Codec.encode(subName); - WebTarget path = topicPath(tn, "subscription", encodedSubName, "skipByMessageIds", "0"); + WebTarget path = topicPath(tn, "subscription", encodedSubName, "skipByMessageIds"); messageIds = messageIds == null ? new HashMap<>() : messageIds; return asyncPostRequest(path, Entity.entity(messageIds, MediaType.APPLICATION_JSON)); } diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 5ccf7439a3f84..53ff7851d3606 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -865,7 +865,9 @@ private class SkipMessages extends CliCommand { "--subscription" }, description = "Subscription to be skip messages on", required = true) private String subName; - @Option(names = { "-m", "--messageId" }, description = "The message ID to skip", required = true) + @Option(names = { "-m", "--messageId" }, + description = "Skip message by ID (key is ledgerId" + + " and value is either entryId or entryId[:batchIndex]).", required = true) private Map messageIds; @Override diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index 2ef3dc5cda5e1..4ae88cc1eafa3 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -327,4 +327,22 @@ public void testSkipMessages() throws Exception { verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), eq(messageIds)); } + + @Test + public void testSkipMessagesWithBatchIndex() throws Exception { + String topic = "persistent://public/default/testSkipMessagesWithBatchIndex"; + String ledgerId = "123"; + String entryId = "45"; + String batchIndex = "2"; + Map messageIds = new HashMap<>(); + messageIds.put(ledgerId, entryId + ":" + batchIndex); + + cmdTopics.run(new String[]{ + "skip-messages", topic, + "-s", "test-sub", + "-m", ledgerId + "=" + (entryId + ":" + batchIndex) + }); + + verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), eq(messageIds)); + } } From a4911e5f3dd53efa506b6de15ce8cd876cb4d56a Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Nov 2025 14:58:06 +0800 Subject: [PATCH 11/15] feat: implement SkipMessageIdsRequest to support multiple formats for skipping messages --- .../broker/admin/SkipMessageIdsRequest.java | 227 ++++++++++++++++++ .../admin/impl/PersistentTopicsBase.java | 22 +- .../broker/admin/v1/PersistentTopics.java | 5 +- .../broker/admin/v2/PersistentTopics.java | 4 +- .../persistent/BucketDelayedDeliveryTest.java | 11 +- .../client/admin/SkipMessageIdsRequest.java | 78 ++++++ .../apache/pulsar/client/admin/Topics.java | 16 +- .../client/admin/internal/TopicsImpl.java | 15 +- .../apache/pulsar/admin/cli/CmdTopics.java | 58 ++++- .../pulsar/admin/cli/TestCmdTopics.java | 137 +++++++++-- 10 files changed, 533 insertions(+), 40 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java create mode 100644 pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java new file mode 100644 index 0000000000000..d711e32d8de9a --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java @@ -0,0 +1,227 @@ +/* + * 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.admin; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.netty.buffer.Unpooled; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import lombok.Getter; +import org.apache.pulsar.common.api.proto.MessageIdData; + +/** + * Server-side request body for skipping messages by message IDs with support for multiple formats. + * Normalizes to the legacy map used by Subscription#skipMessages(Map). + */ +@Getter +@JsonDeserialize(using = SkipMessageIdsRequest.Deserializer.class) +public class SkipMessageIdsRequest { + private Map legacyMap; + private final List items = new ArrayList<>(); + + public SkipMessageIdsRequest() { + this.legacyMap = new LinkedHashMap<>(); + } + + private void addItem(long ledgerId, long entryId, Integer batchIndex) { + items.add(new MessageIdItem(ledgerId, entryId, batchIndex)); + } + + @Getter + public static class MessageIdItem { + private final long ledgerId; + private final long entryId; + // nullable + private final Integer batchIndex; + + public MessageIdItem(long ledgerId, long entryId, Integer batchIndex) { + this.ledgerId = ledgerId; + this.entryId = entryId; + this.batchIndex = batchIndex; + } + } + + void setLegacyMap(Map legacyMap) { + this.legacyMap = legacyMap; + } + + public static class Deserializer extends JsonDeserializer { + @Override + public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ctxt) throws IOException { + ObjectCodec codec = p.getCodec(); + JsonNode node = codec.readTree(p); + + Map result = new LinkedHashMap<>(); + SkipMessageIdsRequest r = new SkipMessageIdsRequest(); + + if (node == null || node.isNull()) { + throw new IOException("Invalid skipByMessageIds payload: empty body"); + } + + if (node.isArray()) { + // Treat as default byteArray list + ArrayNode arr = (ArrayNode) node; + for (JsonNode idNode : arr) { + if (idNode != null && !idNode.isNull()) { + appendFromBase64(idNode.asText(), result, r); + } + } + r.setLegacyMap(result); + return r; + } + + if (node.isObject()) { + ObjectNode obj = (ObjectNode) node; + JsonNode typeNode = obj.get("type"); + String type = typeNode != null && !typeNode.isNull() ? typeNode.asText() : null; + JsonNode messageIdsNode = obj.get("messageIds"); + + if (messageIdsNode != null) { + if (messageIdsNode.isArray()) { + ArrayNode arr = (ArrayNode) messageIdsNode; + if (type == null || type.isEmpty() || "byteArray".equalsIgnoreCase(type)) { + for (JsonNode idNode : arr) { + if (idNode != null && !idNode.isNull()) { + appendFromBase64(idNode.asText(), result, r); + } + } + } else if ("messageId".equalsIgnoreCase(type)) { + for (JsonNode idObj : arr) { + if (idObj == null || idObj.isNull()) { + continue; + } + long ledgerId = optLong(idObj.get("ledgerId")); + long entryId = optLong(idObj.get("entryId")); + int batchIndex = optInt(idObj.get("batchIndex"), -1); + if (batchIndex >= 0) { + result.put(Long.toString(ledgerId), entryId + ":" + batchIndex); + r.addItem(ledgerId, entryId, batchIndex); + } else { + result.put(Long.toString(ledgerId), Long.toString(entryId)); + r.addItem(ledgerId, entryId, null); + } + } + } else { + // Unknown type with array payload => reject + throw new IOException("Invalid skipByMessageIds payload: unsupported type for array"); + } + r.setLegacyMap(result); + return r; + } else if (messageIdsNode.isObject()) { + if ("map_of_ledgerId_entryId".equalsIgnoreCase(type)) { + ObjectNode midMap = (ObjectNode) messageIdsNode; + Iterator> fields = midMap.fields(); + while (fields.hasNext()) { + Map.Entry e = fields.next(); + String key = e.getKey(); + String valueStr = asScalarText(e.getValue()); + result.put(key, valueStr); + long ledgerId = Long.parseLong(key); + long entryId = Long.parseLong(valueStr); + r.addItem(ledgerId, entryId, null); + } + r.setLegacyMap(result); + return r; + } + throw new IOException("Invalid skipByMessageIds payload:" + + " object messageIds requires type=map_of_ledgerId_entryId"); + } else { + throw new IOException("Invalid skipByMessageIds payload: unsupported messageIds type"); + } + } + + // No messageIds field => reject legacy map form + throw new IOException("Invalid skipByMessageIds payload: missing messageIds"); + } + + throw new IOException("Invalid skipByMessageIds payload: unsupported top-level JSON"); + } + + private static String asScalarText(JsonNode node) { + if (node == null || node.isNull()) { + return null; + } + if (node.isTextual()) { + return node.asText(); + } + if (node.isNumber()) { + return node.asText(); + } + return node.toString(); + } + + private static long optLong(JsonNode node) { + if (node == null || node.isNull()) { + return 0L; + } + try { + return node.asLong(); + } catch (Exception e) { + return 0L; + } + } + + private static int optInt(JsonNode node, int def) { + if (node == null || node.isNull()) { + return def; + } + try { + return node.asInt(); + } catch (Exception e) { + return def; + } + } + + private static void appendFromBase64(String base64, Map result, SkipMessageIdsRequest r) + throws IOException { + if (base64 == null) { + return; + } + byte[] data = Base64.getDecoder().decode(base64); + MessageIdData idData = new MessageIdData(); + try { + idData.parseFrom(Unpooled.wrappedBuffer(data, 0, data.length), data.length); + } catch (Exception e) { + throw new IOException(e); + } + long ledgerId = idData.getLedgerId(); + long entryId = idData.getEntryId(); + int batchIndex = idData.hasBatchIndex() ? idData.getBatchIndex() : -1; + if (batchIndex >= 0) { + result.put(Long.toString(ledgerId), entryId + ":" + batchIndex); + r.addItem(ledgerId, entryId, batchIndex); + } else { + result.put(Long.toString(ledgerId), Long.toString(entryId)); + r.addItem(ledgerId, entryId, null); + } + } + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 6fa1771a1f200..c9b6ec8080819 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -73,6 +73,7 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.admin.SkipMessageIdsRequest; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.service.AnalyzeBacklogResult; @@ -1949,7 +1950,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, } protected void internalSkipByMessageIds(AsyncResponse asyncResponse, String subName, boolean authoritative, - Map messageIds) { + SkipMessageIdsRequest messageIds) { CompletableFuture validationFuture = validateTopicOperationAsync(topicName, TopicOperation.SKIP, subName); validationFuture = validationFuture.thenCompose(__ -> { if (topicName.isGlobal()) { @@ -1983,7 +1984,7 @@ protected void internalSkipByMessageIds(AsyncResponse asyncResponse, String subN private void internalSkipByMessageIdsForPartitionedTopic(AsyncResponse asyncResponse, PartitionedTopicMetadata partitionMetadata, - Map messageIds, + SkipMessageIdsRequest messageIds, String subName) { final List> futures = new ArrayList<>(partitionMetadata.partitions); PulsarAdmin admin; @@ -1995,9 +1996,18 @@ private void internalSkipByMessageIdsForPartitionedTopic(AsyncResponse asyncResp } for (int i = 0; i < partitionMetadata.partitions; i++) { TopicName topicNamePartition = topicName.getPartition(i); + // Rebuild an Admin API request using the parsed items to avoid legacy-map format + List items = new ArrayList<>(); + for (SkipMessageIdsRequest.MessageIdItem it : messageIds.getItems()) { + items.add(new org.apache.pulsar.client.admin.SkipMessageIdsRequest.MessageIdItem( + it.getLedgerId(), it.getEntryId(), it.getBatchIndex())); + } + org.apache.pulsar.client.admin.SkipMessageIdsRequest req = + org.apache.pulsar.client.admin.SkipMessageIdsRequest.forMessageIds(items); + futures.add(admin .topics() - .skipMessagesAsync(topicNamePartition.toString(), subName, messageIds)); + .skipMessagesAsync(topicNamePartition.toString(), subName, req)); } FutureUtil.waitForAll(futures).handle((result, exception) -> { if (exception != null) { @@ -2015,7 +2025,7 @@ private void internalSkipByMessageIdsForPartitionedTopic(AsyncResponse asyncResp } private void internalSkipByMessageIdsForNonPartitionedTopic(AsyncResponse asyncResponse, - Map messageIds, + SkipMessageIdsRequest messageIds, String subName, boolean authoritative) { validateTopicOwnershipAsync(topicName, authoritative) @@ -2042,13 +2052,13 @@ private void internalSkipByMessageIdsForNonPartitionedTopic(AsyncResponse asyncR } private CompletableFuture internalSkipByMessageIdsForSubscriptionAsync( - PersistentTopic topic, String subName, Map messageIds) { + PersistentTopic topic, String subName, SkipMessageIdsRequest messageIds) { Subscription sub = topic.getSubscription(subName); if (sub == null) { return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, getSubNotFoundErrorMessage(topic.getName(), subName))); } - return sub.skipMessages(messageIds); + return sub.skipMessages(messageIds.getLegacyMap()); } protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResponse, int expireTimeInSeconds, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index f1b167d034348..f525e950441f5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -24,7 +24,6 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; -import java.util.Map; import java.util.Optional; import java.util.Set; import javax.ws.rs.DELETE; @@ -43,6 +42,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.admin.SkipMessageIdsRequest; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.GetStatsOptions; @@ -616,6 +616,7 @@ public void skipAllMessages(@Suspended final AsyncResponse asyncResponse, @PathP @Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}") @ApiOperation(hidden = true, value = "Skip messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namesapce or topic or subscription does not exist") }) @@ -646,7 +647,7 @@ public void skipByMessageIds(@Suspended final AsyncResponse asyncResponse, @Path @PathParam("topic") @Encoded String encodedTopic, @PathParam("subName") String encodedSubName, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, - @ApiParam(value = "The message ID to skip") Map messageIds) { + @ApiParam(value = "The message ID to skip") SkipMessageIdsRequest messageIds) { try { validateTopicName(property, cluster, namespace, encodedTopic); internalSkipByMessageIds(asyncResponse, decode(encodedSubName), authoritative, messageIds); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index a32149c1edaeb..05ed821ef6ed4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -60,6 +60,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.ResetCursorData; +import org.apache.pulsar.broker.admin.SkipMessageIdsRequest; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.naming.NamedEntity; import org.apache.pulsar.common.naming.PartitionedManagedLedgerInfo; @@ -1563,6 +1564,7 @@ public void skipAllMessages( @ApiOperation(value = "Skipping messages on a topic subscription.") @ApiResponses(value = { @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1620,7 +1622,7 @@ public void skipByMessageIds( @PathParam("subName") String encodedSubName, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, - @ApiParam(value = "The message ID to skip") Map messageIds) { + @ApiParam(value = "The message ID to skip") SkipMessageIdsRequest messageIds) { try { validateTopicName(tenant, namespace, encodedTopic); internalSkipByMessageIds(asyncResponse, decode(encodedSubName), authoritative, messageIds); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java index c2bedfcf1b2b9..01d8d96256ba1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/BucketDelayedDeliveryTest.java @@ -29,6 +29,7 @@ import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -48,6 +49,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.client.admin.SkipMessageIdsRequest; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -528,11 +530,12 @@ public void testDelayedMessageCancel() throws Exception { final int cancelMessage = 50; MessageIdImpl messageId = (MessageIdImpl) messageIds.get(cancelMessage); - Map ackMessageIds = new HashMap<>(); - ackMessageIds.put(String.valueOf(messageId.getLedgerId()), String.valueOf(messageId.getEntryId())); + SkipMessageIdsRequest.MessageIdItem item0 = new SkipMessageIdsRequest.MessageIdItem( + messageId.getLedgerId(), messageId.getEntryId(), null); + SkipMessageIdsRequest req = SkipMessageIdsRequest.forMessageIds(Collections.singletonList(item0)); - admin.topics().skipMessages(topic + "-partition-0", subName + "-1", ackMessageIds); - admin.topics().skipMessages(topic + "-partition-1", subName + "-1", ackMessageIds); + admin.topics().skipMessages(topic + "-partition-0", subName + "-1", req); + admin.topics().skipMessages(topic + "-partition-1", subName + "-1", req); assertTrue(latch.await(15, TimeUnit.SECONDS), "Not all messages were received in time"); assertFalse((receivedMessages1.contains("msg-" + cancelMessage) diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java new file mode 100644 index 0000000000000..7583796ee4007 --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java @@ -0,0 +1,78 @@ +/* + * 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.client.admin; + +import java.util.List; +import lombok.Getter; +import lombok.Setter; + +/** + * Request DTO used by the admin client to submit a list of message IDs + * for skipping. It supports multiple formats and is serialized to JSON + * that the broker understands (polymorphic deserialization on server). + *

+ * Supported types: + * - type = "byteArray": messageIds is List of base64-encoded MessageId.toByteArray() + * - type = "messageId": messageIds is List (supports batchIndex) + * - type = "map_of_ledgerId_entryId": messageIds is Map (legacy map) + */ +@Setter +@Getter +public class SkipMessageIdsRequest { + // optional; default is byteArray on server when messageIds is an array of strings + private String type; + // List | List | Map + private Object messageIds; + + public SkipMessageIdsRequest() { + } + + public static SkipMessageIdsRequest forByteArrays(List base64MessageIds) { + SkipMessageIdsRequest r = new SkipMessageIdsRequest(); + r.setType("byteArray"); + r.setMessageIds(base64MessageIds); + return r; + } + + public static SkipMessageIdsRequest forMessageIds(List items) { + SkipMessageIdsRequest r = new SkipMessageIdsRequest(); + r.setType("messageId"); + r.setMessageIds(items); + return r; + } + + /** + * Item representing a messageId as ledgerId, entryId and optional batchIndex. + */ + @Setter + @Getter + public static class MessageIdItem { + private long ledgerId; + private long entryId; + // optional + private Integer batchIndex; + + public MessageIdItem(long ledgerId, long entryId, Integer batchIndex) { + this.ledgerId = ledgerId; + this.entryId = entryId; + this.batchIndex = batchIndex; + } + } +} + diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index fd599919faa00..43aa1f936793b 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -1551,9 +1551,21 @@ PartitionedTopicInternalStats getPartitionedInternalStats(String topic) */ CompletableFuture skipMessagesAsync(String topic, String subName, long numMessages); - void skipMessages(String topic, String subName, Map messageIds) throws PulsarAdminException; + /** + * Skip messages by specifying a list of message IDs in various formats. + * Supports base64-encoded MessageId byte arrays and structured messageId objects with batchIndex. + * + * @param topic topic name (persistent://tenant/namespace/name) + * @param subName subscription name + * @param request request payload supporting multiple messageId formats + * @throws PulsarAdminException in case of error + */ + void skipMessages(String topic, String subName, SkipMessageIdsRequest request) throws PulsarAdminException; - CompletableFuture skipMessagesAsync(String topic, String subName, Map messageIds); + /** + * Async version of {@link #skipMessages(String, String, SkipMessageIdsRequest)}. + */ + CompletableFuture skipMessagesAsync(String topic, String subName, SkipMessageIdsRequest request); /** * Expire all messages older than given N (expireTimeInSeconds) seconds for a given subscription. diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index 243ce508841db..56f7a1294bb48 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; +import org.apache.pulsar.client.admin.SkipMessageIdsRequest; import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Message; @@ -827,17 +828,21 @@ public CompletableFuture skipMessagesAsync(String topic, String subName, l } @Override - public void skipMessages(String topic, String subName, Map messageIds) throws PulsarAdminException { - sync(() -> skipMessagesAsync(topic, subName, messageIds)); + public void skipMessages(String topic, String subName, SkipMessageIdsRequest request) + throws PulsarAdminException { + sync(() -> skipMessagesAsync(topic, subName, request)); } @Override - public CompletableFuture skipMessagesAsync(String topic, String subName, Map messageIds) { + public CompletableFuture skipMessagesAsync(String topic, String subName, + SkipMessageIdsRequest request) { TopicName tn = validateTopic(topic); String encodedSubName = Codec.encode(subName); WebTarget path = topicPath(tn, "subscription", encodedSubName, "skipByMessageIds"); - messageIds = messageIds == null ? new HashMap<>() : messageIds; - return asyncPostRequest(path, Entity.entity(messageIds, MediaType.APPLICATION_JSON)); + if (request == null) { + request = new SkipMessageIdsRequest(); + } + return asyncPostRequest(path, Entity.entity(request, MediaType.APPLICATION_JSON)); } @Override diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 53ff7851d3606..9a6e59ccaa621 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -58,6 +58,7 @@ import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.SkipMessageIdsRequest; import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -865,15 +866,62 @@ private class SkipMessages extends CliCommand { "--subscription" }, description = "Subscription to be skip messages on", required = true) private String subName; - @Option(names = { "-m", "--messageId" }, - description = "Skip message by ID (key is ledgerId" - + " and value is either entryId or entryId[:batchIndex]).", required = true) - private Map messageIds; + @Option(names = { "--messageId-base64" }, description = "Base64-encoded MessageId.toByteArray(); repeatable") + private List messageIdBase64; + + @Option(names = { "--messageId-triplet" }, description = "MessageId as ledgerId:entryId[:batchIndex]; repeatable") + private List messageIdTriplets; @Override void run() throws PulsarAdminException { String topic = validateTopicName(topicName); - getTopics().skipMessages(topic, subName, messageIds); + + int modes = 0; + if (messageIdBase64 != null && !messageIdBase64.isEmpty()) { + modes++; + } + if (messageIdTriplets != null && !messageIdTriplets.isEmpty()) { + modes++; + } + if (modes != 1) { + throw new ParameterException("Specify exactly one of --messageId-base64 or --messageId-triplet"); + } + + if (messageIdBase64 != null && !messageIdBase64.isEmpty()) { + SkipMessageIdsRequest req = SkipMessageIdsRequest.forByteArrays(messageIdBase64); + getTopics().skipMessages(topic, subName, req); + return; + } + + if (messageIdTriplets != null && !messageIdTriplets.isEmpty()) { + List items = new ArrayList<>(); + for (String s : messageIdTriplets) { + if (s == null || s.isEmpty()) { + continue; + } + // Format: ledgerId:entryId[:batchIndex] + String[] parts = s.split(":"); + if (parts.length < 2 || parts.length > 3) { + throw new ParameterException("Invalid --messageId-triplet: " + s + + " (expected ledgerId:entryId[:batchIndex])"); + } + long ledgerId; + long entryId; + Integer batchIndex = null; + try { + ledgerId = Long.parseLong(parts[0]); + entryId = Long.parseLong(parts[1]); + if (parts.length == 3) { + batchIndex = Integer.parseInt(parts[2]); + } + } catch (NumberFormatException e) { + throw new ParameterException("Invalid --messageId-triplet: " + s + ", " + e.getMessage()); + } + items.add(new SkipMessageIdsRequest.MessageIdItem(ledgerId, entryId, batchIndex)); + } + SkipMessageIdsRequest req = SkipMessageIdsRequest.forMessageIds(items); + getTopics().skipMessages(topic, subName, req); + } } } diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index 4ae88cc1eafa3..2c8c16c8a8389 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.common.naming.TopicName.DEFAULT_NAMESPACE; import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT; import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; @@ -39,15 +40,14 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import lombok.Cleanup; import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.client.admin.Lookup; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Schemas; +import org.apache.pulsar.client.admin.SkipMessageIdsRequest; import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicDomain; @@ -314,35 +314,142 @@ public void testSetPersistenceWithUnsetMarkDeleteRate() throws Exception { @Test public void testSkipMessages() throws Exception { String topic = "persistent://public/default/testCancelDelayed"; - String ledgerId = "123"; - String entryId = "45"; - Map messageIds = new HashMap<>(); - messageIds.put(ledgerId, entryId); cmdTopics.run(new String[]{ "skip-messages", topic, "-s", "test-sub", - "-m", ledgerId + "=" + entryId + "--messageId-triplet", "123:45" }); - verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), eq(messageIds)); + verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), + argThat((SkipMessageIdsRequest req) -> { + if (req == null) { + return false; + } + if (req.getType() == null || !req.getType().equals("messageId")) { + return false; + } + Object ids = req.getMessageIds(); + if (!(ids instanceof List)) { + return false; + } + List l = (List) ids; + if (l.size() != 1) { + return false; + } + Object i0 = l.get(0); + if (!(i0 instanceof SkipMessageIdsRequest.MessageIdItem)) { + return false; + } + SkipMessageIdsRequest.MessageIdItem m0 = (SkipMessageIdsRequest.MessageIdItem) i0; + return m0.getLedgerId() == 123L && m0.getEntryId() == 45L && m0.getBatchIndex() == null; + })); } @Test public void testSkipMessagesWithBatchIndex() throws Exception { String topic = "persistent://public/default/testSkipMessagesWithBatchIndex"; - String ledgerId = "123"; - String entryId = "45"; - String batchIndex = "2"; - Map messageIds = new HashMap<>(); - messageIds.put(ledgerId, entryId + ":" + batchIndex); cmdTopics.run(new String[]{ "skip-messages", topic, "-s", "test-sub", - "-m", ledgerId + "=" + (entryId + ":" + batchIndex) + "--messageId-triplet", "123:45:2" }); - verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), eq(messageIds)); + verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), + argThat((SkipMessageIdsRequest req) -> { + if (req == null) { + return false; + } + if (req.getType() == null || !req.getType().equals("messageId")) { + return false; + } + Object ids = req.getMessageIds(); + if (!(ids instanceof List)) { + return false; + } + List l = (List) ids; + if (l.size() != 1) { + return false; + } + Object i0 = l.get(0); + if (!(i0 instanceof SkipMessageIdsRequest.MessageIdItem)) { + return false; + } + SkipMessageIdsRequest.MessageIdItem m0 = (SkipMessageIdsRequest.MessageIdItem) i0; + return m0.getLedgerId() == 123L && m0.getEntryId() == 45L + && Integer.valueOf(2).equals(m0.getBatchIndex()); + })); + } + + @Test + public void testSkipMessagesWithBase64Ids() throws Exception { + String topic = "persistent://public/default/testSkipMessagesWithBase64"; + + cmdTopics.run(new String[]{ + "skip-messages", topic, + "-s", "test-sub", + "--messageId-base64", "CLlgEAQwAA==", + "--messageId-base64", "CLlgEAYwAA==" + }); + + verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), + argThat((SkipMessageIdsRequest req) -> { + if (req == null) { + return false; + } + if (req.getType() == null || !req.getType().equals("byteArray")) { + return false; + } + Object ids = req.getMessageIds(); + if (!(ids instanceof List)) { + return false; + } + List list = (List) ids; + return list.size() == 2 + && "CLlgEAQwAA==".equals(list.get(0)) + && "CLlgEAYwAA==".equals(list.get(1)); + })); + } + + @Test + public void testSkipMessagesWithTriplets() throws Exception { + String topic = "persistent://public/default/testSkipMessagesWithTriplets"; + + cmdTopics.run(new String[]{ + "skip-messages", topic, + "-s", "test-sub", + "--messageId-triplet", "123:45", + "--messageId-triplet", "124:46:2" + }); + + verify(mockTopics).skipMessages(eq(topic), eq("test-sub"), + argThat((SkipMessageIdsRequest req) -> { + if (req == null) { + return false; + } + if (req.getType() == null || !req.getType().equals("messageId")) { + return false; + } + Object ids = req.getMessageIds(); + if (!(ids instanceof List list)) { + return false; + } + if (list.size() != 2) { + return false; + } + Object i0 = list.get(0); + Object i1 = list.get(1); + if (!(i0 instanceof SkipMessageIdsRequest.MessageIdItem m0) + || !(i1 instanceof SkipMessageIdsRequest.MessageIdItem m1)) { + return false; + } + return m0.getLedgerId() == 123L + && m0.getEntryId() == 45L + && m0.getBatchIndex() == null + && m1.getLedgerId() == 124L + && m1.getEntryId() == 46L + && Integer.valueOf(2).equals(m1.getBatchIndex()); + })); } } From 2e775807b8dcab5239bf995ae0768a6029e57055 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Nov 2025 17:41:04 +0800 Subject: [PATCH 12/15] feat: refactor skipMessages to accept List and update related classes --- .../broker/admin/SkipMessageIdsRequest.java | 79 ++++--------------- .../admin/impl/PersistentTopicsBase.java | 42 +++++++++- .../broker/admin/v1/PersistentTopics.java | 1 + .../broker/admin/v2/PersistentTopics.java | 3 +- .../pulsar/broker/service/Subscription.java | 2 +- .../NonPersistentSubscription.java | 3 +- .../persistent/PersistentSubscription.java | 47 +++++------ .../apache/pulsar/admin/cli/CmdTopics.java | 3 +- 8 files changed, 83 insertions(+), 97 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java index d711e32d8de9a..55f1ac708b3d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java @@ -30,47 +30,36 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Base64; -import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import lombok.Getter; import org.apache.pulsar.common.api.proto.MessageIdData; /** * Server-side request body for skipping messages by message IDs with support for multiple formats. - * Normalizes to the legacy map used by Subscription#skipMessages(Map). */ @Getter @JsonDeserialize(using = SkipMessageIdsRequest.Deserializer.class) public class SkipMessageIdsRequest { - private Map legacyMap; private final List items = new ArrayList<>(); - public SkipMessageIdsRequest() { - this.legacyMap = new LinkedHashMap<>(); - } + public SkipMessageIdsRequest() { } private void addItem(long ledgerId, long entryId, Integer batchIndex) { items.add(new MessageIdItem(ledgerId, entryId, batchIndex)); } - @Getter - public static class MessageIdItem { - private final long ledgerId; - private final long entryId; - // nullable - private final Integer batchIndex; - - public MessageIdItem(long ledgerId, long entryId, Integer batchIndex) { - this.ledgerId = ledgerId; - this.entryId = entryId; - this.batchIndex = batchIndex; + public record MessageIdItem(long ledgerId, long entryId, Integer batchIndex) { + public long getLedgerId() { + return ledgerId; + } + + public long getEntryId() { + return entryId; } - } - void setLegacyMap(Map legacyMap) { - this.legacyMap = legacyMap; + public Integer getBatchIndex() { + return batchIndex; + } } public static class Deserializer extends JsonDeserializer { @@ -78,8 +67,6 @@ public static class Deserializer extends JsonDeserializer public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ctxt) throws IOException { ObjectCodec codec = p.getCodec(); JsonNode node = codec.readTree(p); - - Map result = new LinkedHashMap<>(); SkipMessageIdsRequest r = new SkipMessageIdsRequest(); if (node == null || node.isNull()) { @@ -91,10 +78,9 @@ public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ct ArrayNode arr = (ArrayNode) node; for (JsonNode idNode : arr) { if (idNode != null && !idNode.isNull()) { - appendFromBase64(idNode.asText(), result, r); + appendFromBase64(idNode.asText(), r); } } - r.setLegacyMap(result); return r; } @@ -110,7 +96,7 @@ public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ct if (type == null || type.isEmpty() || "byteArray".equalsIgnoreCase(type)) { for (JsonNode idNode : arr) { if (idNode != null && !idNode.isNull()) { - appendFromBase64(idNode.asText(), result, r); + appendFromBase64(idNode.asText(), r); } } } else if ("messageId".equalsIgnoreCase(type)) { @@ -122,10 +108,8 @@ public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ct long entryId = optLong(idObj.get("entryId")); int batchIndex = optInt(idObj.get("batchIndex"), -1); if (batchIndex >= 0) { - result.put(Long.toString(ledgerId), entryId + ":" + batchIndex); r.addItem(ledgerId, entryId, batchIndex); } else { - result.put(Long.toString(ledgerId), Long.toString(entryId)); r.addItem(ledgerId, entryId, null); } } @@ -133,26 +117,10 @@ public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ct // Unknown type with array payload => reject throw new IOException("Invalid skipByMessageIds payload: unsupported type for array"); } - r.setLegacyMap(result); return r; } else if (messageIdsNode.isObject()) { - if ("map_of_ledgerId_entryId".equalsIgnoreCase(type)) { - ObjectNode midMap = (ObjectNode) messageIdsNode; - Iterator> fields = midMap.fields(); - while (fields.hasNext()) { - Map.Entry e = fields.next(); - String key = e.getKey(); - String valueStr = asScalarText(e.getValue()); - result.put(key, valueStr); - long ledgerId = Long.parseLong(key); - long entryId = Long.parseLong(valueStr); - r.addItem(ledgerId, entryId, null); - } - r.setLegacyMap(result); - return r; - } - throw new IOException("Invalid skipByMessageIds payload:" - + " object messageIds requires type=map_of_ledgerId_entryId"); + // legacy map format is no longer supported + throw new IOException("Invalid skipByMessageIds payload: legacy map format is not supported"); } else { throw new IOException("Invalid skipByMessageIds payload: unsupported messageIds type"); } @@ -165,19 +133,6 @@ public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ct throw new IOException("Invalid skipByMessageIds payload: unsupported top-level JSON"); } - private static String asScalarText(JsonNode node) { - if (node == null || node.isNull()) { - return null; - } - if (node.isTextual()) { - return node.asText(); - } - if (node.isNumber()) { - return node.asText(); - } - return node.toString(); - } - private static long optLong(JsonNode node) { if (node == null || node.isNull()) { return 0L; @@ -200,7 +155,7 @@ private static int optInt(JsonNode node, int def) { } } - private static void appendFromBase64(String base64, Map result, SkipMessageIdsRequest r) + private static void appendFromBase64(String base64, SkipMessageIdsRequest r) throws IOException { if (base64 == null) { return; @@ -216,10 +171,8 @@ private static void appendFromBase64(String base64, Map result, long entryId = idData.getEntryId(); int batchIndex = idData.hasBatchIndex() ? idData.getBatchIndex() : -1; if (batchIndex >= 0) { - result.put(Long.toString(ledgerId), entryId + ":" + batchIndex); r.addItem(ledgerId, entryId, batchIndex); } else { - result.put(Long.toString(ledgerId), Long.toString(entryId)); r.addItem(ledgerId, entryId, null); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index c9b6ec8080819..99384f733a5ee 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -36,6 +36,8 @@ import java.util.Base64; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -82,6 +84,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionInvalidCursorPosition; import org.apache.pulsar.broker.service.GetStatsOptions; import org.apache.pulsar.broker.service.MessageExpirer; +import org.apache.pulsar.broker.service.SkipEntry; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; @@ -2058,7 +2061,44 @@ private CompletableFuture internalSkipByMessageIdsForSubscriptionAsync( return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, getSubNotFoundErrorMessage(topic.getName(), subName))); } - return sub.skipMessages(messageIds.getLegacyMap()); + // Build List from parsed items + Map aggregated = new LinkedHashMap<>(); + for (SkipMessageIdsRequest.MessageIdItem it : messageIds.getItems()) { + long ledgerId = it.getLedgerId(); + long entryId = it.getEntryId(); + Integer batchIndex = it.getBatchIndex(); + String key = ledgerId + ":" + entryId; + AggregatedSkip agg = aggregated.computeIfAbsent(key, k -> new AggregatedSkip(ledgerId, entryId)); + if (batchIndex == null) { + agg.full = true; + } else { + agg.indexes.add(batchIndex); + } + } + List skipEntries = new ArrayList<>(aggregated.size()); + for (AggregatedSkip v : aggregated.values()) { + if (v.full) { + skipEntries.add(new SkipEntry(v.ledgerId, v.entryId, null)); + } else { + // sort indexes to have deterministic order + List idx = new ArrayList<>(v.indexes); + Collections.sort(idx); + skipEntries.add(new SkipEntry(v.ledgerId, v.entryId, idx)); + } + } + return sub.skipMessages(skipEntries); + } + + private static final class AggregatedSkip { + final long ledgerId; + final long entryId; + boolean full = false; + final LinkedHashSet indexes = new LinkedHashSet<>(); + + AggregatedSkip(long ledgerId, long entryId) { + this.ledgerId = ledgerId; + this.entryId = entryId; + } } protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResponse, int expireTimeInSeconds, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index f525e950441f5..85a3b58a0e76b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -640,6 +640,7 @@ public void skipMessages(@Suspended final AsyncResponse asyncResponse, @PathPara @ApiOperation(hidden = true, value = "Skip messages on a topic subscription.") @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 400, message = "Bad Request: invalid messageIds format"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namesapce or topic or subscription does not exist") }) public void skipByMessageIds(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 05ed821ef6ed4..013ef42a0e34f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -49,6 +49,7 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.broker.admin.AdminResource; +import org.apache.pulsar.broker.admin.SkipMessageIdsRequest; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.GetStatsOptions; @@ -60,7 +61,6 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.ResetCursorData; -import org.apache.pulsar.broker.admin.SkipMessageIdsRequest; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.naming.NamedEntity; import org.apache.pulsar.common.naming.PartitionedManagedLedgerInfo; @@ -1603,6 +1603,7 @@ public void skipMessages( @ApiResponses(value = { @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), + @ApiResponse(code = 400, message = "Bad Request: invalid messageIds format"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic or subscription does not exist"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index c9ff8226b0d56..540efa3d5f1b4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -80,7 +80,7 @@ default long getNumberOfEntriesDelayed() { CompletableFuture skipMessages(int numMessagesToSkip); - CompletableFuture skipMessages(Map messageIds); + CompletableFuture skipMessages(List entries); CompletableFuture resetCursor(long timestamp); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index 78415bcf936e3..2c469ad8542bc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -38,6 +38,7 @@ import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.GetStatsOptions; +import org.apache.pulsar.broker.service.SkipEntry; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -250,7 +251,7 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { } @Override - public CompletableFuture skipMessages(Map messageIds) { + public CompletableFuture skipMessages(List entries) { // No-op return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 32d2fc67c2983..15313df170171 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -24,8 +24,8 @@ import com.google.common.base.MoreObjects; import io.netty.buffer.ByteBuf; import java.util.ArrayList; -import java.util.Collections; import java.util.BitSet; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -56,8 +56,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.InvalidCursorPositionException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.bookkeeper.mledger.ScanOutcome; +import org.apache.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.ServiceConfiguration; @@ -76,6 +76,7 @@ import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; import org.apache.pulsar.broker.service.GetStatsOptions; +import org.apache.pulsar.broker.service.SkipEntry; import org.apache.pulsar.broker.service.StickyKeyDispatcher; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; @@ -817,7 +818,7 @@ public void skipEntriesFailed(ManagedLedgerException exception, Object ctx) { } @Override - public CompletableFuture skipMessages(Map messageIds) { + public CompletableFuture skipMessages(List entries) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Skipping messages by messageIds, current backlog {}", topicName, subName, cursor.getNumberOfEntriesInBacklog(false)); @@ -831,28 +832,15 @@ public CompletableFuture skipMessages(Map messageIds) { List fullEntryPositions = new ArrayList<>(); Map> partialAckIndexByPos = new HashMap<>(); // key: ledgerId:entryId - for (Map.Entry entry : messageIds.entrySet()) { - final long ledgerId; - final String value = entry.getValue(); - try { - ledgerId = Long.parseLong(entry.getKey()); - } catch (Exception e) { - return CompletableFuture.failedFuture(new NotAllowedException("Invalid message ID ledgerId.")); - } - - try { - if (value.contains(":")) { - int idx = value.indexOf(':'); - long entryId = Long.parseLong(value.substring(0, idx)); - int batchIndex = Integer.parseInt(value.substring(idx + 1)); - String key = ledgerId + ":" + entryId; - partialAckIndexByPos.computeIfAbsent(key, __ -> new ArrayList<>()).add(batchIndex); - } else { - long entryId = Long.parseLong(value); - fullEntryPositions.add(PositionFactory.create(ledgerId, entryId)); - } - } catch (Exception e) { - return CompletableFuture.failedFuture(new NotAllowedException("Invalid message ID value.")); + for (SkipEntry e : entries) { + final long ledgerId = e.getLedgerId(); + final long entryId = e.getEntryId(); + List batchIdx = e.getBatchIndexes(); + if (batchIdx == null || batchIdx.isEmpty()) { + fullEntryPositions.add(PositionFactory.create(ledgerId, entryId)); + } else { + String key = ledgerId + ":" + entryId; + partialAckIndexByPos.computeIfAbsent(key, __ -> new ArrayList<>()).addAll(batchIdx); } } @@ -876,13 +864,13 @@ public CompletableFuture skipMessages(Map messageIds) { cursor.asyncReplayEntries(positionsToLoad, new AsyncCallbacks.ReadEntriesCallback() { @Override - public void readEntriesComplete(List entries, Object ctx) { + public void readEntriesComplete(List readEntries, Object ctx) { try { - List positionsForAck = new ArrayList<>(fullEntryPositions.size() + entries.size()); + List positionsForAck = new ArrayList<>(fullEntryPositions.size() + readEntries.size()); // include full-entry positions positionsForAck.addAll(fullEntryPositions); - for (Entry entry : entries) { + for (Entry entry : readEntries) { try { final long ledgerId = entry.getLedgerId(); final long entryId = entry.getEntryId(); @@ -912,7 +900,8 @@ public void readEntriesComplete(List entries, Object ctx) { bitSet.clear(bi); } long[] ackSet = bitSet.toLongArray(); - Position posWithAckSet = AckSetStateUtil.createPositionWithAckSet(ledgerId, entryId, ackSet); + Position posWithAckSet = AckSetStateUtil.createPositionWithAckSet( + ledgerId, entryId, ackSet); positionsForAck.add(posWithAckSet); } finally { entry.release(); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index 9a6e59ccaa621..f5b41cfa3dbf2 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -869,7 +869,8 @@ private class SkipMessages extends CliCommand { @Option(names = { "--messageId-base64" }, description = "Base64-encoded MessageId.toByteArray(); repeatable") private List messageIdBase64; - @Option(names = { "--messageId-triplet" }, description = "MessageId as ledgerId:entryId[:batchIndex]; repeatable") + @Option(names = { "--messageId-triplet" }, description = "MessageId as ledgerId:entryId[:batchIndex];" + + " repeatable") private List messageIdTriplets; @Override From e7a988a1cd12e8504398e51d5bb277636f79a2fc Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Nov 2025 18:13:35 +0800 Subject: [PATCH 13/15] feat: add SkipEntry model for skipping messages with optional batch indexes --- .../pulsar/broker/service/SkipEntry.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java new file mode 100644 index 0000000000000..bfdd1f13958d3 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java @@ -0,0 +1,46 @@ +/* + * 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.service; + +import java.util.List; +import lombok.Getter; + +/** + * Internal model for skipping messages by entry, with optional batch indexes. + * If {@code batchIndexes} is null or empty, the whole entry is skipped. + */ +@Getter +public final class SkipEntry { + private final long ledgerId; + private final long entryId; + // null or empty => full entry + private final List batchIndexes; + + public SkipEntry(long ledgerId, long entryId, List batchIndexes) { + this.ledgerId = ledgerId; + this.entryId = entryId; + if (batchIndexes == null || batchIndexes.isEmpty()) { + this.batchIndexes = null; + } else { + // make a defensive copy + this.batchIndexes = List.copyOf(batchIndexes); + } + } +} + From 46137890084d4a7874394a650a72752f9a0c10e3 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Nov 2025 18:23:36 +0800 Subject: [PATCH 14/15] fix: update subscription description for clarity and add constructor to MessageIdItem --- .../apache/pulsar/client/admin/SkipMessageIdsRequest.java | 5 +++++ .../main/java/org/apache/pulsar/admin/cli/CmdTopics.java | 6 +++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java index 7583796ee4007..f910c0abff39e 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java @@ -68,6 +68,11 @@ public static class MessageIdItem { // optional private Integer batchIndex; + public MessageIdItem(long ledgerId, long entryId) { + this.ledgerId = ledgerId; + this.entryId = entryId; + } + public MessageIdItem(long ledgerId, long entryId, Integer batchIndex) { this.ledgerId = ledgerId; this.entryId = entryId; diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index f5b41cfa3dbf2..5d9d4d56af977 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -844,7 +844,7 @@ private class Skip extends CliCommand { private String topicName; @Option(names = { "-s", - "--subscription" }, description = "Subscription to be skip messages on", required = true) + "--subscription" }, description = "Subscription to skip messages on", required = true) private String subName; @Option(names = { "-n", "--count" }, description = "Number of messages to skip", required = true) @@ -863,7 +863,7 @@ private class SkipMessages extends CliCommand { private String topicName; @Option(names = { "-s", - "--subscription" }, description = "Subscription to be skip messages on", required = true) + "--subscription" }, description = "Subscription to skip messages on", required = true) private String subName; @Option(names = { "--messageId-base64" }, description = "Base64-encoded MessageId.toByteArray(); repeatable") @@ -933,7 +933,7 @@ private class ExpireMessages extends CliCommand { private String topicName; @Option(names = { "-s", - "--subscription" }, description = "Subscription to be skip messages on", required = true) + "--subscription" }, description = "Subscription to skip messages on", required = true) private String subName; @Option(names = { "-t", "--expireTime" }, description = "Expire messages older than time in seconds " From 275a3cbdea07e9d4d89a636546fd820cb74bec02 Mon Sep 17 00:00:00 2001 From: Denovo1998 Date: Sun, 9 Nov 2025 18:24:48 +0800 Subject: [PATCH 15/15] fix: remove unnecessary whitespace in SkipEntry.java --- .../main/java/org/apache/pulsar/broker/service/SkipEntry.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java index bfdd1f13958d3..0a96faa8bd26f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java @@ -43,4 +43,3 @@ public SkipEntry(long ledgerId, long entryId, List batchIndexes) { } } } -