Skip to content

Commit 4855751

Browse files
garyrussellartembilan
authored andcommitted
GH-1278: Zombie Fencing with Batch Listener
Resolves #1278 Support transaction per partition with Batch Listeners so the `transactional.id` is tied to the group/topic/partition. * Fix javadoc
1 parent 5bd49b3 commit 4855751

File tree

7 files changed

+518
-9
lines changed

7 files changed

+518
-9
lines changed

spring-kafka/src/main/java/org/springframework/kafka/listener/ContainerProperties.java

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -180,6 +180,8 @@ public enum AckMode {
180180

181181
private Duration consumerStartTimout = DEFAULT_CONSUMER_START_TIMEOUT;
182182

183+
private boolean subBatchPerPartition;
184+
183185
/**
184186
* Create properties for a container that will subscribe to the specified topics.
185187
* @param topics the topics.
@@ -590,6 +592,23 @@ public void setConsumerStartTimout(Duration consumerStartTimout) {
590592
this.consumerStartTimout = consumerStartTimout;
591593
}
592594

595+
public boolean isSubBatchPerPartition() {
596+
return this.subBatchPerPartition;
597+
}
598+
599+
/**
600+
* When using a batch message listener whether to dispatch records by partition (with
601+
* a transaction for each sub batch if transactions are in use) or the complete batch
602+
* received by the {@code poll()}. Useful when using transactions to enable zombie
603+
* fencing, by using a {code transactional.id} that is unique for each
604+
* group/topic/partition.
605+
* @param subBatchPerPartition true for a separate transaction for each partition.
606+
* @since 2.3.2
607+
*/
608+
public void setSubBatchPerPartition(boolean subBatchPerPartition) {
609+
this.subBatchPerPartition = subBatchPerPartition;
610+
}
611+
593612
@Override
594613
public String toString() {
595614
return "ContainerProperties ["
@@ -611,6 +630,7 @@ public String toString() {
611630
+ ", monitorInterval=" + this.monitorInterval
612631
+ (this.scheduler != null ? ", scheduler=" + this.scheduler : "")
613632
+ ", noPollThreshold=" + this.noPollThreshold
633+
+ ", subBatchPerPartition=" + this.subBatchPerPartition
614634
+ "]";
615635
}
616636

spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java

Lines changed: 52 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import java.util.Collections;
2424
import java.util.HashMap;
2525
import java.util.Iterator;
26+
import java.util.LinkedList;
2627
import java.util.List;
2728
import java.util.Map;
2829
import java.util.Map.Entry;
@@ -37,7 +38,6 @@
3738
import java.util.concurrent.atomic.AtomicBoolean;
3839
import java.util.regex.Pattern;
3940
import java.util.stream.Collectors;
40-
import java.util.stream.StreamSupport;
4141

4242
import org.apache.commons.logging.LogFactory;
4343
import org.apache.kafka.clients.consumer.Consumer;
@@ -555,13 +555,11 @@ private final class ListenerConsumer implements SchedulingAwareRunnable, Consume
555555

556556
private final MicrometerHolder micrometerHolder;
557557

558-
private Map<TopicPartition, OffsetMetadata> definedPartitions;
559-
560558
private final AtomicBoolean polling = new AtomicBoolean();
561559

562-
private volatile Collection<TopicPartition> assignedPartitions;
560+
private final boolean subBatchPerPartition = this.containerProperties.isSubBatchPerPartition();
563561

564-
private volatile Thread consumerThread;
562+
private Map<TopicPartition, OffsetMetadata> definedPartitions;
565563

566564
private int count;
567565

@@ -579,8 +577,16 @@ private final class ListenerConsumer implements SchedulingAwareRunnable, Consume
579577

580578
private int nackIndex;
581579

580+
private Iterator<TopicPartition> batchIterator;
581+
582+
private ConsumerRecords<K, V> lastBatch;
583+
582584
private volatile boolean consumerPaused;
583585

586+
private volatile Collection<TopicPartition> assignedPartitions;
587+
588+
private volatile Thread consumerThread;
589+
584590
private volatile long lastPoll = System.currentTimeMillis();
585591

586592
@SuppressWarnings(UNCHECKED)
@@ -929,7 +935,7 @@ protected void pollAndInvoke() {
929935
checkPaused();
930936
this.lastPoll = System.currentTimeMillis();
931937
this.polling.set(true);
932-
ConsumerRecords<K, V> records = this.consumer.poll(this.pollTimeout);
938+
ConsumerRecords<K, V> records = doPoll();
933939
if (!this.polling.compareAndSet(true, false)) {
934940
/*
935941
* There is a small race condition where wakeIfNecessary was called between
@@ -953,6 +959,31 @@ protected void pollAndInvoke() {
953959
}
954960
}
955961

962+
private ConsumerRecords<K, V> doPoll() {
963+
ConsumerRecords<K, V> records;
964+
if (this.isBatchListener && this.subBatchPerPartition) {
965+
if (this.batchIterator == null) {
966+
this.lastBatch = this.consumer.poll(this.pollTimeout);
967+
if (this.lastBatch.count() == 0) {
968+
return this.lastBatch;
969+
}
970+
else {
971+
this.batchIterator = this.lastBatch.partitions().iterator();
972+
}
973+
}
974+
TopicPartition next = this.batchIterator.next();
975+
List<ConsumerRecord<K, V>> subBatch = this.lastBatch.records(next);
976+
records = new ConsumerRecords<>(Collections.singletonMap(next, subBatch));
977+
if (!this.batchIterator.hasNext()) {
978+
this.batchIterator = null;
979+
}
980+
}
981+
else {
982+
records = this.consumer.poll(this.pollTimeout);
983+
}
984+
return records;
985+
}
986+
956987
void wakeIfNecessary() {
957988
if (this.polling.getAndSet(false)) {
958989
this.consumer.wakeup();
@@ -1186,6 +1217,10 @@ private void invokeBatchListenerInTx(final ConsumerRecords<K, V> records,
11861217
final List<ConsumerRecord<K, V>> recordList) {
11871218

11881219
try {
1220+
if (this.subBatchPerPartition) {
1221+
ConsumerRecord<K, V> record = recordList.get(0);
1222+
TransactionSupport.setTransactionIdSuffix(zombieFenceTxIdSuffix(record.topic(), record.partition()));
1223+
}
11891224
this.transactionTemplate.execute(new TransactionCallbackWithoutResult() {
11901225

11911226
@Override
@@ -1221,6 +1256,11 @@ protected void doInTransactionWithoutResult(TransactionStatus status) {
12211256
batchAfterRollback(records, recordList, e, afterRollbackProcessorToUse);
12221257
}
12231258
}
1259+
finally {
1260+
if (this.subBatchPerPartition) {
1261+
TransactionSupport.clearTransactionIdSuffix();
1262+
}
1263+
}
12241264
}
12251265

12261266
private void batchAfterRollback(final ConsumerRecords<K, V> records,
@@ -1241,8 +1281,12 @@ private void batchAfterRollback(final ConsumerRecords<K, V> records,
12411281
}
12421282

12431283
private List<ConsumerRecord<K, V>> createRecordList(final ConsumerRecords<K, V> records) {
1244-
return StreamSupport.stream(records.spliterator(), false)
1245-
.collect(Collectors.toList());
1284+
Iterator<ConsumerRecord<K, V>> iterator = records.iterator();
1285+
List<ConsumerRecord<K, V>> list = new LinkedList<>();
1286+
while (iterator.hasNext()) {
1287+
list.add(iterator.next());
1288+
}
1289+
return list;
12461290
}
12471291

12481292
/**

spring-kafka/src/test/java/org/springframework/kafka/listener/SeekToCurrentOnErrorBatchModeTXTests.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -109,26 +109,32 @@ public void discardRemainingRecordsFromPollAndSeek() throws Exception {
109109
inOrder.verify(this.producer).commitTransaction();
110110
offsets.clear();
111111
offsets.put(new TopicPartition("foo", 0), new OffsetAndMetadata(2L));
112+
inOrder.verify(this.producer).beginTransaction();
112113
inOrder.verify(this.producer).sendOffsetsToTransaction(offsets, CONTAINER_ID);
113114
inOrder.verify(this.producer).commitTransaction();
114115
offsets.clear();
115116
offsets.put(new TopicPartition("foo", 1), new OffsetAndMetadata(1L));
117+
inOrder.verify(this.producer).beginTransaction();
116118
inOrder.verify(this.producer).sendOffsetsToTransaction(offsets, CONTAINER_ID);
117119
inOrder.verify(this.producer).commitTransaction();
120+
inOrder.verify(this.producer).beginTransaction();
118121
inOrder.verify(this.consumer).seek(new TopicPartition("foo", 1), 1L);
119122
inOrder.verify(this.consumer).seek(new TopicPartition("foo", 2), 0L);
120123
inOrder.verify(this.producer).abortTransaction();
121124
inOrder.verify(this.consumer).poll(Duration.ofMillis(ContainerProperties.DEFAULT_POLL_TIMEOUT));
122125
offsets.clear();
123126
offsets.put(new TopicPartition("foo", 1), new OffsetAndMetadata(2L));
127+
inOrder.verify(this.producer).beginTransaction();
124128
inOrder.verify(this.producer).sendOffsetsToTransaction(offsets, CONTAINER_ID);
125129
inOrder.verify(this.producer).commitTransaction();
126130
offsets.clear();
127131
offsets.put(new TopicPartition("foo", 2), new OffsetAndMetadata(1L));
132+
inOrder.verify(this.producer).beginTransaction();
128133
inOrder.verify(this.producer).sendOffsetsToTransaction(offsets, CONTAINER_ID);
129134
inOrder.verify(this.producer).commitTransaction();
130135
offsets.clear();
131136
offsets.put(new TopicPartition("foo", 2), new OffsetAndMetadata(2L));
137+
inOrder.verify(this.producer).beginTransaction();
132138
inOrder.verify(this.producer).sendOffsetsToTransaction(offsets, CONTAINER_ID);
133139
inOrder.verify(this.producer).commitTransaction();
134140
inOrder.verify(this.consumer).poll(Duration.ofMillis(ContainerProperties.DEFAULT_POLL_TIMEOUT));
Lines changed: 196 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,196 @@
1+
/*
2+
* Copyright 2017-2019 the original author or authors.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* https://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package org.springframework.kafka.listener;
18+
19+
import static org.assertj.core.api.Assertions.assertThat;
20+
import static org.mockito.ArgumentMatchers.any;
21+
import static org.mockito.ArgumentMatchers.anyMap;
22+
import static org.mockito.ArgumentMatchers.eq;
23+
import static org.mockito.BDDMockito.given;
24+
import static org.mockito.BDDMockito.willAnswer;
25+
import static org.mockito.Mockito.inOrder;
26+
import static org.mockito.Mockito.mock;
27+
import static org.mockito.Mockito.times;
28+
29+
import java.time.Duration;
30+
import java.util.ArrayList;
31+
import java.util.Arrays;
32+
import java.util.Collection;
33+
import java.util.Collections;
34+
import java.util.LinkedHashMap;
35+
import java.util.List;
36+
import java.util.Map;
37+
import java.util.concurrent.CountDownLatch;
38+
import java.util.concurrent.TimeUnit;
39+
import java.util.concurrent.atomic.AtomicInteger;
40+
41+
import org.apache.kafka.clients.consumer.Consumer;
42+
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
43+
import org.apache.kafka.clients.consumer.ConsumerRecord;
44+
import org.apache.kafka.clients.consumer.ConsumerRecords;
45+
import org.apache.kafka.common.TopicPartition;
46+
import org.apache.kafka.common.record.TimestampType;
47+
import org.junit.jupiter.api.Test;
48+
import org.mockito.InOrder;
49+
50+
import org.springframework.beans.factory.annotation.Autowired;
51+
import org.springframework.context.annotation.Bean;
52+
import org.springframework.context.annotation.Configuration;
53+
import org.springframework.kafka.annotation.EnableKafka;
54+
import org.springframework.kafka.annotation.KafkaListener;
55+
import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory;
56+
import org.springframework.kafka.config.KafkaListenerEndpointRegistry;
57+
import org.springframework.kafka.core.ConsumerFactory;
58+
import org.springframework.kafka.test.utils.KafkaTestUtils;
59+
import org.springframework.test.annotation.DirtiesContext;
60+
import org.springframework.test.context.junit.jupiter.SpringJUnitConfig;
61+
62+
/**
63+
* @author Gary Russell
64+
* @since 2.3.2
65+
*
66+
*/
67+
@SpringJUnitConfig
68+
@DirtiesContext
69+
public class SubBatchPerPartitionTests {
70+
71+
private static final String CONTAINER_ID = "container";
72+
73+
@SuppressWarnings("rawtypes")
74+
@Autowired
75+
private Consumer consumer;
76+
77+
@Autowired
78+
private Config config;
79+
80+
@Autowired
81+
private KafkaListenerEndpointRegistry registry;
82+
83+
/*
84+
* Deliver 6 records from three partitions, fail on the second record second
85+
* partition.
86+
*/
87+
@SuppressWarnings("unchecked")
88+
@Test
89+
public void discardRemainingRecordsFromPollAndSeek() throws Exception {
90+
assertThat(this.config.deliveryLatch.await(10, TimeUnit.SECONDS)).isTrue();
91+
assertThat(this.config.commitLatch.await(10, TimeUnit.SECONDS)).isTrue();
92+
assertThat(this.config.pollLatch.await(10, TimeUnit.SECONDS)).isTrue();
93+
this.registry.stop();
94+
assertThat(this.config.closeLatch.await(10, TimeUnit.SECONDS)).isTrue();
95+
InOrder inOrder = inOrder(this.consumer);
96+
inOrder.verify(this.consumer).subscribe(any(Collection.class), any(ConsumerRebalanceListener.class));
97+
inOrder.verify(this.consumer).poll(Duration.ofMillis(ContainerProperties.DEFAULT_POLL_TIMEOUT));
98+
inOrder.verify(this.consumer, times(3)).commitSync(any(), eq(Duration.ofSeconds(60)));
99+
inOrder.verify(this.consumer).poll(Duration.ofMillis(ContainerProperties.DEFAULT_POLL_TIMEOUT));
100+
assertThat(this.config.contents).contains("foo", "bar", "baz", "qux", "fiz", "buz");
101+
}
102+
103+
@Configuration
104+
@EnableKafka
105+
public static class Config {
106+
107+
private final List<String> contents = new ArrayList<>();
108+
109+
private final CountDownLatch pollLatch = new CountDownLatch(2);
110+
111+
private final CountDownLatch deliveryLatch = new CountDownLatch(3);
112+
113+
private final CountDownLatch commitLatch = new CountDownLatch(3);
114+
115+
private final CountDownLatch closeLatch = new CountDownLatch(1);
116+
117+
@KafkaListener(id = CONTAINER_ID, topics = "foo")
118+
public void foo(List<String> in) {
119+
contents.addAll(in);
120+
this.deliveryLatch.countDown();
121+
}
122+
123+
@SuppressWarnings({ "rawtypes" })
124+
@Bean
125+
public ConsumerFactory consumerFactory() {
126+
ConsumerFactory consumerFactory = mock(ConsumerFactory.class);
127+
final Consumer consumer = consumer();
128+
given(consumerFactory.createConsumer(CONTAINER_ID, "", "-0", KafkaTestUtils.defaultPropertyOverrides()))
129+
.willReturn(consumer);
130+
return consumerFactory;
131+
}
132+
133+
@SuppressWarnings({ "rawtypes", "unchecked" })
134+
@Bean
135+
public Consumer consumer() {
136+
final Consumer consumer = mock(Consumer.class);
137+
final TopicPartition topicPartition0 = new TopicPartition("foo", 0);
138+
final TopicPartition topicPartition1 = new TopicPartition("foo", 1);
139+
final TopicPartition topicPartition2 = new TopicPartition("foo", 2);
140+
willAnswer(i -> {
141+
((ConsumerRebalanceListener) i.getArgument(1)).onPartitionsAssigned(
142+
Arrays.asList(topicPartition0, topicPartition1, topicPartition2));
143+
return null;
144+
}).given(consumer).subscribe(any(Collection.class), any(ConsumerRebalanceListener.class));
145+
Map<TopicPartition, List<ConsumerRecord>> records1 = new LinkedHashMap<>();
146+
records1.put(topicPartition0, Arrays.asList(
147+
new ConsumerRecord("foo", 0, 0L, 0L, TimestampType.NO_TIMESTAMP_TYPE, 0, 0, 0, null, "foo"),
148+
new ConsumerRecord("foo", 0, 1L, 0L, TimestampType.NO_TIMESTAMP_TYPE, 0, 0, 0, null, "bar")));
149+
records1.put(topicPartition1, Arrays.asList(
150+
new ConsumerRecord("foo", 1, 0L, 0L, TimestampType.NO_TIMESTAMP_TYPE, 0, 0, 0, null, "baz"),
151+
new ConsumerRecord("foo", 1, 1L, 0L, TimestampType.NO_TIMESTAMP_TYPE, 0, 0, 0, null, "qux")));
152+
records1.put(topicPartition2, Arrays.asList(
153+
new ConsumerRecord("foo", 2, 0L, 0L, TimestampType.NO_TIMESTAMP_TYPE, 0, 0, 0, null, "fiz"),
154+
new ConsumerRecord("foo", 2, 1L, 0L, TimestampType.NO_TIMESTAMP_TYPE, 0, 0, 0, null, "buz")));
155+
final AtomicInteger which = new AtomicInteger();
156+
willAnswer(i -> {
157+
this.pollLatch.countDown();
158+
switch (which.getAndIncrement()) {
159+
case 0:
160+
return new ConsumerRecords(records1);
161+
default:
162+
try {
163+
Thread.sleep(100);
164+
}
165+
catch (@SuppressWarnings("unused") InterruptedException e) {
166+
Thread.currentThread().interrupt();
167+
}
168+
return new ConsumerRecords(Collections.emptyMap());
169+
}
170+
}).given(consumer).poll(Duration.ofMillis(ContainerProperties.DEFAULT_POLL_TIMEOUT));
171+
willAnswer(i -> {
172+
this.commitLatch.countDown();
173+
return null;
174+
}).given(consumer).commitSync(anyMap(), any());
175+
willAnswer(i -> {
176+
this.closeLatch.countDown();
177+
return null;
178+
}).given(consumer).close();
179+
return consumer;
180+
}
181+
182+
@SuppressWarnings({ "rawtypes", "unchecked" })
183+
@Bean
184+
public ConcurrentKafkaListenerContainerFactory kafkaListenerContainerFactory() {
185+
ConcurrentKafkaListenerContainerFactory factory = new ConcurrentKafkaListenerContainerFactory();
186+
factory.setConsumerFactory(consumerFactory());
187+
factory.getContainerProperties().setAckOnError(false);
188+
factory.setBatchListener(true);
189+
factory.getContainerProperties().setMissingTopicsFatal(false);
190+
factory.getContainerProperties().setSubBatchPerPartition(true);
191+
return factory;
192+
}
193+
194+
}
195+
196+
}

0 commit comments

Comments
 (0)