/
Producer.java
681 lines (591 loc) · 25.5 KB
/
Producer.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
/**
* 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.broker.service;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.scurrilous.circe.checksum.Crc32cIntChecksum.computeChecksum;
import static org.apache.pulsar.broker.service.AbstractReplicator.REPL_PRODUCER_NAME_DELIMITER;
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.base.MoreObjects;
import io.netty.buffer.ByteBuf;
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicClosedException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException;
import org.apache.pulsar.broker.service.Topic.PublishContext;
import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.transaction.TxnID;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.api.proto.ProducerAccessMode;
import org.apache.pulsar.common.api.proto.ServerError;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.stats.NonPersistentPublisherStatsImpl;
import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.DateFormatter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Represents a currently connected producer.
*/
public class Producer {
private final Topic topic;
private final TransportCnx cnx;
private final String producerName;
private final long epoch;
private final boolean userProvidedProducerName;
private final long producerId;
private final String appId;
private Rate msgIn;
private Rate chunkedMessageRate;
// it records msg-drop rate only for non-persistent topic
private final Rate msgDrop;
private volatile long pendingPublishAcks = 0;
private static final AtomicLongFieldUpdater<Producer> pendingPublishAcksUpdater = AtomicLongFieldUpdater
.newUpdater(Producer.class, "pendingPublishAcks");
private boolean isClosed = false;
private final CompletableFuture<Void> closeFuture;
private final PublisherStatsImpl stats;
private final boolean isRemote;
private final String remoteCluster;
private final boolean isNonPersistentTopic;
private final boolean isEncrypted;
private final ProducerAccessMode accessMode;
private Optional<Long> topicEpoch;
private final Map<String, String> metadata;
private final SchemaVersion schemaVersion;
private final String clientAddress; // IP address only, no port number included
private final AtomicBoolean isDisconnecting = new AtomicBoolean(false);
public Producer(Topic topic, TransportCnx cnx, long producerId, String producerName, String appId,
boolean isEncrypted, Map<String, String> metadata, SchemaVersion schemaVersion, long epoch,
boolean userProvidedProducerName,
ProducerAccessMode accessMode,
Optional<Long> topicEpoch) {
this.topic = topic;
this.cnx = cnx;
this.producerId = producerId;
this.producerName = checkNotNull(producerName);
this.userProvidedProducerName = userProvidedProducerName;
this.epoch = epoch;
this.closeFuture = new CompletableFuture<>();
this.appId = appId;
this.msgIn = new Rate();
this.chunkedMessageRate = new Rate();
this.isNonPersistentTopic = topic instanceof NonPersistentTopic;
this.msgDrop = this.isNonPersistentTopic ? new Rate() : null;
this.metadata = metadata != null ? metadata : Collections.emptyMap();
this.stats = isNonPersistentTopic ? new NonPersistentPublisherStatsImpl() : new PublisherStatsImpl();
if (cnx.hasHAProxyMessage()) {
stats.setAddress(cnx.getHAProxyMessage().sourceAddress() + ":" + cnx.getHAProxyMessage().sourcePort());
} else {
stats.setAddress(cnx.clientAddress().toString());
}
stats.setConnectedSince(DateFormatter.now());
stats.setClientVersion(cnx.getClientVersion());
stats.setProducerName(producerName);
stats.producerId = producerId;
stats.metadata = this.metadata;
stats.accessMode = Commands.convertProducerAccessMode(accessMode);
this.isRemote = producerName
.startsWith(cnx.getBrokerService().pulsar().getConfiguration().getReplicatorPrefix());
this.remoteCluster = isRemote ? producerName.split("\\.")[2].split(REPL_PRODUCER_NAME_DELIMITER)[0] : null;
this.isEncrypted = isEncrypted;
this.schemaVersion = schemaVersion;
this.accessMode = accessMode;
this.topicEpoch = topicEpoch;
this.clientAddress = cnx.clientSourceAddress();
}
@Override
public int hashCode() {
return Objects.hash(producerName);
}
@Override
public boolean equals(Object obj) {
if (obj instanceof Producer) {
Producer other = (Producer) obj;
return Objects.equals(producerName, other.producerName) && Objects.equals(topic, other.topic);
}
return false;
}
public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize,
boolean isChunked) {
if (checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize)) {
publishMessageToTopic(headersAndPayload, sequenceId, batchSize, isChunked);
}
}
public void publishMessage(long producerId, long lowestSequenceId, long highestSequenceId,
ByteBuf headersAndPayload, long batchSize, boolean isChunked) {
if (lowestSequenceId > highestSequenceId) {
cnx.execute(() -> {
cnx.getCommandSender().sendSendError(producerId, highestSequenceId, ServerError.MetadataError,
"Invalid lowest or highest sequence id");
cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes());
});
return;
}
if (checkAndStartPublish(producerId, highestSequenceId, headersAndPayload, batchSize)) {
publishMessageToTopic(headersAndPayload, lowestSequenceId, highestSequenceId, batchSize, isChunked);
}
}
public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize) {
if (isClosed) {
cnx.execute(() -> {
cnx.getCommandSender().sendSendError(producerId, sequenceId, ServerError.PersistenceError,
"Producer is closed");
cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes());
});
return false;
}
if (!verifyChecksum(headersAndPayload)) {
cnx.execute(() -> {
cnx.getCommandSender().sendSendError(producerId, sequenceId, ServerError.ChecksumError,
"Checksum failed on the broker");
cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes());
});
return false;
}
if (topic.isEncryptionRequired()) {
headersAndPayload.markReaderIndex();
MessageMetadata msgMetadata = Commands.parseMessageMetadata(headersAndPayload);
headersAndPayload.resetReaderIndex();
int encryptionKeysCount = msgMetadata.getEncryptionKeysCount();
// Check whether the message is encrypted or not
if (encryptionKeysCount < 1) {
log.warn("[{}] Messages must be encrypted", getTopic().getName());
cnx.execute(() -> {
cnx.getCommandSender().sendSendError(producerId, sequenceId, ServerError.MetadataError,
"Messages must be encrypted");
cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes());
});
return false;
}
}
startPublishOperation((int) batchSize, headersAndPayload.readableBytes());
return true;
}
private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked) {
topic.publishMessage(headersAndPayload,
MessagePublishContext.get(this, sequenceId, msgIn,
headersAndPayload.readableBytes(), batchSize,
isChunked, System.nanoTime()));
}
private void publishMessageToTopic(ByteBuf headersAndPayload, long lowestSequenceId, long highestSequenceId,
long batchSize, boolean isChunked) {
topic.publishMessage(headersAndPayload,
MessagePublishContext.get(this, lowestSequenceId,
highestSequenceId, msgIn, headersAndPayload.readableBytes(), batchSize,
isChunked, System.nanoTime()));
}
private boolean verifyChecksum(ByteBuf headersAndPayload) {
if (hasChecksum(headersAndPayload)) {
int readerIndex = headersAndPayload.readerIndex();
try {
int checksum = readChecksum(headersAndPayload);
long computedChecksum = computeChecksum(headersAndPayload);
if (checksum == computedChecksum) {
return true;
} else {
log.error("[{}] [{}] Failed to verify checksum", topic, producerName);
return false;
}
} finally {
headersAndPayload.readerIndex(readerIndex);
}
} else {
// ignore if checksum is not available
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Payload does not have checksum to verify", topic, producerName);
}
return true;
}
}
private void startPublishOperation(int batchSize, long msgSize) {
// A single thread is incrementing/decrementing this counter, so we can use lazySet which doesn't involve a mem
// barrier
pendingPublishAcksUpdater.lazySet(this, pendingPublishAcks + 1);
// increment publish-count
this.getTopic().incrementPublishCount(batchSize, msgSize);
}
private void publishOperationCompleted() {
long newPendingPublishAcks = this.pendingPublishAcks - 1;
pendingPublishAcksUpdater.lazySet(this, newPendingPublishAcks);
// Check the close future to avoid grabbing the mutex every time the pending acks goes down to 0
if (newPendingPublishAcks == 0 && !closeFuture.isDone()) {
synchronized (this) {
if (isClosed && !closeFuture.isDone()) {
closeNow(true);
}
}
}
}
public void recordMessageDrop(int batchSize) {
if (this.isNonPersistentTopic) {
msgDrop.recordEvent(batchSize);
}
}
/**
* Return the sequence id of.
*
* @return the sequence id
*/
public long getLastSequenceId() {
if (isNonPersistentTopic) {
return -1;
} else {
return ((PersistentTopic) topic).getLastPublishedSequenceId(producerName);
}
}
public TransportCnx getCnx() {
return this.cnx;
}
private static final class MessagePublishContext implements PublishContext, Runnable {
private Producer producer;
private long sequenceId;
private long ledgerId;
private long entryId;
private Rate rateIn;
private int msgSize;
private long batchSize;
private boolean chunked;
private long startTimeNs;
private String originalProducerName;
private long originalSequenceId;
private long highestSequenceId;
private long originalHighestSequenceId;
public String getProducerName() {
return producer.getProducerName();
}
public long getSequenceId() {
return sequenceId;
}
public boolean isChunked() {
return chunked;
}
@Override
public long getHighestSequenceId() {
return highestSequenceId;
}
@Override
public void setOriginalProducerName(String originalProducerName) {
this.originalProducerName = originalProducerName;
}
@Override
public void setOriginalSequenceId(long originalSequenceId) {
this.originalSequenceId = originalSequenceId;
}
@Override
public String getOriginalProducerName() {
return originalProducerName;
}
@Override
public long getOriginalSequenceId() {
return originalSequenceId;
}
@Override
public void setOriginalHighestSequenceId(long originalHighestSequenceId) {
this.originalHighestSequenceId = originalHighestSequenceId;
}
@Override
public long getOriginalHighestSequenceId() {
return originalHighestSequenceId;
}
/**
* Executed from managed ledger thread when the message is persisted.
*/
@Override
public void completed(Exception exception, long ledgerId, long entryId) {
if (exception != null) {
final ServerError serverError = getServerError(exception);
producer.cnx.execute(() -> {
if (!(exception instanceof TopicClosedException)) {
// For TopicClosed exception there's no need to send explicit error, since the client was
// already notified
long callBackSequenceId = Math.max(highestSequenceId, sequenceId);
producer.cnx.getCommandSender().sendSendError(producer.producerId, callBackSequenceId,
serverError, exception.getMessage());
}
producer.cnx.completedSendOperation(producer.isNonPersistentTopic, msgSize);
producer.publishOperationCompleted();
recycle();
});
} else {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] [{}] triggered send callback. cnx {}, sequenceId {}", producer.topic,
producer.producerName, producer.producerId, producer.cnx.clientAddress(), sequenceId);
}
this.ledgerId = ledgerId;
this.entryId = entryId;
producer.cnx.execute(this);
}
}
private ServerError getServerError(Exception exception) {
ServerError serverError;
if (exception instanceof TopicTerminatedException) {
serverError = ServerError.TopicTerminatedError;
} else if (exception instanceof BrokerServiceException.NotAllowedException) {
serverError = ServerError.NotAllowedError;
} else {
serverError = ServerError.PersistenceError;
}
return serverError;
}
/**
* Executed from I/O thread when sending receipt back to client.
*/
@Override
public void run() {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] [{}] Persisted message. cnx {}, sequenceId {}", producer.topic,
producer.producerName, producer.producerId, producer.cnx, sequenceId);
}
// stats
rateIn.recordMultipleEvents(batchSize, msgSize);
producer.topic.recordAddLatency(System.nanoTime() - startTimeNs, TimeUnit.NANOSECONDS);
producer.cnx.getCommandSender().sendSendReceiptResponse(producer.producerId, sequenceId, highestSequenceId,
ledgerId, entryId);
producer.cnx.completedSendOperation(producer.isNonPersistentTopic, msgSize);
if (this.chunked) {
producer.chunkedMessageRate.recordEvent();
}
producer.publishOperationCompleted();
recycle();
}
static MessagePublishContext get(Producer producer, long sequenceId, Rate rateIn, int msgSize,
long batchSize, boolean chunked, long startTimeNs) {
MessagePublishContext callback = RECYCLER.get();
callback.producer = producer;
callback.sequenceId = sequenceId;
callback.rateIn = rateIn;
callback.msgSize = msgSize;
callback.batchSize = batchSize;
callback.chunked = chunked;
callback.originalProducerName = null;
callback.originalSequenceId = -1L;
callback.startTimeNs = startTimeNs;
return callback;
}
static MessagePublishContext get(Producer producer, long lowestSequenceId, long highestSequenceId, Rate rateIn,
int msgSize, long batchSize, boolean chunked, long startTimeNs) {
MessagePublishContext callback = RECYCLER.get();
callback.producer = producer;
callback.sequenceId = lowestSequenceId;
callback.highestSequenceId = highestSequenceId;
callback.rateIn = rateIn;
callback.msgSize = msgSize;
callback.batchSize = batchSize;
callback.originalProducerName = null;
callback.originalSequenceId = -1L;
callback.startTimeNs = startTimeNs;
callback.chunked = chunked;
return callback;
}
@Override
public long getNumberOfMessages() {
return batchSize;
}
private final Handle<MessagePublishContext> recyclerHandle;
private MessagePublishContext(Handle<MessagePublishContext> recyclerHandle) {
this.recyclerHandle = recyclerHandle;
}
private static final Recycler<MessagePublishContext> RECYCLER = new Recycler<MessagePublishContext>() {
protected MessagePublishContext newObject(Handle<MessagePublishContext> handle) {
return new MessagePublishContext(handle);
}
};
public void recycle() {
producer = null;
sequenceId = -1L;
highestSequenceId = -1L;
originalSequenceId = -1L;
originalHighestSequenceId = -1L;
rateIn = null;
msgSize = 0;
ledgerId = -1L;
entryId = -1L;
batchSize = 0L;
startTimeNs = -1L;
chunked = false;
recyclerHandle.recycle(this);
}
}
public Topic getTopic() {
return topic;
}
public String getProducerName() {
return producerName;
}
public long getProducerId() {
return producerId;
}
public Map<String, String> getMetadata() {
return metadata;
}
@Override
public String toString() {
return MoreObjects.toStringHelper(this).add("topic", topic).add("client", cnx.clientAddress())
.add("producerName", producerName).add("producerId", producerId).toString();
}
/**
* Close the producer immediately if: a. the connection is dropped b. it's a graceful close and no pending publish
* acks are left else wait for pending publish acks
*
* @return completable future indicate completion of close
*/
public synchronized CompletableFuture<Void> close(boolean removeFromTopic) {
if (log.isDebugEnabled()) {
log.debug("Closing producer {} -- isClosed={}", this, isClosed);
}
if (!isClosed) {
isClosed = true;
if (log.isDebugEnabled()) {
log.debug("Trying to close producer {} -- cnxIsActive: {} -- pendingPublishAcks: {}", this,
cnx.isActive(), pendingPublishAcks);
}
if (!cnx.isActive() || pendingPublishAcks == 0) {
closeNow(removeFromTopic);
}
}
return closeFuture;
}
public void closeNow(boolean removeFromTopic) {
if (removeFromTopic) {
topic.removeProducer(this);
}
cnx.removedProducer(this);
if (log.isDebugEnabled()) {
log.debug("Removed producer: {}", this);
}
closeFuture.complete(null);
isDisconnecting.set(false);
}
/**
* It closes the producer from server-side and sends command to client to disconnect producer from existing
* connection without closing that connection.
*
* @return Completable future indicating completion of producer close
*/
public CompletableFuture<Void> disconnect() {
if (!closeFuture.isDone() && isDisconnecting.compareAndSet(false, true)) {
log.info("Disconnecting producer: {}", this);
cnx.execute(() -> {
cnx.closeProducer(this);
closeNow(true);
});
}
return closeFuture;
}
public void updateRates() {
msgIn.calculateRate();
chunkedMessageRate.calculateRate();
stats.msgRateIn = msgIn.getRate();
stats.msgThroughputIn = msgIn.getValueRate();
stats.averageMsgSize = msgIn.getAverageValue();
stats.chunkedMessageRate = chunkedMessageRate.getRate();
if (chunkedMessageRate.getCount() > 0 && this.topic instanceof PersistentTopic) {
((PersistentTopic) this.topic).msgChunkPublished = true;
}
if (this.isNonPersistentTopic) {
msgDrop.calculateRate();
((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getRate();
}
}
public void updateRates(int numOfMessages, long msgSizeInBytes) {
msgIn.recordMultipleEvents(numOfMessages, msgSizeInBytes);
}
public boolean isRemote() {
return isRemote;
}
public String getRemoteCluster() {
return remoteCluster;
}
public PublisherStatsImpl getStats() {
return stats;
}
public boolean isNonPersistentTopic() {
return isNonPersistentTopic;
}
public long getEpoch() {
return epoch;
}
public boolean isUserProvidedProducerName() {
return userProvidedProducerName;
}
@VisibleForTesting
long getPendingPublishAcks() {
return pendingPublishAcks;
}
public void checkPermissions() {
TopicName topicName = TopicName.get(topic.getName());
if (cnx.getBrokerService().getAuthorizationService() != null) {
try {
if (cnx.getBrokerService().getAuthorizationService().canProduce(topicName, appId,
cnx.getAuthenticationData())) {
return;
}
} catch (Exception e) {
log.warn("[{}] Get unexpected error while autorizing [{}] {}", appId, topic.getName(), e.getMessage(),
e);
}
log.info("[{}] is not allowed to produce on topic [{}] anymore", appId, topic.getName());
disconnect();
}
}
public void checkEncryption() {
if (topic.isEncryptionRequired() && !isEncrypted) {
log.info("[{}] [{}] Unencrypted producer is not allowed to produce on topic [{}] anymore",
producerId, producerName, topic.getName());
disconnect();
}
}
public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, long highSequenceId,
ByteBuf headersAndPayload, long batchSize, boolean isChunked) {
checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize);
topic.publishTxnMessage(txnID, headersAndPayload,
MessagePublishContext.get(this, sequenceId, highSequenceId, msgIn,
headersAndPayload.readableBytes(), batchSize, isChunked, System.nanoTime()));
}
public SchemaVersion getSchemaVersion() {
return schemaVersion;
}
public ProducerAccessMode getAccessMode() {
return accessMode;
}
public Optional<Long> getTopicEpoch() {
return topicEpoch;
}
public String getClientAddress() {
return clientAddress;
}
public boolean isDisconnecting() {
return isDisconnecting.get();
}
private static final Logger log = LoggerFactory.getLogger(Producer.class);
}