PIP-330: getMessagesById gets all messages

Motivation

The org.apache.pulsar.client.admin.Topics provides getMessageById(java.lang.String, long, long) method to get the message, which returns one message. If the message id refers to a batch message, we can only get the first message, not all messages.

This behavior affects our analysis of messages by the message id.

Goals

In Scope

Add a method that returns all messages by message id to the org.apache.pulsar.client.admin.Topics interface.

Detailed Design

Design & Implementation Details

  1. Add a set of methods to the org.apache.pulsar.client.admin.Topics interface:
public interface Topics {
    List<Message<byte[]>> getMessagesById(String topic, long ledgerId, long entryId) throws PulsarAdminException;
    CompletableFuture<List<Message<byte[]>>> getMessagesByIdAsync(String topic, long ledgerId, long entryId);
}
  1. Deprecate the following methods in the org.apache.pulsar.client.admin.Topics interface:
public interface Topics {
    /**
     * @deprecated Use {@link #getMessagesById(String, long, long)} instead.
     */
    @Deprecated
    Message<byte[]> getMessageById(String topic, long ledgerId, long entryId) throws PulsarAdminException;

    /**
     * @deprecated Use {@link #getMessagesByIdAsync(String, long, long)} instead.
     */
    @Deprecated
    CompletableFuture<Message<byte[]>> getMessageByIdAsync(String topic, long ledgerId, long entryId);
}

General Notes

This PIP doesn't change the output of bin/pulsar-admin topics get-message-by-id, which still outputs one message.

Links