Skip to content

Commit 01c9bba

Browse files
garyrussellartembilan
authored andcommitted
GH-1168: Fix sendOffsets in local transaction
Fixes #1168 `sendOffsetsToTransaction` assumed the transaction was started by a `KafkaTransactionManager` and using `executeInTransaction` failed. Look for a local transactional producer before checking for a thread-bound transaction. **cherry-pick to all supported branches** # Conflicts: # spring-kafka/src/test/java/org/springframework/kafka/core/KafkaTemplateTransactionTests.java # Conflicts: # spring-kafka/src/test/java/org/springframework/kafka/core/KafkaTemplateTransactionTests.java
1 parent 15ab184 commit 01c9bba

File tree

2 files changed

+30
-12
lines changed

2 files changed

+30
-12
lines changed

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

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -331,13 +331,15 @@ public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offs
331331

332332
@Override
333333
public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId) {
334-
@SuppressWarnings("unchecked")
335-
KafkaResourceHolder<K, V> resourceHolder = (KafkaResourceHolder<K, V>) TransactionSynchronizationManager
336-
.getResource(this.producerFactory);
337-
Assert.isTrue(resourceHolder != null, "No transaction in process");
338-
if (resourceHolder.getProducer() != null) {
339-
resourceHolder.getProducer().sendOffsetsToTransaction(offsets, consumerGroupId);
334+
Producer<K, V> producer = this.producers.get();
335+
if (producer == null) {
336+
@SuppressWarnings("unchecked")
337+
KafkaResourceHolder<K, V> resourceHolder = (KafkaResourceHolder<K, V>) TransactionSynchronizationManager
338+
.getResource(this.producerFactory);
339+
Assert.isTrue(resourceHolder != null, "No transaction in process");
340+
producer = resourceHolder.getProducer();
340341
}
342+
producer.sendOffsetsToTransaction(offsets, consumerGroupId);
341343
}
342344

343345
protected void closeProducer(Producer<K, V> producer, boolean inLocalTx) {

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

Lines changed: 22 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -38,11 +38,13 @@
3838
import org.apache.kafka.clients.consumer.ConsumerConfig;
3939
import org.apache.kafka.clients.consumer.ConsumerRecord;
4040
import org.apache.kafka.clients.consumer.ConsumerRecords;
41+
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
4142
import org.apache.kafka.clients.producer.Callback;
4243
import org.apache.kafka.clients.producer.MockProducer;
4344
import org.apache.kafka.clients.producer.Producer;
4445
import org.apache.kafka.clients.producer.ProducerConfig;
4546
import org.apache.kafka.clients.producer.ProducerRecord;
47+
import org.apache.kafka.common.TopicPartition;
4648
import org.apache.kafka.common.serialization.StringDeserializer;
4749
import org.apache.kafka.common.serialization.StringSerializer;
4850
import org.assertj.core.api.Assertions;
@@ -77,10 +79,13 @@ public class KafkaTemplateTransactionTests {
7779

7880
private static final String STRING_KEY_TOPIC = "stringKeyTopic";
7981

82+
private static final String LOCAL_TX_IN_TOPIC = "localTxInTopic";
83+
8084
@ClassRule
81-
public static KafkaEmbedded embeddedKafka = new KafkaEmbedded(1, true, STRING_KEY_TOPIC)
82-
.brokerProperty(KafkaConfig.TransactionsTopicReplicationFactorProp(), "1")
83-
.brokerProperty(KafkaConfig.TransactionsTopicMinISRProp(), "1");
85+
public static KafkaEmbedded embeddedKafka = new KafkaEmbedded(1, true, STRING_KEY_TOPIC,
86+
LOCAL_TX_IN_TOPIC)
87+
.brokerProperty(KafkaConfig.TransactionsTopicReplicationFactorProp(), "1")
88+
.brokerProperty(KafkaConfig.TransactionsTopicMinISRProp(), "1");
8489

8590
@Test
8691
public void testLocalTransaction() throws Exception {
@@ -93,13 +98,22 @@ public void testLocalTransaction() throws Exception {
9398
template.setDefaultTopic(STRING_KEY_TOPIC);
9499
Map<String, Object> consumerProps = KafkaTestUtils.consumerProps("testTxString", "false", embeddedKafka);
95100
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
101+
consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
96102
DefaultKafkaConsumerFactory<String, String> cf = new DefaultKafkaConsumerFactory<>(consumerProps);
97103
cf.setKeyDeserializer(new StringDeserializer());
98104
Consumer<String, String> consumer = cf.createConsumer();
99-
embeddedKafka.consumeFromAnEmbeddedTopic(consumer, STRING_KEY_TOPIC);
105+
embeddedKafka.consumeFromAllEmbeddedTopics(consumer);
106+
template.executeInTransaction(kt -> kt.send(LOCAL_TX_IN_TOPIC, "one"));
107+
ConsumerRecord<String, String> singleRecord = KafkaTestUtils.getSingleRecord(consumer, LOCAL_TX_IN_TOPIC);
100108
template.executeInTransaction(t -> {
101109
t.sendDefault("foo", "bar");
102110
t.sendDefault("baz", "qux");
111+
t.sendOffsetsToTransaction(Collections.singletonMap(
112+
new TopicPartition(LOCAL_TX_IN_TOPIC, singleRecord.partition()),
113+
new OffsetAndMetadata(singleRecord.offset() + 1L)), "testLocalTx");
114+
assertThat(KafkaTestUtils.getPropertyValue(
115+
KafkaTestUtils.getPropertyValue(template, "producers", ThreadLocal.class).get(),
116+
"delegate.transactionManager.transactionalId")).isEqualTo("my.transaction.0");
103117
return null;
104118
});
105119
ConsumerRecords<String, String> records = KafkaTestUtils.getRecords(consumer);
@@ -112,6 +126,8 @@ public void testLocalTransaction() throws Exception {
112126
}
113127
record = iterator.next();
114128
assertThat(record).has(Assertions.<ConsumerRecord<String, String>>allOf(key("baz"), value("qux")));
129+
// 2 log slots, 1 for the record, 1 for the commit
130+
assertThat(consumer.position(new TopicPartition(LOCAL_TX_IN_TOPIC, singleRecord.partition()))).isEqualTo(2L);
115131
consumer.close();
116132
assertThat(KafkaTestUtils.getPropertyValue(pf, "cache", BlockingQueue.class).size()).isEqualTo(1);
117133
pf.destroy();
@@ -210,8 +226,8 @@ public void testNoTx() {
210226
KafkaTemplate<String, String> template = new KafkaTemplate<>(pf);
211227
template.setDefaultTopic(STRING_KEY_TOPIC);
212228
assertThatThrownBy(() -> template.send("foo", "bar"))
213-
.isInstanceOf(IllegalStateException.class)
214-
.hasMessageContaining("No transaction is in process;");
229+
.isInstanceOf(IllegalStateException.class)
230+
.hasMessageContaining("No transaction is in process;");
215231
}
216232

217233
@Test

0 commit comments

Comments
 (0)