@InterfaceAudience.Public @InterfaceStability.Stable public interface Reader<T> extends Closeable
Modifier and Type | Method and Description |
---|---|
CompletableFuture<Void> |
closeAsync()
Asynchronously close the reader and stop the broker to push more messages.
|
String |
getTopic() |
boolean |
hasMessageAvailable()
Check if there is any message available to read from the current position.
|
CompletableFuture<Boolean> |
hasMessageAvailableAsync()
Asynchronously check if there is any message available to read from the current position.
|
boolean |
hasReachedEndOfTopic()
Return true if the topic was terminated and this reader has reached the end of the topic.
|
boolean |
isConnected() |
Message<T> |
readNext()
Read the next message in the topic.
|
Message<T> |
readNext(int timeout,
TimeUnit unit)
Read the next message in the topic waiting for a maximum time.
|
CompletableFuture<Message<T>> |
readNextAsync()
Read asynchronously the next message in the topic.
|
void |
seek(long timestamp)
Reset the subscription associated with this reader to a specific message publish time.
|
void |
seek(MessageId messageId)
Reset the subscription associated with this reader to a specific message id.
|
CompletableFuture<Void> |
seekAsync(long timestamp)
Reset the subscription associated with this reader to a specific message publish time.
|
CompletableFuture<Void> |
seekAsync(MessageId messageId)
Reset the subscription associated with this reader to a specific message id.
|
String getTopic()
Message<T> readNext() throws PulsarClientException
This method will block until a message is available.
PulsarClientException
Message<T> readNext(int timeout, TimeUnit unit) throws PulsarClientException
Returns null if no message is received before the timeout.
PulsarClientException
CompletableFuture<Message<T>> readNextAsync()
readNextAsync()
should be called subsequently once returned CompletableFuture
gets complete
with received message. Else it creates backlog of receive requests in the application.
The returned future can be cancelled before completion by calling .cancel(false)
(CompletableFuture.cancel(boolean)
) to remove it from the the backlog of receive requests. Another
choice for ensuring a proper clean up of the returned future is to use the CompletableFuture.orTimeout method
which is available on JDK9+. That would remove it from the backlog of receive requests if receiving exceeds
the timeout.
PulsarClientException
if the reader
is already closed.CompletableFuture<Void> closeAsync()
boolean hasReachedEndOfTopic()
Note that this only applies to a "terminated" topic (where the topic is "sealed" and no
more messages can be published) and not just that the reader is simply caught up with
the publishers. Use hasMessageAvailable()
to check for for that.
boolean hasMessageAvailable() throws PulsarClientException
This check can be used by an application to scan through a topic and stop when the reader reaches the current last published message. For example:
while (reader.hasMessageAvailable()) {
Message<String> msg = reader.readNext();
// Do something
}
// Done reading
Note that this call might be blocking (see hasMessageAvailableAsync()
for async version) and
that even if this call returns true, that will not guarantee that a subsequent call to readNext()
will not block.
PulsarClientException
- if there was any error in the operationCompletableFuture<Boolean> hasMessageAvailableAsync()
This check can be used by an application to scan through a topic and stop when the reader reaches the current last published message.
PulsarClientException
if there was any error in the operationboolean isConnected()
void seek(MessageId messageId) throws PulsarClientException
The message id can either be a specific message or represent the first or last messages in the topic.
MessageId.earliest
: Reset the reader on the earliest message available in the topic
MessageId.latest
: Reset the reader on the latest message in the topic
Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on the individual partitions.
messageId
- the message id where to reposition the readerPulsarClientException
void seek(long timestamp) throws PulsarClientException
Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on the individual partitions.
timestamp
- the message publish time where to reposition the readerPulsarClientException
CompletableFuture<Void> seekAsync(MessageId messageId)
The message id can either be a specific message or represent the first or last messages in the topic.
MessageId.earliest
: Reset the reader on the earliest message available in the topic
MessageId.latest
: Reset the reader on the latest message in the topic
Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on the individual partitions.
messageId
- the message id where to position the readerCompletableFuture<Void> seekAsync(long timestamp)
Note: this operation can only be done on non-partitioned topics. For these, one can rather perform the seek() on the individual partitions.
timestamp
- the message publish time where to position the readerCopyright © 2017–2021 Apache Software Foundation. All rights reserved.