4949import java .util .function .Function ;
5050import java .util .stream .Collectors ;
5151import java .util .stream .IntStream ;
52+ import javax .annotation .Nullable ;
5253import org .apache .commons .lang3 .tuple .Pair ;
5354import org .apache .pulsar .client .api .BatchReceivePolicy ;
5455import org .apache .pulsar .client .api .Consumer ;
@@ -101,7 +102,8 @@ public class MultiTopicsConsumerImpl<T> extends ConsumerBase<T> {
101102 private final MultiTopicConsumerStatsRecorderImpl stats ;
102103 private final ConsumerConfigurationData <T > internalConfig ;
103104
104- private volatile MessageIdAdv startMessageId ;
105+ private final MessageIdAdv startMessageId ;
106+ private volatile boolean duringSeek = false ;
105107 private final long startMessageRollbackDurationInSec ;
106108 MultiTopicsConsumerImpl (PulsarClientImpl client , ConsumerConfigurationData <T > conf ,
107109 ExecutorProvider executorProvider , CompletableFuture <Consumer <T >> subscribeFuture , Schema <T > schema ,
@@ -235,6 +237,10 @@ private void startReceivingMessages(List<ConsumerImpl<T>> newConsumers) {
235237 }
236238
237239 private void receiveMessageFromConsumer (ConsumerImpl <T > consumer , boolean batchReceive ) {
240+ if (duringSeek ) {
241+ log .info ("[{}] Pause receiving messages for topic {} due to seek" , subscription , consumer .getTopic ());
242+ return ;
243+ }
238244 CompletableFuture <List <Message <T >>> messagesFuture ;
239245 if (batchReceive ) {
240246 messagesFuture = consumer .batchReceiveAsync ().thenApply (msgs -> ((MessagesImpl <T >) msgs ).getMessageList ());
@@ -252,7 +258,7 @@ private void receiveMessageFromConsumer(ConsumerImpl<T> consumer, boolean batchR
252258 }
253259 // Process the message, add to the queue and trigger listener or async callback
254260 messages .forEach (msg -> {
255- if (isValidConsumerEpoch ((MessageImpl <T >) msg )) {
261+ if (isValidConsumerEpoch ((MessageImpl <T >) msg ) && ! duringSeek ) {
256262 messageReceived (consumer , msg );
257263 }
258264 });
@@ -748,12 +754,7 @@ public void seek(Function<String, Object> function) throws PulsarClientException
748754
749755 @ Override
750756 public CompletableFuture <Void > seekAsync (Function <String , Object > function ) {
751- List <CompletableFuture <Void >> futures = new ArrayList <>(consumers .size ());
752- consumers .values ().forEach (consumer -> futures .add (consumer .seekAsync (function )));
753- unAckedMessageTracker .clear ();
754- incomingMessages .clear ();
755- resetIncomingMessageSize ();
756- return FutureUtil .waitForAll (futures );
757+ return seekAllAsync (consumer -> consumer .seekAsync (function ));
757758 }
758759
759760 @ Override
@@ -775,25 +776,44 @@ public CompletableFuture<Void> seekAsync(MessageId messageId) {
775776 );
776777 }
777778
778- final CompletableFuture <Void > seekFuture ;
779779 if (internalConsumer == null ) {
780- List <CompletableFuture <Void >> futures = new ArrayList <>(consumers .size ());
781- consumers .values ().forEach (consumerImpl -> futures .add (consumerImpl .seekAsync (messageId )));
782- seekFuture = FutureUtil .waitForAll (futures );
780+ return seekAllAsync (consumer -> consumer .seekAsync (messageId ));
783781 } else {
784- seekFuture = internalConsumer .seekAsync (messageId );
782+ beforeSeek ();
783+ final CompletableFuture <Void > future = new CompletableFuture <>();
784+ internalConsumer .seekAsync (messageId ).whenComplete ((__ , e ) -> afterSeek (future , e ));
785+ return future ;
785786 }
787+ }
786788
789+ @ Override
790+ public CompletableFuture <Void > seekAsync (long timestamp ) {
791+ return seekAllAsync (consumer -> consumer .seekAsync (timestamp ));
792+ }
793+
794+ private CompletableFuture <Void > seekAllAsync (Function <ConsumerImpl <T >, CompletableFuture <Void >> seekFunc ) {
795+ beforeSeek ();
796+ final CompletableFuture <Void > future = new CompletableFuture <>();
797+ FutureUtil .waitForAll (consumers .values ().stream ().map (seekFunc ).collect (Collectors .toList ()))
798+ .whenComplete ((__ , e ) -> afterSeek (future , e ));
799+ return future ;
800+ }
801+
802+ private void beforeSeek () {
803+ duringSeek = true ;
787804 unAckedMessageTracker .clear ();
788805 clearIncomingMessages ();
789- return seekFuture ;
790806 }
791807
792- @ Override
793- public CompletableFuture <Void > seekAsync (long timestamp ) {
794- List <CompletableFuture <Void >> futures = new ArrayList <>(consumers .size ());
795- consumers .values ().forEach (consumer -> futures .add (consumer .seekAsync (timestamp )));
796- return FutureUtil .waitForAll (futures );
808+ private void afterSeek (CompletableFuture <Void > seekFuture , @ Nullable Throwable throwable ) {
809+ duringSeek = false ;
810+ log .info ("[{}] Resume receiving messages for {} since seek is done" , subscription , consumers .keySet ());
811+ startReceivingMessages (new ArrayList <>(consumers .values ()));
812+ if (throwable == null ) {
813+ seekFuture .complete (null );
814+ } else {
815+ seekFuture .completeExceptionally (throwable );
816+ }
797817 }
798818
799819 @ Override
0 commit comments