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..55f1ac708b3d9 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/SkipMessageIdsRequest.java @@ -0,0 +1,180 @@ +/* + * 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 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 { + @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; + } + long ledgerId = optLong(idObj.get("ledgerId")); + long entryId = optLong(idObj.get("entryId")); + int batchIndex = optInt(idObj.get("batchIndex"), -1); + if (batchIndex >= 0) { + r.addItem(ledgerId, entryId, batchIndex); + } else { + r.addItem(ledgerId, entryId, null); + } + } + } else { + // Unknown type with array payload => reject + throw new IOException("Invalid skipByMessageIds payload: unsupported type for array"); + } + return r; + } else if (messageIdsNode.isObject()) { + // 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"); + } + } + + // 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 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, 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) { + r.addItem(ledgerId, entryId, batchIndex); + } else { + 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 ec61d58d2afe4..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; @@ -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; @@ -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; @@ -1899,7 +1903,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; @@ -1948,6 +1952,155 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, }); } + protected void internalSkipByMessageIds(AsyncResponse asyncResponse, String subName, boolean authoritative, + SkipMessageIdsRequest 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, + SkipMessageIdsRequest 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); + // 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, req)); + } + 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, + 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 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 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, boolean authoritative) { CompletableFuture future = validateTopicOperationAsync(topicName, TopicOperation.EXPIRE_MESSAGES); 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..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 @@ -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; @@ -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") }) @@ -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.") 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 f288c31ec5274..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; @@ -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"), @@ -1595,6 +1597,43 @@ public void skipMessages( } } + @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 = 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"), + @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") SkipMessageIdsRequest messageIds) { + try { + validateTopicName(tenant, namespace, encodedTopic); + internalSkipByMessageIds(asyncResponse, decode(encodedSubName), authoritative, messageIds); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + } + } + @POST @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on a topic subscription.") 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..0a96faa8bd26f --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SkipEntry.java @@ -0,0 +1,45 @@ +/* + * 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); + } + } +} 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..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,6 +80,8 @@ default long getNumberOfEntriesDelayed() { CompletableFuture skipMessages(int numMessagesToSkip); + CompletableFuture skipMessages(List entries); + CompletableFuture resetCursor(long timestamp); CompletableFuture resetCursor(Position position); 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 549a17b2ae429..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; @@ -249,6 +250,12 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { return CompletableFuture.completedFuture(null); } + @Override + public CompletableFuture skipMessages(List entries) { + // 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 056e9a25fda93..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 @@ -23,11 +23,16 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import io.netty.buffer.ByteBuf; +import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; +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; @@ -50,7 +55,9 @@ 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.bookkeeper.mledger.impl.AckSetStateUtil; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.ServiceConfiguration; @@ -69,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; @@ -809,6 +817,114 @@ public void skipEntriesFailed(ManagedLedgerException exception, Object ctx) { return future; } + @Override + public CompletableFuture skipMessages(List entries) { + 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.")); + } + + // Collect full-entry acks and partial (batchIndex) acks + List fullEntryPositions = new ArrayList<>(); + Map> partialAckIndexByPos = new HashMap<>(); // key: ledgerId:entryId + + 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); + } + } + + // 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 readEntries, Object ctx) { + try { + List positionsForAck = new ArrayList<>(fullEntryPositions.size() + readEntries.size()); + // include full-entry positions + positionsForAck.addAll(fullEntryPositions); + + for (Entry entry : readEntries) { + 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 result; + } + @Override public CompletableFuture resetCursor(long timestamp) { CompletableFuture future = new CompletableFuture<>(); 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 3af7b1b4e0cf3..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 @@ -22,16 +22,22 @@ 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.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; +import java.util.ArrayList; +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; @@ -43,11 +49,14 @@ 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; 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.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -468,6 +477,72 @@ public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType sub admin.topics().deletePartitionedTopic(topic); } + @Test + 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); + Set receivedMessages1 = ConcurrentHashMap.newKeySet(); + Set receivedMessages2 = ConcurrentHashMap.newKeySet(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName(subName + "-1") + .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) + .subscriptionName(subName + "-2") + .subscriptionType(SubscriptionType.Shared) + .messageListener((Consumer c, Message msg) -> { + receivedMessages2.add(msg.getValue()); + c.acknowledgeAsync(msg); + latch.countDown(); + }) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + List messageIds = new ArrayList<>(); + + for (int i = 0; i < 100; i++) { + final long deliverAtTime = System.currentTimeMillis() + 3000L; + MessageId messageId = producer.newMessage() + .key(String.valueOf(i)) + .value("msg-" + i) + .deliverAt(deliverAtTime) + .send(); + messageIds.add(i, messageId); + } + + final int cancelMessage = 50; + MessageIdImpl messageId = (MessageIdImpl) messageIds.get(cancelMessage); + + 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", 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) + || receivedMessages2.contains("msg-" + cancelMessage)) + && (receivedMessages1.size() + receivedMessages2.size() == 99), + "msg-" + cancelMessage + " should have been cancelled but was received"); + } private ManagedCursor findCursor(String topic, String subscriptionName) { PersistentTopic persistentTopic = 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..f910c0abff39e --- /dev/null +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/SkipMessageIdsRequest.java @@ -0,0 +1,83 @@ +/* + * 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) { + this.ledgerId = ledgerId; + this.entryId = entryId; + } + + 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 fbcf0b4a07b1f..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,6 +1551,22 @@ PartitionedTopicInternalStats getPartitionedInternalStats(String topic) */ CompletableFuture skipMessagesAsync(String topic, String subName, long numMessages); + /** + * 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; + + /** + * 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 8a1e0e0f8394a..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; @@ -826,6 +827,24 @@ public CompletableFuture skipMessagesAsync(String topic, String subName, l return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); } + @Override + public void skipMessages(String topic, String subName, SkipMessageIdsRequest request) + throws PulsarAdminException { + sync(() -> skipMessagesAsync(topic, subName, request)); + } + + @Override + 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"); + if (request == null) { + request = new SkipMessageIdsRequest(); + } + return asyncPostRequest(path, Entity.entity(request, MediaType.APPLICATION_JSON)); + } + @Override public void expireMessages(String topic, String subName, long expireTimeInSeconds) throws PulsarAdminException { sync(() -> expireMessagesAsync(topic, subName, expireTimeInSeconds)); 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 e29a4dcd7a859..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 @@ -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; @@ -124,6 +125,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()); @@ -842,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) @@ -855,6 +857,75 @@ 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 skip messages on", required = true) + private String subName; + + @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); + + 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); + } + } + } + @Command(description = "Expire messages that older than given expiry time (in seconds) " + "for the subscription") private class ExpireMessages extends CliCommand { @@ -862,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 " 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 70c58150134e3..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; @@ -46,6 +47,7 @@ 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; @@ -308,4 +310,146 @@ public void testSetPersistenceWithUnsetMarkDeleteRate() throws Exception { verify(mockTopics, times(1)).setPersistence("persistent://public/default/topic", new PersistencePolicies(2, 2, 2, -1.0, null)); } + + @Test + public void testSkipMessages() throws Exception { + String topic = "persistent://public/default/testCancelDelayed"; + + cmdTopics.run(new String[]{ + "skip-messages", topic, + "-s", "test-sub", + "--messageId-triplet", "123:45" + }); + + 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"; + + cmdTopics.run(new String[]{ + "skip-messages", topic, + "-s", "test-sub", + "--messageId-triplet", "123:45: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)) { + 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()); + })); + } }