Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
19 commits
Select commit Hold shift + click to select a range
7afe161
Implementing delayed message cancellation in pulsar
Denovo1998 Jan 29, 2025
88ffdf9
feat(broker): Add support for canceling delayed messages in DelayedDe…
Denovo1998 Feb 8, 2025
212f174
add cancelDelayedMessage admin api.
Denovo1998 May 31, 2025
8686255
Merge branch 'master' into delay_msg_cancel
Denovo1998 May 31, 2025
6fe2b11
the cancel command does not need to be added to the sharedBucketPrior…
Denovo1998 Jun 1, 2025
dc079a6
fix test.
Denovo1998 Jun 1, 2025
f66f1b5
clean up useless canceledMessages.
Denovo1998 Jun 1, 2025
391a426
Implement the delayed message cancellation function through acknowled…
Denovo1998 Jun 8, 2025
d7f025c
feat(admin): add skipMessages by message IDs and remove cancelDelayed…
Denovo1998 Jul 17, 2025
5f170cc
use skipByMessageIds as the new path
Denovo1998 Jul 31, 2025
bce6ae3
Merge branch 'master' into delay_msg_cancel
Denovo1998 Nov 6, 2025
347e52f
feat: enhance skipMessages functionality to support batch indices
Denovo1998 Nov 9, 2025
a4911e5
feat: implement SkipMessageIdsRequest to support multiple formats for…
Denovo1998 Nov 9, 2025
2e77580
feat: refactor skipMessages to accept List<SkipEntry> and update rela…
Denovo1998 Nov 9, 2025
e7a988a
feat: add SkipEntry model for skipping messages with optional batch i…
Denovo1998 Nov 9, 2025
4613789
fix: update subscription description for clarity and add constructor …
Denovo1998 Nov 9, 2025
275a3cb
fix: remove unnecessary whitespace in SkipEntry.java
Denovo1998 Nov 9, 2025
a88ace7
Merge branch 'master' into delay_msg_cancel
Denovo1998 Feb 19, 2026
b5efe78
feat(broker): Support skipping messages by ID on partitioned topics
Denovo1998 Feb 19, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,201 @@
/*
* 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.List;
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.
*/
@Getter
@JsonDeserialize(using = SkipMessageIdsRequest.Deserializer.class)
public class SkipMessageIdsRequest {
private final List<MessageIdItem> items = new ArrayList<>();

public SkipMessageIdsRequest() { }

private void addItem(long ledgerId, long entryId, Integer batchIndex) {
items.add(new MessageIdItem(ledgerId, entryId, batchIndex));
}

public record MessageIdItem(long ledgerId, long entryId, Integer batchIndex) {
public long getLedgerId() {
return ledgerId;
}

public long getEntryId() {
return entryId;
}

public Integer getBatchIndex() {
return batchIndex;
}
}

public static class Deserializer extends JsonDeserializer<SkipMessageIdsRequest> {
@Override
public SkipMessageIdsRequest deserialize(JsonParser p, DeserializationContext ctxt) throws IOException {
ObjectCodec codec = p.getCodec();
JsonNode node = codec.readTree(p);
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(), r);
}
}
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(), r);
}
}
} else if ("messageId".equalsIgnoreCase(type)) {
for (JsonNode idObj : arr) {
if (idObj == null || idObj.isNull()) {
continue;
}
if (!idObj.isObject()) {
throw new IOException("Invalid skipByMessageIds payload:"
+ " messageIds elements must be objects");
}
long ledgerId = requiredLong(idObj.get("ledgerId"), "ledgerId");
long entryId = requiredLong(idObj.get("entryId"), "entryId");
Integer batchIndex = optionalNonNegativeInt(idObj.get("batchIndex"), "batchIndex");
r.addItem(ledgerId, entryId, batchIndex);
}
} else {
// Unknown type with array payload => reject
throw new IOException("Invalid skipByMessageIds payload: unsupported type for array");
}
return r;
} else if (messageIdsNode.isObject()) {
throw new IOException("Invalid skipByMessageIds payload: messageIds must be an array");
} else {
throw new IOException("Invalid skipByMessageIds payload: unsupported messageIds type");
}
}

