26
26
import com .google .common .collect .Lists ;
27
27
import io .netty .util .Timeout ;
28
28
import io .netty .util .TimerTask ;
29
+ import java .io .IOException ;
29
30
import java .util .ArrayList ;
30
31
import java .util .Collection ;
31
32
import java .util .Collections ;
@@ -108,6 +109,7 @@ public class MultiTopicsConsumerImpl<T> extends ConsumerBase<T> {
108
109
private final MessageIdAdv startMessageId ;
109
110
private volatile boolean duringSeek = false ;
110
111
private final long startMessageRollbackDurationInSec ;
112
+ private final ConsumerInterceptors <T > internalConsumerInterceptors ;
111
113
MultiTopicsConsumerImpl (PulsarClientImpl client , ConsumerConfigurationData <T > conf ,
112
114
ExecutorProvider executorProvider , CompletableFuture <Consumer <T >> subscribeFuture , Schema <T > schema ,
113
115
ConsumerInterceptors <T > interceptors , boolean createTopicIfDoesNotExist ) {
@@ -137,6 +139,11 @@ public class MultiTopicsConsumerImpl<T> extends ConsumerBase<T> {
137
139
long startMessageRollbackDurationInSec ) {
138
140
super (client , singleTopic , conf , Math .max (2 , conf .getReceiverQueueSize ()), executorProvider , subscribeFuture ,
139
141
schema , interceptors );
142
+ if (interceptors != null ) {
143
+ this .internalConsumerInterceptors = getInternalConsumerInterceptors (interceptors );
144
+ } else {
145
+ this .internalConsumerInterceptors = null ;
146
+ }
140
147
141
148
checkArgument (conf .getReceiverQueueSize () > 0 ,
142
149
"Receiver queue size needs to be greater than 0 for Topics Consumer" );
@@ -316,7 +323,8 @@ private void messageReceived(ConsumerImpl<T> consumer, Message<T> message) {
316
323
CompletableFuture <Message <T >> receivedFuture = nextPendingReceive ();
317
324
if (receivedFuture != null ) {
318
325
unAckedMessageTracker .add (topicMessage .getMessageId (), topicMessage .getRedeliveryCount ());
319
- completePendingReceive (receivedFuture , topicMessage );
326
+ final Message <T > interceptMessage = beforeConsume (topicMessage );
327
+ completePendingReceive (receivedFuture , interceptMessage );
320
328
} else if (enqueueMessageAndCheckBatchReceive (topicMessage ) && hasPendingBatchReceive ()) {
321
329
notifyPendingBatchReceivedCallBack ();
322
330
}
@@ -369,7 +377,7 @@ protected Message<T> internalReceive() throws PulsarClientException {
369
377
checkState (message instanceof TopicMessageImpl );
370
378
unAckedMessageTracker .add (message .getMessageId (), message .getRedeliveryCount ());
371
379
resumeReceivingFromPausedConsumersIfNeeded ();
372
- return message ;
380
+ return beforeConsume ( message ) ;
373
381
} catch (Exception e ) {
374
382
ExceptionHandler .handleInterruptedException (e );
375
383
throw PulsarClientException .unwrap (e );
@@ -388,6 +396,7 @@ protected Message<T> internalReceive(long timeout, TimeUnit unit) throws PulsarC
388
396
decreaseIncomingMessageSize (message );
389
397
checkArgument (message instanceof TopicMessageImpl );
390
398
trackUnAckedMsgIfNoListener (message .getMessageId (), message .getRedeliveryCount ());
399
+ message = beforeConsume (message );
391
400
}
392
401
resumeReceivingFromPausedConsumersIfNeeded ();
393
402
return message ;
@@ -447,7 +456,7 @@ protected CompletableFuture<Message<T>> internalReceiveAsync() {
447
456
checkState (message instanceof TopicMessageImpl );
448
457
unAckedMessageTracker .add (message .getMessageId (), message .getRedeliveryCount ());
449
458
resumeReceivingFromPausedConsumersIfNeeded ();
450
- result .complete (message );
459
+ result .complete (beforeConsume ( message ) );
451
460
}
452
461
});
453
462
return result ;
@@ -1185,7 +1194,7 @@ private ConsumerImpl<T> createInternalConsumer(ConsumerConfigurationData<T> conf
1185
1194
return ConsumerImpl .newConsumerImpl (client , partitionName ,
1186
1195
configurationData , client .externalExecutorProvider (),
1187
1196
partitionIndex , true , listener != null , subFuture ,
1188
- startMessageId , schema , interceptors ,
1197
+ startMessageId , schema , this . internalConsumerInterceptors ,
1189
1198
createIfDoesNotExist , startMessageRollbackDurationInSec );
1190
1199
}
1191
1200
@@ -1595,4 +1604,45 @@ private CompletableFuture<List<Integer>> getExistsPartitions(String topic) {
1595
1604
return list ;
1596
1605
});
1597
1606
}
1607
+
1608
+ private ConsumerInterceptors <T > getInternalConsumerInterceptors (ConsumerInterceptors <T > multiTopicInterceptors ) {
1609
+ return new ConsumerInterceptors <T >(new ArrayList <>()) {
1610
+
1611
+ @ Override
1612
+ public Message <T > beforeConsume (Consumer <T > consumer , Message <T > message ) {
1613
+ return message ;
1614
+ }
1615
+
1616
+ @ Override
1617
+ public void onAcknowledge (Consumer <T > consumer , MessageId messageId , Throwable exception ) {
1618
+ multiTopicInterceptors .onAcknowledge (consumer , messageId , exception );
1619
+ }
1620
+
1621
+ @ Override
1622
+ public void onAcknowledgeCumulative (Consumer <T > consumer ,
1623
+ MessageId messageId , Throwable exception ) {
1624
+ multiTopicInterceptors .onAcknowledgeCumulative (consumer , messageId , exception );
1625
+ }
1626
+
1627
+ @ Override
1628
+ public void onNegativeAcksSend (Consumer <T > consumer , Set <MessageId > set ) {
1629
+ multiTopicInterceptors .onNegativeAcksSend (consumer , set );
1630
+ }
1631
+
1632
+ @ Override
1633
+ public void onAckTimeoutSend (Consumer <T > consumer , Set <MessageId > set ) {
1634
+ multiTopicInterceptors .onAckTimeoutSend (consumer , set );
1635
+ }
1636
+
1637
+ @ Override
1638
+ public void onPartitionsChange (String topicName , int partitions ) {
1639
+ multiTopicInterceptors .onPartitionsChange (topicName , partitions );
1640
+ }
1641
+
1642
+ @ Override
1643
+ public void close () throws IOException {
1644
+ multiTopicInterceptors .close ();
1645
+ }
1646
+ };
1647
+ }
1598
1648
}
0 commit comments