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 ;
@@ -151,6 +154,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
151
154
152
155
String replicatorPrefix = serviceConf .getReplicatorPrefix () + "." ;
153
156
this .isRemote = producerName .startsWith (replicatorPrefix );
157
+ this .isRemoteOrShadow = isRemoteOrShadow (producerName , serviceConf .getReplicatorPrefix ());
154
158
this .remoteCluster = parseRemoteClusterName (producerName , isRemote , replicatorPrefix );
155
159
156
160
this .isEncrypted = isEncrypted ;
@@ -162,6 +166,13 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
162
166
this .brokerInterceptor = cnx .getBrokerService ().getInterceptor ();
163
167
}
164
168
169
+ /**
170
+ * Difference with "isRemote" is whether the prefix string is end with a dot.
171
+ */
172
+ public static boolean isRemoteOrShadow (String producerName , String replicatorPrefix ) {
173
+ return producerName != null && producerName .startsWith (replicatorPrefix );
174
+ }
175
+
165
176
/**
166
177
* Producer name for replicator is in format.
167
178
* "replicatorPrefix.localCluster" (old)
@@ -270,11 +281,16 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he
270
281
return true ;
271
282
}
272
283
284
+ private boolean isSupportsReplDedupByLidAndEid () {
285
+ // Non-Persistent topic does not have ledger id or entry id, so it does not support.
286
+ return cnx .isClientSupportsReplDedupByLidAndEid () && topic .isPersistent ();
287
+ }
288
+
273
289
private void publishMessageToTopic (ByteBuf headersAndPayload , long sequenceId , long batchSize , boolean isChunked ,
274
290
boolean isMarker , Position position ) {
275
291
MessagePublishContext messagePublishContext =
276
292
MessagePublishContext .get (this , sequenceId , msgIn , headersAndPayload .readableBytes (),
277
- batchSize , isChunked , System .nanoTime (), isMarker , position );
293
+ batchSize , isChunked , System .nanoTime (), isMarker , position , isSupportsReplDedupByLidAndEid () );
278
294
if (brokerInterceptor != null ) {
279
295
brokerInterceptor
280
296
.onMessagePublish (this , headersAndPayload , messagePublishContext );
@@ -286,7 +302,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenc
286
302
long batchSize , boolean isChunked , boolean isMarker , Position position ) {
287
303
MessagePublishContext messagePublishContext = MessagePublishContext .get (this , lowestSequenceId ,
288
304
highestSequenceId , msgIn , headersAndPayload .readableBytes (), batchSize ,
289
- isChunked , System .nanoTime (), isMarker , position );
305
+ isChunked , System .nanoTime (), isMarker , position , isSupportsReplDedupByLidAndEid () );
290
306
if (brokerInterceptor != null ) {
291
307
brokerInterceptor
292
308
.onMessagePublish (this , headersAndPayload , messagePublishContext );
@@ -382,6 +398,7 @@ private static final class MessagePublishContext implements PublishContext, Runn
382
398
private long batchSize ;
383
399
private boolean chunked ;
384
400
private boolean isMarker ;
401
+ private boolean supportsReplDedupByLidAndEid ;
385
402
386
403
private long startTimeNs ;
387
404
@@ -472,6 +489,11 @@ public long getOriginalSequenceId() {
472
489
return originalSequenceId ;
473
490
}
474
491
492
+ @ Override
493
+ public boolean supportsReplDedupByLidAndEid () {
494
+ return supportsReplDedupByLidAndEid ;
495
+ }
496
+
475
497
@ Override
476
498
public void setOriginalHighestSequenceId (long originalHighestSequenceId ) {
477
499
this .originalHighestSequenceId = originalHighestSequenceId ;
@@ -539,8 +561,12 @@ public void run() {
539
561
// stats
540
562
rateIn .recordMultipleEvents (batchSize , msgSize );
541
563
producer .topic .recordAddLatency (System .nanoTime () - startTimeNs , TimeUnit .NANOSECONDS );
542
- producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , highestSequenceId ,
543
- ledgerId , entryId );
564
+ if (producer .isRemoteOrShadow && producer .isSupportsReplDedupByLidAndEid ()) {
565
+ sendSendReceiptResponseRepl ();
566
+ } else {
567
+ // Repl V1 is the same as normal for this handling.
568
+ sendSendReceiptResponseNormal ();
569
+ }
544
570
producer .cnx .completedSendOperation (producer .isNonPersistentTopic , msgSize );
545
571
if (this .chunked ) {
546
572
producer .chunkedMessageRate .recordEvent ();
@@ -553,8 +579,46 @@ public void run() {
553
579
recycle ();
554
580
}
555
581
582
+ private void sendSendReceiptResponseRepl () {
583
+ // Case-1: is a repl marker.
584
+ boolean isReplMarker = getProperty (MSG_PROP_IS_REPL_MARKER ) != null ;
585
+ if (isReplMarker ) {
586
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , Long .MIN_VALUE ,
587
+ ledgerId , entryId );
588
+
589
+ return ;
590
+ }
591
+ // Case-2: is a repl message.
592
+ Object positionPairObj = getProperty (MSG_PROP_REPL_SOURCE_POSITION );
593
+ if (positionPairObj == null || !(positionPairObj instanceof long [])
594
+ || ((long []) positionPairObj ).length < 2 ) {
595
+ log .error ("[{}] Message can not determine whether the message is duplicated due to the acquired"
596
+ + " messages props were are invalid. producer={}. supportsReplDedupByLidAndEid: {},"
597
+ + " sequence-id {}, prop-{}: not in expected format" ,
598
+ producer .topic .getName (), producer .producerName ,
599
+ supportsReplDedupByLidAndEid (), getSequenceId (),
600
+ MSG_PROP_REPL_SOURCE_POSITION );
601
+ producer .cnx .getCommandSender ().sendSendError (producer .producerId ,
602
+ Math .max (highestSequenceId , sequenceId ),
603
+ ServerError .PersistenceError , "Message can not determine whether the message is"
604
+ + " duplicated due to the acquired messages props were are invalid" );
605
+ return ;
606
+ }
607
+ long [] positionPair = (long []) positionPairObj ;
608
+ long replSequenceLId = positionPair [0 ];
609
+ long replSequenceEId = positionPair [1 ];
610
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , replSequenceLId ,
611
+ replSequenceEId , ledgerId , entryId );
612
+ }
613
+
614
+ private void sendSendReceiptResponseNormal () {
615
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , highestSequenceId ,
616
+ ledgerId , entryId );
617
+ }
618
+
556
619
static MessagePublishContext get (Producer producer , long sequenceId , Rate rateIn , int msgSize ,
557
- long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ) {
620
+ long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ,
621
+ boolean supportsReplDedupByLidAndEid ) {
558
622
MessagePublishContext callback = RECYCLER .get ();
559
623
callback .producer = producer ;
560
624
callback .sequenceId = sequenceId ;
@@ -566,6 +630,7 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
566
630
callback .originalSequenceId = -1L ;
567
631
callback .startTimeNs = startTimeNs ;
568
632
callback .isMarker = isMarker ;
633
+ callback .supportsReplDedupByLidAndEid = supportsReplDedupByLidAndEid ;
569
634
callback .ledgerId = position == null ? -1 : position .getLedgerId ();
570
635
callback .entryId = position == null ? -1 : position .getEntryId ();
571
636
if (callback .propertyMap != null ) {
@@ -575,7 +640,8 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
575
640
}
576
641
577
642
static MessagePublishContext get (Producer producer , long lowestSequenceId , long highestSequenceId , Rate rateIn ,
578
- int msgSize , long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ) {
643
+ int msgSize , long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ,
644
+ boolean supportsReplDedupByLidAndEid ) {
579
645
MessagePublishContext callback = RECYCLER .get ();
580
646
callback .producer = producer ;
581
647
callback .sequenceId = lowestSequenceId ;
@@ -588,6 +654,7 @@ static MessagePublishContext get(Producer producer, long lowestSequenceId, long
588
654
callback .startTimeNs = startTimeNs ;
589
655
callback .chunked = chunked ;
590
656
callback .isMarker = isMarker ;
657
+ callback .supportsReplDedupByLidAndEid = supportsReplDedupByLidAndEid ;
591
658
callback .ledgerId = position == null ? -1 : position .getLedgerId ();
592
659
callback .entryId = position == null ? -1 : position .getEntryId ();
593
660
if (callback .propertyMap != null ) {
@@ -815,7 +882,8 @@ public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, lon
815
882
}
816
883
MessagePublishContext messagePublishContext =
817
884
MessagePublishContext .get (this , sequenceId , highSequenceId , msgIn ,
818
- headersAndPayload .readableBytes (), batchSize , isChunked , System .nanoTime (), isMarker , null );
885
+ headersAndPayload .readableBytes (), batchSize , isChunked , System .nanoTime (), isMarker , null ,
886
+ cnx .isClientSupportsReplDedupByLidAndEid ());
819
887
if (brokerInterceptor != null ) {
820
888
brokerInterceptor
821
889
.onMessagePublish (this , headersAndPayload , messagePublishContext );
0 commit comments