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 ;
25
23
import static org .apache .pulsar .common .protocol .Commands .hasChecksum ;
26
24
import static org .apache .pulsar .common .protocol .Commands .readChecksum ;
27
25
import com .google .common .annotations .VisibleForTesting ;
@@ -89,7 +87,6 @@ public class Producer {
89
87
90
88
private final PublisherStatsImpl stats ;
91
89
private final boolean isRemote ;
92
- private final boolean isRemoteOrShadow ;
93
90
private final String remoteCluster ;
94
91
private final boolean isNonPersistentTopic ;
95
92
private final boolean isShadowTopic ;
@@ -151,7 +148,6 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
151
148
152
149
String replicatorPrefix = serviceConf .getReplicatorPrefix () + "." ;
153
150
this .isRemote = producerName .startsWith (replicatorPrefix );
154
- this .isRemoteOrShadow = isRemoteOrShadow (producerName , serviceConf .getReplicatorPrefix ());
155
151
this .remoteCluster = parseRemoteClusterName (producerName , isRemote , replicatorPrefix );
156
152
157
153
this .isEncrypted = isEncrypted ;
@@ -164,13 +160,6 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
164
160
this .brokerInterceptor = cnx .getBrokerService ().getInterceptor ();
165
161
}
166
162
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
-
174
163
/**
175
164
* Producer name for replicator is in format.
176
165
* "replicatorPrefix.localCluster" (old)
@@ -292,16 +281,11 @@ private boolean checkCanProduceTxnOnTopic(long sequenceId, ByteBuf headersAndPay
292
281
return true ;
293
282
}
294
283
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
-
300
284
private void publishMessageToTopic (ByteBuf headersAndPayload , long sequenceId , long batchSize , boolean isChunked ,
301
285
boolean isMarker , Position position ) {
302
286
MessagePublishContext messagePublishContext =
303
287
MessagePublishContext .get (this , sequenceId , msgIn , headersAndPayload .readableBytes (),
304
- batchSize , isChunked , System .nanoTime (), isMarker , position , isSupportsReplDedupByLidAndEid () );
288
+ batchSize , isChunked , System .nanoTime (), isMarker , position );
305
289
if (brokerInterceptor != null ) {
306
290
brokerInterceptor
307
291
.onMessagePublish (this , headersAndPayload , messagePublishContext );
@@ -313,7 +297,7 @@ private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenc
313
297
long batchSize , boolean isChunked , boolean isMarker , Position position ) {
314
298
MessagePublishContext messagePublishContext = MessagePublishContext .get (this , lowestSequenceId ,
315
299
highestSequenceId , msgIn , headersAndPayload .readableBytes (), batchSize ,
316
- isChunked , System .nanoTime (), isMarker , position , isSupportsReplDedupByLidAndEid () );
300
+ isChunked , System .nanoTime (), isMarker , position );
317
301
if (brokerInterceptor != null ) {
318
302
brokerInterceptor
319
303
.onMessagePublish (this , headersAndPayload , messagePublishContext );
@@ -409,7 +393,6 @@ private static final class MessagePublishContext implements PublishContext, Runn
409
393
private long batchSize ;
410
394
private boolean chunked ;
411
395
private boolean isMarker ;
412
- private boolean supportsReplDedupByLidAndEid ;
413
396
414
397
private long startTimeNs ;
415
398
@@ -500,11 +483,6 @@ public long getOriginalSequenceId() {
500
483
return originalSequenceId ;
501
484
}
502
485
503
- @ Override
504
- public boolean supportsReplDedupByLidAndEid () {
505
- return supportsReplDedupByLidAndEid ;
506
- }
507
-
508
486
@ Override
509
487
public void setOriginalHighestSequenceId (long originalHighestSequenceId ) {
510
488
this .originalHighestSequenceId = originalHighestSequenceId ;
@@ -572,12 +550,8 @@ public void run() {
572
550
// stats
573
551
rateIn .recordMultipleEvents (batchSize , msgSize );
574
552
producer .topic .recordAddLatency (System .nanoTime () - startTimeNs , TimeUnit .NANOSECONDS );
575
- if (producer .isRemoteOrShadow && producer .isSupportsReplDedupByLidAndEid ()) {
576
- sendSendReceiptResponseRepl ();
577
- } else {
578
- // Repl V1 is the same as normal for this handling.
579
- sendSendReceiptResponseNormal ();
580
- }
553
+ producer .cnx .getCommandSender ().sendSendReceiptResponse (producer .producerId , sequenceId , highestSequenceId ,
554
+ ledgerId , entryId );
581
555
producer .cnx .completedSendOperation (producer .isNonPersistentTopic , msgSize );
582
556
if (this .chunked ) {
583
557
producer .chunkedMessageRate .recordEvent ();
@@ -590,46 +564,8 @@ public void run() {
590
564
recycle ();
591
565
}
592
566
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
-
630
567
static MessagePublishContext get (Producer producer , long sequenceId , Rate rateIn , int msgSize ,
631
- long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ,
632
- boolean supportsReplDedupByLidAndEid ) {
568
+ long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ) {
633
569
MessagePublishContext callback = RECYCLER .get ();
634
570
callback .producer = producer ;
635
571
callback .sequenceId = sequenceId ;
@@ -641,7 +577,6 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
641
577
callback .originalSequenceId = -1L ;
642
578
callback .startTimeNs = startTimeNs ;
643
579
callback .isMarker = isMarker ;
644
- callback .supportsReplDedupByLidAndEid = supportsReplDedupByLidAndEid ;
645
580
callback .ledgerId = position == null ? -1 : position .getLedgerId ();
646
581
callback .entryId = position == null ? -1 : position .getEntryId ();
647
582
if (callback .propertyMap != null ) {
@@ -651,8 +586,7 @@ static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn
651
586
}
652
587
653
588
static MessagePublishContext get (Producer producer , long lowestSequenceId , long highestSequenceId , Rate rateIn ,
654
- int msgSize , long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ,
655
- boolean supportsReplDedupByLidAndEid ) {
589
+ int msgSize , long batchSize , boolean chunked , long startTimeNs , boolean isMarker , Position position ) {
656
590
MessagePublishContext callback = RECYCLER .get ();
657
591
callback .producer = producer ;
658
592
callback .sequenceId = lowestSequenceId ;
@@ -665,7 +599,6 @@ static MessagePublishContext get(Producer producer, long lowestSequenceId, long
665
599
callback .startTimeNs = startTimeNs ;
666
600
callback .chunked = chunked ;
667
601
callback .isMarker = isMarker ;
668
- callback .supportsReplDedupByLidAndEid = supportsReplDedupByLidAndEid ;
669
602
callback .ledgerId = position == null ? -1 : position .getLedgerId ();
670
603
callback .entryId = position == null ? -1 : position .getEntryId ();
671
604
if (callback .propertyMap != null ) {
@@ -896,8 +829,7 @@ public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, lon
896
829
}
897
830
MessagePublishContext messagePublishContext =
898
831
MessagePublishContext .get (this , sequenceId , highSequenceId , msgIn ,
899
- headersAndPayload .readableBytes (), batchSize , isChunked , System .nanoTime (), isMarker , null ,
900
- cnx .isClientSupportsReplDedupByLidAndEid ());
832
+ headersAndPayload .readableBytes (), batchSize , isChunked , System .nanoTime (), isMarker , null );
901
833
if (brokerInterceptor != null ) {
902
834
brokerInterceptor
903
835
.onMessagePublish (this , headersAndPayload , messagePublishContext );
0 commit comments