20
20
21
21
import static com .scurrilous .circe .checksum .Crc32cIntChecksum .computeChecksum ;
22
22
import static org .apache .pulsar .broker .service .AbstractReplicator .REPL_PRODUCER_NAME_DELIMITER ;
23
+ import static org .apache .pulsar .client .impl .GeoReplicationProducerImpl .MSG_PROP_IS_REPL_MARKER ;
24
+ import static org .apache .pulsar .client .impl .GeoReplicationProducerImpl .MSG_PROP_REPL_SOURCE_POSITION ;
23
25
import static org .apache .pulsar .common .protocol .Commands .hasChecksum ;
24
26
import static org .apache .pulsar .common .protocol .Commands .readChecksum ;
25
27
import com .google .common .annotations .VisibleForTesting ;
@@ -87,6 +89,7 @@ public class Producer {
87
89
88
90
private final PublisherStatsImpl stats ;
89
91
private final boolean isRemote ;
92
+ private final boolean isRemoteOrShadow ;
90
93
private final String remoteCluster ;
91
94
private final boolean isNonPersistentTopic ;
92
95
private final boolean isShadowTopic ;
@@ -148,6 +151,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
148
151
149
152
String replicatorPrefix = serviceConf .getReplicatorPrefix () + "." ;
150
153
this .isRemote = producerName .startsWith (replicatorPrefix );
154
+ this .isRemoteOrShadow = isRemoteOrShadow (producerName , serviceConf .getReplicatorPrefix ());
151
155
this .remoteCluster = parseRemoteClusterName (producerName , isRemote , replicatorPrefix );
152
156
153
157
this .isEncrypted = isEncrypted ;
@@ -160,6 +164,13 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
160
164
this .brokerInterceptor = cnx .getBrokerService ().getInterceptor ();
161
165
}
162
166
167
+ /**
168
+ * Difference with "isRemote" is whether the prefix string is end with a dot.
169
+ */
170
+ public static boolean isRemoteOrShadow (String producerName , String replicatorPrefix ) {
171
+ return producerName != null && producerName .startsWith (replicatorPrefix );
172
+ }
173
+
163
174
/**
164
175
* Producer name for replicator is in format.
165
176
* "replicatorPrefix.localCluster" (old)
@@ -281,11 +292,16 @@ private boolean checkCanProduceTxnOnTopic(long sequenceId, ByteBuf headersAndPay
281
292
return true ;
282
293
}
283
294
295
+ private boolean isSupportsReplDedupByLidAndEid () {
296
+ // Non-Persistent topic does not have ledger id or entry id, so it does not support.
297
+ return cnx .isClientSupportsReplDedupByLidAndEid () && topic .isPersistent ();
298
+ }
299
+
284
300
private void publishMessageToTopic (ByteBuf headersAndPayload , long sequenceId , long batchSize , boolean isChunked ,
285
301
boolean isMarker , Position position ) {
286
302
MessagePublishContext messagePublishContext =
287
303
MessagePublishContext .get (this , sequenceId , msgIn , headersAndPayload .readableBytes (),
288
- batchSize , isChunked , System .nanoTime (), isMarker , position );
304
+ batchSize , isChunked , System .nanoTime (), isMarker , position , isSupportsReplDedupByLidAndEid () );
289
305
if (brokerInterceptor != null ) {
290
306
brokerInterceptor
291
307
.onMessagePublish (this , headersAndPayload , messagePublishContext );
@@ -297,7 +313,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenc
297
313
long batchSize , boolean isChunked , boolean isMarker , Position position ) {
298
314
MessagePublishContext messagePublishContext = MessagePublishContext .get (this , lowestSequenceId ,
299
315
highestSequenceId , msgIn , headersAndPayload .readableBytes (), batchSize ,
300
- isChunked , System .nanoTime (), isMarker , position );
316
+ isChunked , System .nanoTime (), isMarker , position , isSupportsReplDedupByLidAndEid () );
301
317
if (brokerInterceptor != null ) {
302
318
brokerInterceptor
303
319
.onMessagePublish (this , headersAndPayload , messagePublishContext );
@@ -393,6 +409,7 @@ private static final class MessagePublishContext implements PublishContext, Runn
393
409
private long batchSize ;
394
410
private boolean chunked ;
395
411
private boolean isMarker ;
412
+ private boolean supportsReplDedupByLidAndEid ;
396
413
397
414
private long startTimeNs ;
398
415
@@ -483,6 +500,11 @@ public long getOriginalSequenceId() {
483
500
return originalSequenceId ;
484
501
}
485
502
503
+ @ Override
504
+ public boolean supportsReplDedupByLidAndEid () {
505
+ return supportsReplDedupByLidAndEid ;
506
+ }
507
+
486
508
@ Override
487
509
public void setOriginalHighestSequenceId (long originalHighestSequenceId ) {
488
510
this .originalHighestSequenceId = originalHighestSequenceId ;
@@ -550,8 +572,12 @@ public void run() {
550
572
// stats
551
573
rateIn .recordMultipleEvents (batchSize , msgSize );
552
574
producer .topic .recordAddLatency (System .nanoTime () - startTimeNs , TimeUnit .NANOSECONDS );
553
- producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , highestSequenceId ,
554
- ledgerId , entryId );
575
+ if (producer .isRemoteOrShadow && producer .isSupportsReplDedupByLidAndEid ()) {
576
+ sendSendReceiptResponseRepl ();
577
+ } else {
578
+ // Repl V1 is the same as normal for this handling.
579
+ sendSendReceiptResponseNormal ();
580
+ }
555
581
producer .cnx .completedSendOperation (producer .isNonPersistentTopic , msgSize );
556
582
if (this .chunked ) {
557
583
producer .chunkedMessageRate .recordEvent ();
@@ -564,8 +590,46 @@ public void run() {
564
590
recycle ();
565
591
}
566
592
593
+ private void sendSendReceiptResponseRepl () {
594
+ // Case-1: is a repl marker.
595
+ boolean isReplMarker = getProperty (MSG_PROP_IS_REPL_MARKER ) != null ;
596
+ if (isReplMarker ) {
597
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , Long .MIN_VALUE ,
598
+ ledgerId , entryId );
599
+
600
+ return ;
601
+ }
602
+ // Case-2: is a repl message.
603
+ Object positionPairObj = getProperty (MSG_PROP_REPL_SOURCE_POSITION );
604
+ if (positionPairObj == null || !(positionPairObj instanceof long [])
605
+ || ((long []) positionPairObj ).length < 2 ) {
606
+ log .error ("[{}] Message can not determine whether the message is duplicated due to the acquired"
607
+ + " messages props were are invalid. producer={}. supportsReplDedupByLidAndEid: {},"
608
+ + " sequence-id {}, prop-{}: not in expected format" ,
609
+ producer .topic .getName (), producer .producerName ,
610
+ supportsReplDedupByLidAndEid (), getSequenceId (),
611
+ MSG_PROP_REPL_SOURCE_POSITION );
612
+ producer .cnx .getCommandSender ().sendSendError (producer .producerId ,
613
+ Math .max (highestSequenceId , sequenceId ),
614
+ ServerError .PersistenceError , "Message can not determine whether the message is"
615
+ + " duplicated due to the acquired messages props were are invalid" );
616
+ return ;
617
+ }
618
+ long [] positionPair = (long []) positionPairObj ;
619
+ long replSequenceLId = positionPair [0 ];
620
+ long replSequenceEId = positionPair [1 ];
621
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , replSequenceLId ,
622
+ replSequenceEId , ledgerId , entryId );
623
+ }
624
+
625
+ private void sendSendReceiptResponseNormal () {
626
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , highestSequenceId ,
627
+ ledgerId , entryId );
628
+ }
629
+
567
630
static MessagePublishContext get (Producer producer , long sequenceId , Rate rateIn , int msgSize ,
568
- long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ) {
631
+ long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ,
632
+ boolean supportsReplDedupByLidAndEid ) {
569
633
MessagePublishContext callback = RECYCLER .get ();
570
634
callback .producer = producer ;
571
635
callback .sequenceId = sequenceId ;
@@ -577,6 +641,7 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
577
641
callback .originalSequenceId = -1L ;
578
642
callback .startTimeNs = startTimeNs ;
579
643
callback .isMarker = isMarker ;
644
+ callback .supportsReplDedupByLidAndEid = supportsReplDedupByLidAndEid ;
580
645
callback .ledgerId = position == null ? -1 : position .getLedgerId ();
581
646
callback .entryId = position == null ? -1 : position .getEntryId ();
582
647
if (callback .propertyMap != null ) {
@@ -586,7 +651,8 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
586
651
}
587
652
588
653
static MessagePublishContext get (Producer producer , long lowestSequenceId , long highestSequenceId , Rate rateIn ,
589
- int msgSize , long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ) {
654
+ int msgSize , long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ,
655
+ boolean supportsReplDedupByLidAndEid ) {
590
656
MessagePublishContext callback = RECYCLER .get ();
591
657
callback .producer = producer ;
592
658
callback .sequenceId = lowestSequenceId ;
@@ -599,6 +665,7 @@ static MessagePublishContext get(Producer producer, long lowestSequenceId, long
599
665
callback .startTimeNs = startTimeNs ;
600
666
callback .chunked = chunked ;
601
667
callback .isMarker = isMarker ;
668
+ callback .supportsReplDedupByLidAndEid = supportsReplDedupByLidAndEid ;
602
669
callback .ledgerId = position == null ? -1 : position .getLedgerId ();
603
670
callback .entryId = position == null ? -1 : position .getEntryId ();
604
671
if (callback .propertyMap != null ) {
@@ -829,7 +896,8 @@ public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, lon
829
896
}
830
897
MessagePublishContext messagePublishContext =
831
898
MessagePublishContext .get (this , sequenceId , highSequenceId , msgIn ,
832
- headersAndPayload .readableBytes (), batchSize , isChunked , System .nanoTime (), isMarker , null );
899
+ headersAndPayload .readableBytes (), batchSize , isChunked , System .nanoTime (), isMarker , null ,
900
+ cnx .isClientSupportsReplDedupByLidAndEid ());
833
901
if (brokerInterceptor != null ) {
834
902
brokerInterceptor
835
903
.onMessagePublish (this , headersAndPayload , messagePublishContext );
0 commit comments