forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
/
ProducerImpl.java
1880 lines (1690 loc) · 81.7 KB
/
ProducerImpl.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.client.impl;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.scurrilous.circe.checksum.Crc32cIntChecksum.computeChecksum;
import static com.scurrilous.circe.checksum.Crc32cIntChecksum.resumeChecksum;
import static java.lang.String.format;
import static org.apache.pulsar.client.impl.MessageImpl.SchemaState.Broken;
import static org.apache.pulsar.client.impl.MessageImpl.SchemaState.None;
import static org.apache.pulsar.client.impl.ProducerBase.MultiSchemaMode.Auto;
import static org.apache.pulsar.client.impl.ProducerBase.MultiSchemaMode.Enabled;
import static org.apache.pulsar.common.protocol.Commands.hasChecksum;
import static org.apache.pulsar.common.protocol.Commands.readChecksum;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Queues;
import io.netty.buffer.ByteBuf;
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;
import io.netty.util.ReferenceCountUtil;
import io.netty.util.Timeout;
import io.netty.util.TimerTask;
import io.netty.util.concurrent.ScheduledFuture;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Queue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.client.api.BatcherBuilder;
import org.apache.pulsar.client.api.CompressionType;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageCrypto;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerAccessMode;
import org.apache.pulsar.client.api.ProducerCryptoFailureAction;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.PulsarClientException.CryptoException;
import org.apache.pulsar.client.api.PulsarClientException.TimeoutException;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.transaction.Transaction;
import org.apache.pulsar.client.impl.conf.ProducerConfigurationData;
import org.apache.pulsar.client.impl.crypto.MessageCryptoBc;
import org.apache.pulsar.client.impl.schema.JSONSchema;
import org.apache.pulsar.client.impl.transaction.TransactionImpl;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.api.proto.ProtocolVersion;
import org.apache.pulsar.common.compression.CompressionCodec;
import org.apache.pulsar.common.compression.CompressionCodecProvider;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.protocol.ByteBufPair;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.Commands.ChecksumType;
import org.apache.pulsar.common.protocol.schema.SchemaHash;
import org.apache.pulsar.common.schema.SchemaInfo;
import org.apache.pulsar.common.schema.SchemaType;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.common.util.FutureUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ProducerImpl<T> extends ProducerBase<T> implements TimerTask, ConnectionHandler.Connection {
// Producer id, used to identify a producer within a single connection
protected final long producerId;
// Variable is used through the atomic updater
private volatile long msgIdGenerator;
private final Queue<OpSendMsg> pendingMessages;
private final Optional<Semaphore> semaphore;
private volatile Timeout sendTimeout = null;
private long createProducerTimeout;
private final BatchMessageContainerBase batchMessageContainer;
private CompletableFuture<MessageId> lastSendFuture = CompletableFuture.completedFuture(null);
// Globally unique producer name
private String producerName;
private boolean userProvidedProducerName = false;
private String connectionId;
private String connectedSince;
private final int partitionIndex;
private final ProducerStatsRecorder stats;
private final CompressionCodec compressor;
static final AtomicLongFieldUpdater<ProducerImpl> LAST_SEQ_ID_PUBLISHED_UPDATER = AtomicLongFieldUpdater
.newUpdater(ProducerImpl.class, "lastSequenceIdPublished");
private volatile long lastSequenceIdPublished;
static final AtomicLongFieldUpdater<ProducerImpl> LAST_SEQ_ID_PUSHED_UPDATER = AtomicLongFieldUpdater
.newUpdater(ProducerImpl.class, "lastSequenceIdPushed");
protected volatile long lastSequenceIdPushed;
private volatile boolean isLastSequenceIdPotentialDuplicated;
private final MessageCrypto msgCrypto;
private ScheduledFuture<?> keyGeneratorTask = null;
private final Map<String, String> metadata;
private Optional<byte[]> schemaVersion = Optional.empty();
private final ConnectionHandler connectionHandler;
private ScheduledFuture<?> batchTimerTask;
private Optional<Long> topicEpoch = Optional.empty();
@SuppressWarnings("rawtypes")
private static final AtomicLongFieldUpdater<ProducerImpl> msgIdGeneratorUpdater = AtomicLongFieldUpdater
.newUpdater(ProducerImpl.class, "msgIdGenerator");
public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfigurationData conf,
CompletableFuture<Producer<T>> producerCreatedFuture, int partitionIndex, Schema<T> schema,
ProducerInterceptors interceptors) {
super(client, topic, conf, producerCreatedFuture, schema, interceptors);
this.producerId = client.newProducerId();
this.producerName = conf.getProducerName();
if (StringUtils.isNotBlank(producerName)) {
this.userProvidedProducerName = true;
}
this.partitionIndex = partitionIndex;
this.pendingMessages = createPendingMessagesQueue();
if (conf.getMaxPendingMessages() > 0) {
this.semaphore = Optional.of(new Semaphore(conf.getMaxPendingMessages(), true));
} else {
this.semaphore = Optional.empty();
}
this.compressor = CompressionCodecProvider.getCompressionCodec(conf.getCompressionType());
if (conf.getInitialSequenceId() != null) {
long initialSequenceId = conf.getInitialSequenceId();
this.lastSequenceIdPublished = initialSequenceId;
this.lastSequenceIdPushed = initialSequenceId;
this.msgIdGenerator = initialSequenceId + 1L;
} else {
this.lastSequenceIdPublished = -1L;
this.lastSequenceIdPushed = -1L;
this.msgIdGenerator = 0L;
}
if (conf.isEncryptionEnabled()) {
String logCtx = "[" + topic + "] [" + producerName + "] [" + producerId + "]";
if (conf.getMessageCrypto() != null) {
this.msgCrypto = conf.getMessageCrypto();
} else {
// default to use MessageCryptoBc;
MessageCrypto msgCryptoBc;
try {
msgCryptoBc = new MessageCryptoBc(logCtx, true);
} catch (Exception e) {
log.error("MessageCryptoBc may not included in the jar in Producer. e:", e);
msgCryptoBc = null;
}
this.msgCrypto = msgCryptoBc;
}
} else {
this.msgCrypto = null;
}
if (this.msgCrypto != null) {
// Regenerate data key cipher at fixed interval
keyGeneratorTask = client.eventLoopGroup().scheduleWithFixedDelay(() -> {
try {
msgCrypto.addPublicKeyCipher(conf.getEncryptionKeys(), conf.getCryptoKeyReader());
} catch (CryptoException e) {
if (!producerCreatedFuture.isDone()) {
log.warn("[{}] [{}] [{}] Failed to add public key cipher.", topic, producerName, producerId);
producerCreatedFuture.completeExceptionally(
PulsarClientException.wrap(e,
String.format("The producer %s of the topic %s " +
"adds the public key cipher was failed",
producerName, topic)));
}
}
}, 0L, 4L, TimeUnit.HOURS);
}
if (conf.getSendTimeoutMs() > 0) {
sendTimeout = client.timer().newTimeout(this, conf.getSendTimeoutMs(), TimeUnit.MILLISECONDS);
}
this.createProducerTimeout = System.currentTimeMillis() + client.getConfiguration().getOperationTimeoutMs();
if (conf.isBatchingEnabled()) {
BatcherBuilder containerBuilder = conf.getBatcherBuilder();
if (containerBuilder == null) {
containerBuilder = BatcherBuilder.DEFAULT;
}
this.batchMessageContainer = (BatchMessageContainerBase)containerBuilder.build();
this.batchMessageContainer.setProducer(this);
} else {
this.batchMessageContainer = null;
}
if (client.getConfiguration().getStatsIntervalSeconds() > 0) {
stats = new ProducerStatsRecorderImpl(client, conf, this);
} else {
stats = ProducerStatsDisabled.INSTANCE;
}
if (conf.getProperties().isEmpty()) {
metadata = Collections.emptyMap();
} else {
metadata = Collections.unmodifiableMap(new HashMap<>(conf.getProperties()));
}
this.connectionHandler = new ConnectionHandler(this,
new BackoffBuilder()
.setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS)
.setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS)
.setMandatoryStop(Math.max(100, conf.getSendTimeoutMs() - 100), TimeUnit.MILLISECONDS)
.create(),
this);
grabCnx();
}
protected Queue<OpSendMsg> createPendingMessagesQueue() {
return new ArrayDeque<>();
}
public ConnectionHandler getConnectionHandler() {
return connectionHandler;
}
private boolean isBatchMessagingEnabled() {
return conf.isBatchingEnabled();
}
private boolean isMultiSchemaEnabled(boolean autoEnable) {
if (multiSchemaMode != Auto) {
return multiSchemaMode == Enabled;
}
if (autoEnable) {
multiSchemaMode = Enabled;
return true;
}
return false;
}
@Override
public long getLastSequenceId() {
return lastSequenceIdPublished;
}
@Override
CompletableFuture<MessageId> internalSendAsync(Message<?> message) {
CompletableFuture<MessageId> future = new CompletableFuture<>();
MessageImpl<?> interceptorMessage = (MessageImpl) beforeSend(message);
//Retain the buffer used by interceptors callback to get message. Buffer will release after complete interceptors.
interceptorMessage.getDataBuffer().retain();
if (interceptors != null) {
interceptorMessage.getProperties();
}
sendAsync(interceptorMessage, new SendCallback() {
SendCallback nextCallback = null;
MessageImpl<?> nextMsg = null;
long createdAt = System.nanoTime();
@Override
public CompletableFuture<MessageId> getFuture() {
return future;
}
@Override
public SendCallback getNextSendCallback() {
return nextCallback;
}
@Override
public MessageImpl<?> getNextMessage() {
return nextMsg;
}
@Override
public void sendComplete(Exception e) {
try {
if (e != null) {
stats.incrementSendFailed();
onSendAcknowledgement(interceptorMessage, null, e);
future.completeExceptionally(e);
} else {
onSendAcknowledgement(interceptorMessage, interceptorMessage.getMessageId(), null);
future.complete(interceptorMessage.getMessageId());
stats.incrementNumAcksReceived(System.nanoTime() - createdAt);
}
} finally {
interceptorMessage.getDataBuffer().release();
}
while (nextCallback != null) {
SendCallback sendCallback = nextCallback;
MessageImpl<?> msg = nextMsg;
//Retain the buffer used by interceptors callback to get message. Buffer will release after complete interceptors.
try {
msg.getDataBuffer().retain();
if (e != null) {
stats.incrementSendFailed();
onSendAcknowledgement(msg, null, e);
sendCallback.getFuture().completeExceptionally(e);
} else {
onSendAcknowledgement(msg, msg.getMessageId(), null);
sendCallback.getFuture().complete(msg.getMessageId());
stats.incrementNumAcksReceived(System.nanoTime() - createdAt);
}
nextMsg = nextCallback.getNextMessage();
nextCallback = nextCallback.getNextSendCallback();
} finally {
msg.getDataBuffer().release();
}
}
}
@Override
public void addCallback(MessageImpl<?> msg, SendCallback scb) {
nextMsg = msg;
nextCallback = scb;
}
});
return future;
}
@Override
CompletableFuture<MessageId> internalSendWithTxnAsync(Message<?> message, Transaction txn) {
if (txn == null) {
return internalSendAsync(message);
} else {
return ((TransactionImpl) txn).registerProducedTopic(topic)
.thenCompose(ignored -> internalSendAsync(message));
}
}
/**
* Compress the payload if compression is configured
* @param payload
* @return a new payload
*/
private ByteBuf applyCompression(ByteBuf payload) {
ByteBuf compressedPayload = compressor.encode(payload);
payload.release();
return compressedPayload;
}
public void sendAsync(Message<?> message, SendCallback callback) {
checkArgument(message instanceof MessageImpl);
if (!isValidProducerState(callback, message.getSequenceId())) {
return;
}
MessageImpl<?> msg = (MessageImpl<?>) message;
MessageMetadata msgMetadata = msg.getMessageBuilder();
ByteBuf payload = msg.getDataBuffer();
int uncompressedSize = payload.readableBytes();
if (!canEnqueueRequest(callback, message.getSequenceId(), uncompressedSize)) {
return;
}
// If compression is enabled, we are compressing, otherwise it will simply use the same buffer
ByteBuf compressedPayload = payload;
boolean compressed = false;
// Batch will be compressed when closed
// If a message has a delayed delivery time, we'll always send it individually
if (!isBatchMessagingEnabled() || msgMetadata.hasDeliverAtTime()) {
compressedPayload = applyCompression(payload);
compressed = true;
// validate msg-size (For batching this will be check at the batch completion size)
int compressedSize = compressedPayload.readableBytes();
if (compressedSize > ClientCnx.getMaxMessageSize() && !this.conf.isChunkingEnabled()) {
compressedPayload.release();
String compressedStr = (!isBatchMessagingEnabled() && conf.getCompressionType() != CompressionType.NONE)
? "Compressed"
: "";
PulsarClientException.InvalidMessageException invalidMessageException = new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s sends a %s message with %d bytes that exceeds %d bytes",
producerName, topic, compressedStr, compressedSize, ClientCnx.getMaxMessageSize()));
completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException);
return;
}
}
if (!msg.isReplicated() && msgMetadata.hasProducerName()) {
PulsarClientException.InvalidMessageException invalidMessageException =
new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s can not reuse the same message", producerName, topic), msg.getSequenceId());
completeCallbackAndReleaseSemaphore(uncompressedSize, callback, invalidMessageException);
compressedPayload.release();
return;
}
if (!populateMessageSchema(msg, callback)) {
compressedPayload.release();
return;
}
// send in chunks
int totalChunks = canAddToBatch(msg) ? 1
: Math.max(1, compressedPayload.readableBytes()) / ClientCnx.getMaxMessageSize()
+ (Math.max(1, compressedPayload.readableBytes()) % ClientCnx.getMaxMessageSize() == 0 ? 0 : 1);
// chunked message also sent individually so, try to acquire send-permits
for (int i = 0; i < (totalChunks - 1); i++) {
if (!canEnqueueRequest(callback, message.getSequenceId(), 0 /* The memory was already reserved */)) {
return;
}
}
try {
synchronized (this) {
int readStartIndex = 0;
long sequenceId;
if (!msgMetadata.hasSequenceId()) {
sequenceId = msgIdGeneratorUpdater.getAndIncrement(this);
msgMetadata.setSequenceId(sequenceId);
} else {
sequenceId = msgMetadata.getSequenceId();
}
String uuid = totalChunks > 1 ? String.format("%s-%d", producerName, sequenceId) : null;
for (int chunkId = 0; chunkId < totalChunks; chunkId++) {
serializeAndSendMessage(msg, payload, sequenceId, uuid, chunkId, totalChunks,
readStartIndex, ClientCnx.getMaxMessageSize(), compressedPayload, compressed,
compressedPayload.readableBytes(), uncompressedSize, callback);
readStartIndex = ((chunkId + 1) * ClientCnx.getMaxMessageSize());
}
}
} catch (PulsarClientException e) {
e.setSequenceId(msg.getSequenceId());
completeCallbackAndReleaseSemaphore(uncompressedSize, callback, e);
} catch (Throwable t) {
completeCallbackAndReleaseSemaphore(uncompressedSize, callback, new PulsarClientException(t, msg.getSequenceId()));
}
}
private void serializeAndSendMessage(MessageImpl<?> msg, ByteBuf payload,
long sequenceId, String uuid, int chunkId, int totalChunks, int readStartIndex, int chunkMaxSizeInBytes, ByteBuf compressedPayload,
boolean compressed, int compressedPayloadSize,
int uncompressedSize, SendCallback callback) throws IOException, InterruptedException {
ByteBuf chunkPayload = compressedPayload;
MessageMetadata msgMetadata = msg.getMessageBuilder();
if (totalChunks > 1 && TopicName.get(topic).isPersistent()) {
chunkPayload = compressedPayload.slice(readStartIndex,
Math.min(chunkMaxSizeInBytes, chunkPayload.readableBytes() - readStartIndex));
// don't retain last chunk payload and builder as it will be not needed for next chunk-iteration and it will
// be released once this chunk-message is sent
if (chunkId != totalChunks - 1) {
chunkPayload.retain();
}
if (uuid != null) {
msgMetadata.setUuid(uuid);
}
msgMetadata.setChunkId(chunkId)
.setNumChunksFromMsg(totalChunks)
.setTotalChunkMsgSize(compressedPayloadSize);
}
if (!msgMetadata.hasPublishTime()) {
msgMetadata.setPublishTime(client.getClientClock().millis());
checkArgument(!msgMetadata.hasProducerName());
msgMetadata.setProducerName(producerName);
if (conf.getCompressionType() != CompressionType.NONE) {
msgMetadata
.setCompression(CompressionCodecProvider.convertToWireProtocol(conf.getCompressionType()));
}
msgMetadata.setUncompressedSize(uncompressedSize);
}
if (canAddToBatch(msg) && totalChunks <= 1) {
if (canAddToCurrentBatch(msg)) {
// should trigger complete the batch message, new message will add to a new batch and new batch
// sequence id use the new message, so that broker can handle the message duplication
if (sequenceId <= lastSequenceIdPushed) {
isLastSequenceIdPotentialDuplicated = true;
if (sequenceId <= lastSequenceIdPublished) {
log.warn("Message with sequence id {} is definitely a duplicate", sequenceId);
} else {
log.info("Message with sequence id {} might be a duplicate but cannot be determined at this time.",
sequenceId);
}
doBatchSendAndAdd(msg, callback, payload);
} else {
// Should flush the last potential duplicated since can't combine potential duplicated messages
// and non-duplicated messages into a batch.
if (isLastSequenceIdPotentialDuplicated) {
doBatchSendAndAdd(msg, callback, payload);
} else {
// handle boundary cases where message being added would exceed
// batch size and/or max message size
boolean isBatchFull = batchMessageContainer.add(msg, callback);
lastSendFuture = callback.getFuture();
payload.release();
if (isBatchFull) {
batchMessageAndSend();
}
}
isLastSequenceIdPotentialDuplicated = false;
}
} else {
doBatchSendAndAdd(msg, callback, payload);
}
} else {
// in this case compression has not been applied by the caller
// but we have to compress the payload if compression is configured
if (!compressed) {
chunkPayload = applyCompression(chunkPayload);
}
ByteBuf encryptedPayload = encryptMessage(msgMetadata, chunkPayload);
// When publishing during replication, we need to set the correct number of message in batch
// This is only used in tracking the publish rate stats
int numMessages = msg.getMessageBuilder().hasNumMessagesInBatch()
? msg.getMessageBuilder().getNumMessagesInBatch()
: 1;
final OpSendMsg op;
if (msg.getSchemaState() == MessageImpl.SchemaState.Ready) {
ByteBufPair cmd = sendMessage(producerId, sequenceId, numMessages, msgMetadata, encryptedPayload);
op = OpSendMsg.create(msg, cmd, sequenceId, callback);
} else {
op = OpSendMsg.create(msg, null, sequenceId, callback);
final MessageMetadata finalMsgMetadata = msgMetadata;
op.rePopulate = () -> {
op.cmd = sendMessage(producerId, sequenceId, numMessages, finalMsgMetadata, encryptedPayload);
};
}
op.setNumMessagesInBatch(numMessages);
op.setBatchSizeByte(encryptedPayload.readableBytes());
if (totalChunks > 1) {
op.totalChunks = totalChunks;
op.chunkId = chunkId;
}
lastSendFuture = callback.getFuture();
processOpSendMsg(op);
}
}
private boolean populateMessageSchema(MessageImpl msg, SendCallback callback) {
MessageMetadata msgMetadataBuilder = msg.getMessageBuilder();
if (msg.getSchemaInternal() == schema) {
schemaVersion.ifPresent(v -> msgMetadataBuilder.setSchemaVersion(v));
msg.setSchemaState(MessageImpl.SchemaState.Ready);
return true;
}
if (!isMultiSchemaEnabled(true)) {
PulsarClientException.InvalidMessageException e = new PulsarClientException.InvalidMessageException(
format("The producer %s of the topic %s is disabled the `MultiSchema`", producerName, topic)
, msg.getSequenceId());
completeCallbackAndReleaseSemaphore(msg.getUncompressedSize(), callback, e);
return false;
}
SchemaHash schemaHash = SchemaHash.of(msg.getSchemaInternal());
byte[] schemaVersion = schemaCache.get(schemaHash);
if (schemaVersion != null) {
msgMetadataBuilder.setSchemaVersion(schemaVersion);
msg.setSchemaState(MessageImpl.SchemaState.Ready);
}
return true;
}
private boolean rePopulateMessageSchema(MessageImpl msg) {
SchemaHash schemaHash = SchemaHash.of(msg.getSchemaInternal());
byte[] schemaVersion = schemaCache.get(schemaHash);
if (schemaVersion == null) {
return false;
}
msg.getMessageBuilder().setSchemaVersion(schemaVersion);
msg.setSchemaState(MessageImpl.SchemaState.Ready);
return true;
}
private void tryRegisterSchema(ClientCnx cnx, MessageImpl msg, SendCallback callback) {
if (!changeToRegisteringSchemaState()) {
return;
}
SchemaInfo schemaInfo = msg.hasReplicateFrom() ? msg.getSchemaInfoForReplicator() : msg.getSchemaInfo();
schemaInfo = Optional.ofNullable(schemaInfo)
.filter(si -> si.getType().getValue() > 0)
.orElse(Schema.BYTES.getSchemaInfo());
getOrCreateSchemaAsync(cnx, schemaInfo).handle((v, ex) -> {
if (ex != null) {
Throwable t = FutureUtil.unwrapCompletionException(ex);
log.warn("[{}] [{}] GetOrCreateSchema error", topic, producerName, t);
if (t instanceof PulsarClientException.IncompatibleSchemaException) {
msg.setSchemaState(MessageImpl.SchemaState.Broken);
callback.sendComplete((PulsarClientException.IncompatibleSchemaException) t);
}
} else {
log.warn("[{}] [{}] GetOrCreateSchema succeed", topic, producerName);
SchemaHash schemaHash = SchemaHash.of(msg.getSchemaInternal());
schemaCache.putIfAbsent(schemaHash, v);
msg.getMessageBuilder().setSchemaVersion(v);
msg.setSchemaState(MessageImpl.SchemaState.Ready);
}
cnx.ctx().channel().eventLoop().execute(() -> {
synchronized (ProducerImpl.this) {
recoverProcessOpSendMsgFrom(cnx, msg);
}
});
return null;
});
}
private CompletableFuture<byte[]> getOrCreateSchemaAsync(ClientCnx cnx, SchemaInfo schemaInfo) {
if (!Commands.peerSupportsGetOrCreateSchema(cnx.getRemoteEndpointProtocolVersion())) {
return FutureUtil.failedFuture(
new PulsarClientException.NotSupportedException(
format("The command `GetOrCreateSchema` is not supported for the protocol version %d. " +
"The producer is %s, topic is %s", cnx.getRemoteEndpointProtocolVersion(), producerName, topic)));
}
long requestId = client.newRequestId();
ByteBuf request = Commands.newGetOrCreateSchema(requestId, topic, schemaInfo);
log.info("[{}] [{}] GetOrCreateSchema request", topic, producerName);
return cnx.sendGetOrCreateSchema(request, requestId);
}
protected ByteBuf encryptMessage(MessageMetadata msgMetadata, ByteBuf compressedPayload)
throws PulsarClientException {
if (!conf.isEncryptionEnabled() || msgCrypto == null) {
return compressedPayload;
}
try {
int maxSize = msgCrypto.getMaxOutputSize(compressedPayload.readableBytes());
ByteBuf encryptedPayload = PulsarByteBufAllocator.DEFAULT.buffer(maxSize);
ByteBuffer targetBuffer = encryptedPayload.nioBuffer(0, maxSize);
msgCrypto.encrypt(conf.getEncryptionKeys(), conf.getCryptoKeyReader(), () -> msgMetadata,
compressedPayload.nioBuffer(), targetBuffer);
encryptedPayload.writerIndex(targetBuffer.remaining());
compressedPayload.release();
return encryptedPayload;
} catch (PulsarClientException e) {
// Unless config is set to explicitly publish un-encrypted message upon failure, fail the request
if (conf.getCryptoFailureAction() == ProducerCryptoFailureAction.SEND) {
log.warn("[{}] [{}] Failed to encrypt message {}. Proceeding with publishing unencrypted message",
topic, producerName, e.getMessage());
return compressedPayload;
}
throw e;
}
}
protected ByteBufPair sendMessage(long producerId, long sequenceId, int numMessages, MessageMetadata msgMetadata,
ByteBuf compressedPayload) {
return Commands.newSend(producerId, sequenceId, numMessages, getChecksumType(), msgMetadata, compressedPayload);
}
protected ByteBufPair sendMessage(long producerId, long lowestSequenceId, long highestSequenceId, int numMessages, MessageMetadata msgMetadata,
ByteBuf compressedPayload) {
return Commands.newSend(producerId, lowestSequenceId, highestSequenceId, numMessages, getChecksumType(), msgMetadata, compressedPayload);
}
protected ChecksumType getChecksumType() {
if (connectionHandler.cnx() == null
|| connectionHandler.cnx().getRemoteEndpointProtocolVersion() >= brokerChecksumSupportedVersion()) {
return ChecksumType.Crc32c;
} else {
return ChecksumType.None;
}
}
private boolean canAddToBatch(MessageImpl<?> msg) {
return msg.getSchemaState() == MessageImpl.SchemaState.Ready
&& isBatchMessagingEnabled() && !msg.getMessageBuilder().hasDeliverAtTime();
}
private boolean canAddToCurrentBatch(MessageImpl<?> msg) {
return batchMessageContainer.haveEnoughSpace(msg)
&& (!isMultiSchemaEnabled(false) || batchMessageContainer.hasSameSchema(msg))
&& batchMessageContainer.hasSameTxn(msg);
}
private void doBatchSendAndAdd(MessageImpl<?> msg, SendCallback callback, ByteBuf payload) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Closing out batch to accommodate large message with size {}", topic, producerName,
msg.getUncompressedSize());
}
try {
batchMessageAndSend();
batchMessageContainer.add(msg, callback);
lastSendFuture = callback.getFuture();
} finally {
payload.release();
}
}
private boolean isValidProducerState(SendCallback callback, long sequenceId) {
switch (getState()) {
case Ready:
// OK
case Connecting:
// We are OK to queue the messages on the client, it will be sent to the broker once we get the connection
case RegisteringSchema:
// registering schema
return true;
case Closing:
case Closed:
callback.sendComplete(new PulsarClientException.AlreadyClosedException("Producer already closed", sequenceId));
return false;
case ProducerFenced:
callback.sendComplete(new PulsarClientException.ProducerFencedException("Producer was fenced"));
return false;
case Terminated:
callback.sendComplete(new PulsarClientException.TopicTerminatedException("Topic was terminated", sequenceId));
return false;
case Failed:
case Uninitialized:
default:
callback.sendComplete(new PulsarClientException.NotConnectedException(sequenceId));
return false;
}
}
private boolean canEnqueueRequest(SendCallback callback, long sequenceId, int payloadSize) {
try {
if (conf.isBlockIfQueueFull()) {
if (semaphore.isPresent()) {
semaphore.get().acquire();
}
client.getMemoryLimitController().reserveMemory(payloadSize);
} else {
if (!semaphore.map(Semaphore::tryAcquire).orElse(true)) {
callback.sendComplete(new PulsarClientException.ProducerQueueIsFullError("Producer send queue is full", sequenceId));
return false;
}
if (!client.getMemoryLimitController().tryReserveMemory(payloadSize)) {
semaphore.ifPresent(Semaphore::release);
callback.sendComplete(new PulsarClientException.MemoryBufferIsFullError("Client memory buffer is full", sequenceId));
return false;
}
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
callback.sendComplete(new PulsarClientException(e, sequenceId));
return false;
}
return true;
}
private static final class WriteInEventLoopCallback implements Runnable {
private ProducerImpl<?> producer;
private ByteBufPair cmd;
private long sequenceId;
private ClientCnx cnx;
private OpSendMsg op;
static WriteInEventLoopCallback create(ProducerImpl<?> producer, ClientCnx cnx, OpSendMsg op) {
WriteInEventLoopCallback c = RECYCLER.get();
c.producer = producer;
c.cnx = cnx;
c.sequenceId = op.sequenceId;
c.cmd = op.cmd;
c.op = op;
return c;
}
@Override
public void run() {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Sending message cnx {}, sequenceId {}", producer.topic, producer.producerName, cnx,
sequenceId);
}
try {
cnx.ctx().writeAndFlush(cmd, cnx.ctx().voidPromise());
op.updateSentTimestamp();
} finally {
recycle();
}
}
private void recycle() {
producer = null;
cnx = null;
cmd = null;
sequenceId = -1;
op = null;
recyclerHandle.recycle(this);
}
private final Handle<WriteInEventLoopCallback> recyclerHandle;
private WriteInEventLoopCallback(Handle<WriteInEventLoopCallback> recyclerHandle) {
this.recyclerHandle = recyclerHandle;
}
private static final Recycler<WriteInEventLoopCallback> RECYCLER = new Recycler<WriteInEventLoopCallback>() {
@Override
protected WriteInEventLoopCallback newObject(Handle<WriteInEventLoopCallback> handle) {
return new WriteInEventLoopCallback(handle);
}
};
}
@Override
public CompletableFuture<Void> closeAsync() {
final State currentState = getAndUpdateState(state -> {
if (state == State.Closed) {
return state;
}
return State.Closing;
});
if (currentState == State.Closed || currentState == State.Closing) {
return CompletableFuture.completedFuture(null);
}
Timeout timeout = sendTimeout;
if (timeout != null) {
timeout.cancel();
sendTimeout = null;
}
ScheduledFuture<?> batchTimerTask = this.batchTimerTask;
if (batchTimerTask != null) {
batchTimerTask.cancel(false);
this.batchTimerTask = null;
}
if (keyGeneratorTask != null && !keyGeneratorTask.isCancelled()) {
keyGeneratorTask.cancel(false);
}
stats.cancelStatsTimeout();
ClientCnx cnx = cnx();
if (cnx == null || currentState != State.Ready) {
log.info("[{}] [{}] Closed Producer (not connected)", topic, producerName);
synchronized (this) {
setState(State.Closed);
client.cleanupProducer(this);
PulsarClientException ex = new PulsarClientException.AlreadyClosedException(
format("The producer %s of the topic %s was already closed when closing the producers",
producerName, topic));
pendingMessages.forEach(msg -> {
msg.sendComplete(ex);
msg.cmd.release();
msg.recycle();
});
pendingMessages.clear();
}
return CompletableFuture.completedFuture(null);
}
long requestId = client.newRequestId();
ByteBuf cmd = Commands.newCloseProducer(producerId, requestId);
CompletableFuture<Void> closeFuture = new CompletableFuture<>();
cnx.sendRequestWithId(cmd, requestId).handle((v, exception) -> {
cnx.removeProducer(producerId);
if (exception == null || !cnx.ctx().channel().isActive()) {
// Either we've received the success response for the close producer command from the broker, or the
// connection did break in the meantime. In any case, the producer is gone.
synchronized (ProducerImpl.this) {
log.info("[{}] [{}] Closed Producer", topic, producerName);
setState(State.Closed);
pendingMessages.forEach(msg -> {
msg.cmd.release();
msg.recycle();
});
pendingMessages.clear();
}
closeFuture.complete(null);
client.cleanupProducer(this);
} else {
closeFuture.completeExceptionally(exception);
}
return null;
});
return closeFuture;
}
@Override
public boolean isConnected() {
return connectionHandler.cnx() != null && (getState() == State.Ready);
}
@Override
public long getLastDisconnectedTimestamp() {
return connectionHandler.lastConnectionClosedTimestamp;
}
public boolean isWritable() {
ClientCnx cnx = connectionHandler.cnx();
return cnx != null && cnx.channel().isWritable();
}
public void terminated(ClientCnx cnx) {
State previousState = getAndUpdateState(state -> (state == State.Closed ? State.Closed : State.Terminated));
if (previousState != State.Terminated && previousState != State.Closed) {
log.info("[{}] [{}] The topic has been terminated", topic, producerName);
setClientCnx(null);
synchronized (this) {
failPendingMessages(cnx,
new PulsarClientException.TopicTerminatedException(
format("The topic %s that the producer %s produces to has been terminated", topic, producerName)));
}
}
}
void ackReceived(ClientCnx cnx, long sequenceId, long highestSequenceId, long ledgerId, long entryId) {
OpSendMsg op = null;
synchronized (this) {
op = pendingMessages.peek();
if (op == null) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Got ack for timed out msg {} - {}", topic, producerName, sequenceId, highestSequenceId);
}
return;
}
if (sequenceId > op.sequenceId) {
log.warn("[{}] [{}] Got ack for msg. expecting: {} - {} - got: {} - {} - queue-size: {}", topic, producerName,
op.sequenceId, op.highestSequenceId, sequenceId, highestSequenceId, pendingMessages.size());
// Force connection closing so that messages can be re-transmitted in a new connection
cnx.channel().close();
return;
} else if (sequenceId < op.sequenceId) {
// Ignoring the ack since it's referring to a message that has already timed out.
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Got ack for timed out msg. expecting: {} - {} - got: {} - {}", topic, producerName,
op.sequenceId, op.highestSequenceId, sequenceId, highestSequenceId);
}
return;
} else {
// Add check `sequenceId >= highestSequenceId` for backward compatibility.
if (sequenceId >= highestSequenceId || highestSequenceId == op.highestSequenceId) {
// Message was persisted correctly
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Received ack for msg {} ", topic, producerName, sequenceId);
}
pendingMessages.remove();
releaseSemaphoreForSendOp(op);
} else {
log.warn("[{}] [{}] Got ack for batch msg error. expecting: {} - {} - got: {} - {} - queue-size: {}", topic, producerName,
op.sequenceId, op.highestSequenceId, sequenceId, highestSequenceId, pendingMessages.size());
// Force connection closing so that messages can be re-transmitted in a new connection
cnx.channel().close();
return;
}
}
}
OpSendMsg finalOp = op;
LAST_SEQ_ID_PUBLISHED_UPDATER.getAndUpdate(this, last -> Math.max(last, getHighestSequenceId(finalOp)));
op.setMessageId(ledgerId, entryId, partitionIndex);
// if message is chunked then call callback only on last chunk
if (op.totalChunks <= 1 || (op.chunkId == op.totalChunks - 1)) {
try {
// Need to protect ourselves from any exception being thrown in the future handler from the
// application
op.sendComplete(null);
} catch (Throwable t) {
log.warn("[{}] [{}] Got exception while completing the callback for msg {}:", topic,