Skip to content

Commit ced67f1

Browse files
committed
GH-1441: Close producer after fatal send error
Resolves #1441 Close and remove a producer after a `OutOfOrderSequenceException` or subclass. **I expect conflicts so I will do the backports**
1 parent 5441fb2 commit ced67f1

File tree

3 files changed

+126
-50
lines changed

3 files changed

+126
-50
lines changed

spring-kafka/src/main/java/org/springframework/kafka/core/DefaultKafkaProducerFactory.java

Lines changed: 82 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import org.apache.kafka.common.MetricName;
4545
import org.apache.kafka.common.PartitionInfo;
4646
import org.apache.kafka.common.TopicPartition;
47+
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
4748
import org.apache.kafka.common.errors.ProducerFencedException;
4849
import org.apache.kafka.common.errors.TimeoutException;
4950
import org.apache.kafka.common.serialization.Serializer;
@@ -123,6 +124,8 @@ public class DefaultKafkaProducerFactory<K, V> implements ProducerFactory<K, V>,
123124

124125
private final Map<String, CloseSafeProducer<K, V>> consumerProducers = new HashMap<>();
125126

127+
private final ThreadLocal<CloseSafeProducer<K, V>> threadBoundProducers = new ThreadLocal<>();
128+
126129
private final AtomicInteger clientIdCounter = new AtomicInteger();
127130

128131
private Supplier<Serializer<K>> keySerializerSupplier;
@@ -139,8 +142,6 @@ public class DefaultKafkaProducerFactory<K, V> implements ProducerFactory<K, V>,
139142

140143
private boolean producerPerThread;
141144

142-
private ThreadLocal<CloseSafeProducer<K, V>> threadBoundProducers;
143-
144145
private String clientIdPrefix;
145146