// No messageIds field => reject
throw new IOException("Invalid skipByMessageIds payload: missing messageIds");
}

throw new IOException("Invalid skipByMessageIds payload: unsupported top-level JSON");
}

private static long requiredLong(JsonNode node, String fieldName) throws IOException {
if (node == null || node.isNull()) {
throw new IOException("Invalid skipByMessageIds payload: missing " + fieldName);
}
try {
if (node.isNumber()) {
return node.longValue();
}
if (node.isTextual()) {
return Long.parseLong(node.asText());
}
} catch (Exception e) {
throw new IOException("Invalid skipByMessageIds payload: invalid " + fieldName, e);
}
throw new IOException("Invalid skipByMessageIds payload: invalid " + fieldName);
}

private static Integer optionalNonNegativeInt(JsonNode node, String fieldName) throws IOException {
if (node == null || node.isNull()) {
return null;
}
try {
int v;
if (node.isNumber()) {
v = node.intValue();
} else if (node.isTextual()) {
v = Integer.parseInt(node.asText());
} else {
throw new IOException("Invalid skipByMessageIds payload: invalid " + fieldName);
}
return v >= 0 ? v : null;
} catch (NumberFormatException e) {
throw new IOException("Invalid skipByMessageIds payload: invalid " + fieldName, e);
}
}

private static void appendFromBase64(String base64, SkipMessageIdsRequest r)
throws IOException {
if (base64 == null) {
return;
}
byte[] data;
try {
data = Base64.getDecoder().decode(base64);
} catch (IllegalArgumentException e) {
throw new IOException("Invalid skipByMessageIds payload: invalid base64 messageId", e);
}
if (data.length == 0) {
throw new IOException("Invalid skipByMessageIds payload: invalid base64 messageId (empty)");
}
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) {
r.addItem(ledgerId, entryId, batchIndex);
} else {
r.addItem(ledgerId, entryId, null);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -73,6 +75,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;
Expand All @@ -81,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;
Expand Down Expand Up @@ -1915,7 +1919,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;
Expand Down Expand Up @@ -1964,6 +1968,107 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName,
});
}

protected void internalSkipByMessageIds(AsyncResponse asyncResponse, String subName, boolean authoritative,
SkipMessageIdsRequest messageIds) {
CompletableFuture<Void> 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() && 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);
}
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 internalSkipByMessageIdsForNonPartitionedTopic(AsyncResponse asyncResponse,
SkipMessageIdsRequest 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<Void> internalSkipByMessageIdsForSubscriptionAsync(
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)));
}
// Build List<SkipEntry> from parsed items
Map<String, AggregatedSkip> 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<SkipEntry> 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<Integer> 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<Integer> indexes = new LinkedHashSet<>();

AggregatedSkip(long ledgerId, long entryId) {
this.ledgerId = ledgerId;
this.entryId = entryId;
}
}

protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResponse, int expireTimeInSeconds,
boolean authoritative) {
CompletableFuture<Void> future = validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,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;
Expand Down Expand Up @@ -615,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") })
Expand All @@ -633,6 +635,30 @@ public void skipMessages(@Suspended final AsyncResponse asyncResponse, @PathPara
}
}

@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 = 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,
@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") SkipMessageIdsRequest messageIds) {
try {
validateTopicName(property, cluster, namespace, encodedTopic);
internalSkipByMessageIds(asyncResponse, decode(encodedSubName), authoritative, messageIds);
} catch (WebApplicationException wae) {
asyncResponse.resume(wae);
} catch (Exception e) {
asyncResponse.resume(new RestException(e));
}
}

@POST
@Path("/{property}/{cluster}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}")
@ApiOperation(hidden = true, value = "Expire messages on a topic subscription.")
Expand Down
Loading