146147
private volatile CloseSafeProducer<K, V> producer;
@@ -252,7 +253,6 @@ protected String getTransactionIdPrefix() {
252253
*/
253254
public void setProducerPerThread(boolean producerPerThread) {
254255
this.producerPerThread = producerPerThread;
255-
this.threadBoundProducers = new ThreadLocal<>();
256256
}
257257

258258
/**
@@ -305,8 +305,11 @@ public boolean transactionCapable() {
305305
@SuppressWarnings("resource")
306306
@Override
307307
public void destroy() {
308-
CloseSafeProducer<K, V> producerToClose = this.producer;
309-
this.producer = null;
308+
CloseSafeProducer<K, V> producerToClose;
309+
synchronized (this) {
310+
producerToClose = this.producer;
311+
this.producer = null;
312+
}
310313
if (producerToClose != null) {
311314
producerToClose.getDelegate().close(this.physicalCloseTimeout);
312315
}
@@ -380,19 +383,19 @@ public Producer<K, V> createProducer(@Nullable String txIdPrefixArg) {
380383
if (this.producerPerThread) {
381384
CloseSafeProducer<K, V> tlProducer = this.threadBoundProducers.get();
382385
if (tlProducer == null) {
383-
tlProducer = new CloseSafeProducer<>(createKafkaProducer());
386+
tlProducer = new CloseSafeProducer<>(createKafkaProducer(), this::removeProducer,
387+
this.physicalCloseTimeout);
384388
this.threadBoundProducers.set(tlProducer);
385389
}
386390
return tlProducer;
387391
}
388-
if (this.producer == null) {
389-
synchronized (this) {
390-
if (this.producer == null) {
391-
this.producer = new CloseSafeProducer<>(createKafkaProducer());
392-
}
392+
synchronized (this) {
393+
if (this.producer == null) {
394+
this.producer = new CloseSafeProducer<>(createKafkaProducer(), this::removeProducer,
395+
this.physicalCloseTimeout);
393396
}
397+
return this.producer;
394398
}
395-
return this.producer;
396399
}
397400

398401
/**
@@ -447,6 +450,20 @@ private void removeConsumerProducer(CloseSafeProducer<K, V> producerToRemove) {
447450
}
448451
}
449452

453+
/**
454+
* Remove the single shared producer and a thread-bound instance if present.
455+
* @param producerToRemove the producer;
456+
* @since 2.2.13
457+
*/
458+
protected final synchronized void removeProducer(
459+
@SuppressWarnings("unused") CloseSafeProducer<K, V> producerToRemove) {
460+
461+
if (producerToRemove.equals(this.producer)) {
462+
this.producer = null;
463+
}
464+
this.threadBoundProducers.remove();
465+
}
466+
450467
/**
451468
* Subclasses must return a producer from the {@link #getCache()} or a
452469
* new raw producer wrapped in a {@link CloseSafeProducer}.
@@ -482,7 +499,7 @@ private CloseSafeProducer<K, V> doCreateTxProducer(String prefix, String suffix,
482499
newProducer = createRawProducer(newProducerConfigs);
483500
newProducer.initTransactions();
484501
return new CloseSafeProducer<>(newProducer, getCache(prefix), remover,
485-
(String) newProducerConfigs.get(ProducerConfig.TRANSACTIONAL_ID_CONFIG));
502+
(String) newProducerConfigs.get(ProducerConfig.TRANSACTIONAL_ID_CONFIG), this.physicalCloseTimeout);
486503
}
487504

488505
protected Producer<K, V> createRawProducer(Map<String, Object> configs) {
@@ -545,34 +562,43 @@ protected static class CloseSafeProducer<K, V> implements Producer<K, V> {
545562

546563
private final BlockingQueue<CloseSafeProducer<K, V>> cache;
547564

548-
private final Consumer<CloseSafeProducer<K, V>> removeConsumerProducer;
565+
private final Consumer<CloseSafeProducer<K, V>> removeProducer;
549566

550567
private final String txId;
551568

552-
private volatile Exception txFailed;
569+
private final Duration closeTimeout;
570+
571+
private volatile Exception producerFailed;
572+
573+
private volatile boolean closed;
574+
575+
CloseSafeProducer(Producer<K, V> delegate, Consumer<CloseSafeProducer<K, V>> removeProducer,
576+
Duration closeTimeout) {
553577

554-
CloseSafeProducer(Producer<K, V> delegate) {
555-
this(delegate, null, null);
578+
this(delegate, null, removeProducer, null, closeTimeout);
556579
Assert.isTrue(!(delegate instanceof CloseSafeProducer), "Cannot double-wrap a producer");
557580
}
558581

559-
CloseSafeProducer(Producer<K, V> delegate, BlockingQueue<CloseSafeProducer<K, V>> cache) {
560-
this(delegate, cache, null);
582+
CloseSafeProducer(Producer<K, V> delegate, BlockingQueue<CloseSafeProducer<K, V>> cache,
583+
Duration closeTimeout) {
584+
this(delegate, cache, null, closeTimeout);
561585
}
562586

563-
CloseSafeProducer(Producer<K, V> delegate, @Nullable BlockingQueue<CloseSafeProducer<K, V>> cache,
564-
@Nullable Consumer<CloseSafeProducer<K, V>> removeConsumerProducer) {
587+
CloseSafeProducer(Producer<K, V> delegate, BlockingQueue<CloseSafeProducer<K, V>> cache,
588+
@Nullable Consumer<CloseSafeProducer<K, V>> removeConsumerProducer, Duration closeTimeout) {
565589

566-
this(delegate, cache, removeConsumerProducer, null);
590+
this(delegate, cache, removeConsumerProducer, null, closeTimeout);
567591
}
568592

569-
CloseSafeProducer(Producer<K, V> delegate, @Nullable BlockingQueue<CloseSafeProducer<K, V>> cache,
570-
@Nullable Consumer<CloseSafeProducer<K, V>> removeConsumerProducer, @Nullable String txId) {
593+
CloseSafeProducer(Producer<K, V> delegate, BlockingQueue<CloseSafeProducer<K, V>> cache,
594+
@Nullable Consumer<CloseSafeProducer<K, V>> removeProducer, @Nullable String txId,
595+
Duration closeTimeout) {
571596

572597
this.delegate = delegate;
573598
this.cache = cache;
574-
this.removeConsumerProducer = removeConsumerProducer;
599+
this.removeProducer = removeProducer;
575600
this.txId = txId;
601+
this.closeTimeout = closeTimeout;
576602
LOGGER.debug(() -> "Created new Producer: " + this);
577603
}
578604

@@ -589,7 +615,18 @@ public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
589615
@Override
590616
public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
591617
LOGGER.trace(() -> toString() + " send(" + record + ")");
592-
return this.delegate.send(record, callback);
618+
return this.delegate.send(record, new Callback() {
619+
620+
@Override
621+
public void onCompletion(RecordMetadata metadata, Exception exception) {
622+
if (exception instanceof OutOfOrderSequenceException) {
623+
CloseSafeProducer.this.producerFailed = exception;
624+
close(CloseSafeProducer.this.closeTimeout);
625+
}
626+
callback.onCompletion(metadata, exception);
627+
}
628+
629+
});
593630
}
594631

595632
@Override
@@ -621,7 +658,7 @@ public void beginTransaction() throws ProducerFencedException {
621658
}
622659
catch (RuntimeException e) {
623660
LOGGER.error(e, () -> "beginTransaction failed: " + this);
624-
this.txFailed = e;
661+
this.producerFailed = e;
625662
throw e;
626663
}
627664
}
@@ -642,16 +679,16 @@ public void commitTransaction() throws ProducerFencedException {
642679
}
643680
catch (RuntimeException e) {
644681
LOGGER.error(e, () -> "commitTransaction failed: " + this);
645-
this.txFailed = e;
682+
this.producerFailed = e;
646683
throw e;
647684
}
648685
}
649686

650687
@Override
651688
public void abortTransaction() throws ProducerFencedException {
652689
LOGGER.debug(() -> toString() + " abortTransaction()");
653-
if (this.txFailed != null) {
654-
LOGGER.debug(() -> "abortTransaction ignored - previous txFailed: " + this.txFailed.getMessage()
690+
if (this.producerFailed != null) {
691+
LOGGER.debug(() -> "abortTransaction ignored - previous txFailed: " + this.producerFailed.getMessage()
655692
+ ": " + this);
656693
}
657694
else {
@@ -660,7 +697,7 @@ public void abortTransaction() throws ProducerFencedException {
660697
}
661698
catch (RuntimeException e) {
662699
LOGGER.error(e, () -> "Abort failed: " + this);
663-
this.txFailed = e;
700+
this.producerFailed = e;
664701
throw e;
665702
}
666703
}
@@ -681,25 +718,26 @@ public void close(long timeout, @Nullable TimeUnit unit) {
681718
@Override
682719
public void close(@Nullable Duration timeout) {
683720
LOGGER.trace(() -> toString() + " close(" + (timeout == null ? "null" : timeout) + ")");
684-
if (this.cache != null) {
685-
Duration closeTimeout = this.txFailed instanceof TimeoutException
686-
? CLOSE_TIMEOUT_AFTER_TX_TIMEOUT
687-
: timeout;
688-
if (this.txFailed != null) {
689-
LOGGER.warn(() -> "Error during transactional operation; producer removed from cache; "
690-
+ "possible cause: "
691-
+ "broker restarted during transaction: " + this);
692-
this.delegate.close(closeTimeout);
693-
if (this.removeConsumerProducer != null) {
694-
this.removeConsumerProducer.accept(this);
721+
if (!this.closed) {
722+
if (this.producerFailed != null) {
723+
LOGGER.warn(() -> "Error during some operation; producer removed from cache: " + this);
724+
this.closed = true;
725+
this.delegate.close(this.producerFailed instanceof TimeoutException
726+
? CLOSE_TIMEOUT_AFTER_TX_TIMEOUT
727+
: timeout);
728+
if (this.removeProducer != null) {
729+
this.removeProducer.accept(this);
695730
}
696731
}
697732
else {
698-
if (this.removeConsumerProducer == null) { // dedicated consumer producers are not cached
733+
if (this.cache != null && this.removeProducer == null) { // dedicated consumer producers are not cached
699734
synchronized (this) {
700735
if (!this.cache.contains(this)
701736
&& !this.cache.offer(this)) {
702-
this.delegate.close(closeTimeout);
737+
this.closed = true;
738+
this.delegate.close(this.producerFailed instanceof TimeoutException
739+
? CLOSE_TIMEOUT_AFTER_TX_TIMEOUT
740+
: timeout);
703741
}
704742
}
705743
}

spring-kafka/src/test/java/org/springframework/kafka/core/DefaultKafkaProducerFactoryTests.java

Lines changed: 36 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
import static org.assertj.core.api.Assertions.assertThat;
2020
import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
2121
import static org.mockito.ArgumentMatchers.any;
22+
import static org.mockito.ArgumentMatchers.isNull;
2223
import static org.mockito.BDDMockito.given;
2324
import static org.mockito.BDDMockito.willAnswer;
2425
import static org.mockito.BDDMockito.willThrow;
@@ -31,11 +32,14 @@
3132
import java.util.Map;
3233
import java.util.Queue;
3334
import java.util.concurrent.BlockingQueue;
35+
import java.util.concurrent.atomic.AtomicBoolean;
3436
import java.util.concurrent.atomic.AtomicInteger;
3537

38+
import org.apache.kafka.clients.producer.Callback;
3639
import org.apache.kafka.clients.producer.Producer;
3740
import org.apache.kafka.common.KafkaException;
3841
import org.apache.kafka.common.errors.ProducerFencedException;
42+
import org.apache.kafka.common.errors.UnknownProducerIdException;
3943
import org.junit.jupiter.api.Test;
4044
import org.mockito.InOrder;
4145

@@ -67,7 +71,8 @@ protected Producer createTransactionalProducer(String txIdPrefix) {
6771
producer.initTransactions();
6872
BlockingQueue<Producer> cache = getCache(txIdPrefix);
6973
Producer cached = cache.poll();
70-
return cached == null ? new CloseSafeProducer(producer, cache) : cached;
74+
return cached == null ? new CloseSafeProducer(producer, cache,
75+
Duration.ofSeconds(1)) : cached;
7176
}
7277

7378
};
@@ -148,7 +153,7 @@ protected Producer createTransactionalProducer(String txIdPrefix) {
148153
producer.initTransactions();
149154
BlockingQueue<Producer> cache = getCache(txIdPrefix);
150155
Producer cached = cache.poll();
151-
return cached == null ? new CloseSafeProducer(producer, cache) : cached;
156+
return cached == null ? new CloseSafeProducer(producer, cache, Duration.ofSeconds(1)) : cached;
152157
}
153158

154159
};
@@ -221,4 +226,33 @@ protected Producer createRawProducer(Map configs) {
221226
assertThat(KafkaTestUtils.getPropertyValue(pf, "consumerProducers", Map.class)).hasSize(0);
222227
}
223228

229+
@Test
230+
@SuppressWarnings({ "rawtypes", "unchecked" })
231+
void testUnknownProducerIdException() {
232+
final Producer producer1 = mock(Producer.class);
233+
willAnswer(inv -> {
234+
((Callback) inv.getArgument(1)).onCompletion(null, new UnknownProducerIdException("test"));
235+
return null;
236+
}).given(producer1).send(isNull(), any());
237+
final Producer producer2 = mock(Producer.class);
238+
ProducerFactory pf = new DefaultKafkaProducerFactory(new HashMap<>()) {
239+
240+
private final AtomicBoolean first = new AtomicBoolean(true);
241+
242+
@Override
243+
protected Producer createKafkaProducer() {
244+
return this.first.getAndSet(false) ? producer1 : producer2;
245+
}
246+
247+
};
248+
final Producer aProducer = pf.createProducer();
249+
assertThat(aProducer).isNotNull();
250+
aProducer.send(null, (meta, ex) -> { });
251+
aProducer.close(ProducerFactoryUtils.DEFAULT_CLOSE_TIMEOUT);
252+
assertThat(KafkaTestUtils.getPropertyValue(pf, "producer")).isNull();
253+
verify(producer1).close(any(Duration.class));
254+
Producer bProducer = pf.createProducer();
255+
assertThat(bProducer).isNotSameAs(aProducer);
256+
}
257+
224258
}

spring-kafka/src/test/java/org/springframework/kafka/core/KafkaTemplateTransactionTests.java

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -375,7 +375,8 @@ public void testQuickCloseAfterCommitTimeout() {
375375

376376
@Override
377377
public Producer<String, String> createProducer(String txIdPrefixArg) {
378-
CloseSafeProducer<String, String> closeSafeProducer = new CloseSafeProducer<>(producer, getCache());
378+
CloseSafeProducer<String, String> closeSafeProducer = new CloseSafeProducer<>(producer, getCache(),
379+
Duration.ofSeconds(1));
379380
return closeSafeProducer;
380381
}
381382

@@ -400,19 +401,22 @@ public void testNormalCloseAfterCommitCacheFull() {
400401
@SuppressWarnings("unchecked")
401402
Producer<String, String> producer = mock(Producer.class);
402403

403-
DefaultKafkaProducerFactory<String, String> pf = new DefaultKafkaProducerFactory<String, String>(Collections.emptyMap()) {
404+
DefaultKafkaProducerFactory<String, String> pf =
405+
new DefaultKafkaProducerFactory<String, String>(Collections.emptyMap()) {
404406

405407
@SuppressWarnings("unchecked")
406408
@Override
407409
public Producer<String, String> createProducer(String txIdPrefixArg) {
408410
BlockingQueue<CloseSafeProducer<String, String>> cache = new LinkedBlockingDeque<>(1);
409411
try {
410-
cache.put(new CloseSafeProducer<>(mock(Producer.class)));
412+
cache.put(new CloseSafeProducer<>(mock(Producer.class), this::removeProducer,
413+
Duration.ofSeconds(1)));
411414
}
412415
catch (@SuppressWarnings("unused") InterruptedException e) {
413416
Thread.currentThread().interrupt();
414417
}
415-
CloseSafeProducer<String, String> closeSafeProducer = new CloseSafeProducer<>(producer, cache);
418+
CloseSafeProducer<String, String> closeSafeProducer = new CloseSafeProducer<>(producer, cache,
419+
Duration.ofSeconds(1));
416420
return closeSafeProducer;
417421
}
418422

0 commit comments

Comments
 (0)