Skip to content

Commit 82322d4

Browse files
garyrussellartembilan
authored andcommitted
GH-731: Replying template improvements
Resolves #731 - Auto detect reply topic/partition - Improve docs to explain how to use a shared reply topic - Add boolean to reduce log level for unexpected responses * Polishing - PR Comments
1 parent 62a6f87 commit 82322d4

File tree

3 files changed

+131
-12
lines changed

3 files changed

+131
-12
lines changed

spring-kafka/src/main/java/org/springframework/kafka/requestreply/ReplyingKafkaTemplate.java

Lines changed: 70 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
package org.springframework.kafka.requestreply;
1818

1919
import java.nio.ByteBuffer;
20+
import java.nio.charset.StandardCharsets;
2021
import java.time.Instant;
2122
import java.util.Collection;
2223
import java.util.Iterator;
@@ -29,6 +30,7 @@
2930
import org.apache.kafka.clients.producer.ProducerRecord;
3031
import org.apache.kafka.common.TopicPartition;
3132
import org.apache.kafka.common.header.Header;
33+
import org.apache.kafka.common.header.Headers;
3234
import org.apache.kafka.common.header.internals.RecordHeader;
3335

3436
import org.springframework.beans.factory.DisposableBean;
@@ -38,6 +40,7 @@
3840
import org.springframework.kafka.core.KafkaTemplate;
3941
import org.springframework.kafka.core.ProducerFactory;
4042
import org.springframework.kafka.listener.BatchMessageListener;
43+
import org.springframework.kafka.listener.ContainerProperties;
4144
import org.springframework.kafka.listener.GenericMessageListenerContainer;
4245
import org.springframework.kafka.support.KafkaHeaders;
4346
import org.springframework.kafka.support.SendResult;
@@ -66,6 +69,10 @@ public class ReplyingKafkaTemplate<K, V, R> extends KafkaTemplate<K, V> implemen
6669

6770
private final ConcurrentMap<CorrelationKey, RequestReplyFuture<K, V, R>> futures = new ConcurrentHashMap<>();
6871

72+
private final byte[] replyTopic;
73+
74+
private final byte[] replyPartition;
75+
6976
private TaskScheduler scheduler = new ThreadPoolTaskScheduler();
7077

7178
private int phase;
@@ -74,7 +81,9 @@ public class ReplyingKafkaTemplate<K, V, R> extends KafkaTemplate<K, V> implemen
7481

7582
private long replyTimeout = DEFAULT_REPLY_TIMEOUT;
7683

77-
private volatile boolean schedulerSet;
84+
private boolean schedulerSet;
85+
86+
private boolean sharedReplyTopic;
7887

7988
private volatile boolean running;
8089

@@ -89,6 +98,28 @@ public ReplyingKafkaTemplate(ProducerFactory<K, V> producerFactory,
8998
Assert.notNull(replyContainer, "'replyContainer' cannot be null");
9099
this.replyContainer = replyContainer;
91100
this.replyContainer.setupMessageListener(this);
101+
ContainerProperties properties = this.replyContainer.getContainerProperties();
102+
String replyTopic = null;
103+
byte[] replyPartition = null;
104+
if (properties.getTopics() != null && properties.getTopics().length == 1) {
105+
replyTopic = properties.getTopics()[0];
106+
}
107+
else if (properties.getTopicPartitions() != null && properties.getTopicPartitions().length == 1) {
108+
replyTopic = properties.getTopicPartitions()[0].topic();
109+
ByteBuffer buffer = ByteBuffer.allocate(4);
110+
buffer.putInt(properties.getTopicPartitions()[0].partition());
111+
replyPartition = buffer.array();
112+
}
113+
if (replyTopic == null) {
114+
this.replyTopic = null;
115+
this.replyPartition = null;
116+
this.logger.debug("Could not determine container's reply topic/partition; senders must populate "
117+
+ "at least the " + KafkaHeaders.REPLY_PARTITION + " header");
118+
}
119+
else {
120+
this.replyTopic = replyTopic.getBytes(StandardCharsets.UTF_8);
121+
this.replyPartition = replyPartition;
122+
}
92123
}
93124

94125
public void setTaskScheduler(TaskScheduler scheduler) {
@@ -133,6 +164,16 @@ public Collection<TopicPartition> getAssignedReplyTopicPartitions() {
133164
return this.replyContainer.getAssignedPartitions();
134165
}
135166

167+
/**
168+
* Set to true when multiple templates are using the same topic for replies.
169+
* This simply changes logs for unexpected replies to debug instead of error.
170+
* @param sharedReplyTopic true if using a shared topic.
171+
* @since 2.2
172+
*/
173+
public void setSharedReplyTopic(boolean sharedReplyTopic) {
174+
this.sharedReplyTopic = sharedReplyTopic;
175+
}
176+
136177
@Override
137178
public void afterPropertiesSet() throws Exception {
138179
if (!this.schedulerSet) {
@@ -174,7 +215,21 @@ public RequestReplyFuture<K, V, R> sendAndReceive(ProducerRecord<K, V> record) {
174215
Assert.state(this.running, "Template has not been start()ed"); // NOSONAR (sync)
175216
CorrelationKey correlationId = createCorrelationId(record);
176217
Assert.notNull(correlationId, "the created 'correlationId' cannot be null");
177-
record.headers().add(new RecordHeader(KafkaHeaders.CORRELATION_ID, correlationId.getCorrelationId()));
218+
boolean hasReplyTopic = false;
219+
Headers headers = record.headers();
220+
Iterator<Header> iterator = headers.iterator();
221+
while (iterator.hasNext() && !hasReplyTopic) {
222+
if (iterator.next().key().equals(KafkaHeaders.REPLY_TOPIC)) {
223+
hasReplyTopic = true;
224+
}
225+
}
226+
if (!hasReplyTopic && this.replyTopic != null) {
227+
headers.add(new RecordHeader(KafkaHeaders.REPLY_TOPIC, this.replyTopic));
228+
if (this.replyPartition != null) {
229+
headers.add(new RecordHeader(KafkaHeaders.REPLY_PARTITION, this.replyPartition));
230+
}
231+
}
232+
headers.add(new RecordHeader(KafkaHeaders.CORRELATION_ID, correlationId.getCorrelationId()));
178233
if (this.logger.isDebugEnabled()) {
179234
this.logger.debug("Sending: " + record + " with correlationId: " + correlationId);
180235
}
@@ -240,8 +295,14 @@ public void onMessage(List<ConsumerRecord<K, R>> data) {
240295
else {
241296
RequestReplyFuture<K, V, R> future = this.futures.remove(correlationId);
242297
if (future == null) {
243-
this.logger.error("No pending reply: " + record + " with correlationId: "
244-
+ correlationId + ", perhaps timed out");
298+
if (this.sharedReplyTopic) {
299+
if (this.logger.isDebugEnabled()) {
300+
this.logger.debug(missingCorrelationLogMessage(record, correlationId));
301+
}
302+
}
303+
else if (this.logger.isErrorEnabled()) {
304+
this.logger.error(missingCorrelationLogMessage(record, correlationId));
305+
}
245306
}
246307
else {
247308
if (this.logger.isDebugEnabled()) {
@@ -253,6 +314,11 @@ public void onMessage(List<ConsumerRecord<K, R>> data) {
253314
});
254315
}
255316

317+
private String missingCorrelationLogMessage(ConsumerRecord<K, R> record, CorrelationKey correlationId) {
318+
return "No pending reply: " + record + " with correlationId: "
319+
+ correlationId + ", perhaps timed out, or using a shared reply topic";
320+
}
321+
256322
/**
257323
* A listenable future for requests/replies.
258324
*

spring-kafka/src/test/java/org/springframework/kafka/requestreply/ReplyingKafkaTemplateTests.java

Lines changed: 42 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,7 @@
5252
import org.springframework.kafka.listener.KafkaMessageListenerContainer;
5353
import org.springframework.kafka.support.KafkaHeaders;
5454
import org.springframework.kafka.support.SimpleKafkaHeaderMapper;
55+
import org.springframework.kafka.support.TopicPartitionInitialOffset;
5556
import org.springframework.kafka.support.converter.MessagingMessageConverter;
5657
import org.springframework.kafka.test.rule.KafkaEmbedded;
5758
import org.springframework.kafka.test.utils.KafkaTestUtils;
@@ -99,7 +100,6 @@ public void testGood() throws Exception {
99100
try {
100101
template.setReplyTimeout(30_000);
101102
ProducerRecord<Integer, String> record = new ProducerRecord<>(A_REQUEST, "foo");
102-
record.headers().add(new RecordHeader(KafkaHeaders.REPLY_TOPIC, A_REPLY.getBytes()));
103103
RequestReplyFuture<Integer, String, String> future = template.sendAndReceive(record);
104104
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
105105
ConsumerRecord<Integer, String> consumerRecord = future.get(30, TimeUnit.SECONDS);
@@ -127,19 +127,37 @@ public void testMultiListenerMessageReturn() throws Exception {
127127
}
128128
}
129129

130+
@Test
131+
public void testGoodDefaultReplyHeaders() throws Exception {
132+
ReplyingKafkaTemplate<Integer, String, String> template = createTemplate(
133+
new TopicPartitionInitialOffset(A_REPLY, 3));
134+
try {
135+
template.setReplyTimeout(30_000);
136+
ProducerRecord<Integer, String> record = new ProducerRecord<Integer, String>(A_REQUEST, "bar");
137+
RequestReplyFuture<Integer, String, String> future = template.sendAndReceive(record);
138+
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
139+
ConsumerRecord<Integer, String> consumerRecord = future.get(30, TimeUnit.SECONDS);
140+
assertThat(consumerRecord.value()).isEqualTo("BAR");
141+
assertThat(consumerRecord.partition()).isEqualTo(3);
142+
}
143+
finally {
144+
template.stop();
145+
}
146+
}
147+
130148
@Test
131149
public void testGoodSamePartition() throws Exception {
132150
ReplyingKafkaTemplate<Integer, String, String> template = createTemplate(A_REPLY);
133151
try {
134152
template.setReplyTimeout(30_000);
135-
ProducerRecord<Integer, String> record = new ProducerRecord<>(A_REQUEST, 2, null, "foo");
153+
ProducerRecord<Integer, String> record = new ProducerRecord<>(A_REQUEST, 2, null, "baz");
136154
record.headers().add(new RecordHeader(KafkaHeaders.REPLY_TOPIC, A_REPLY.getBytes()));
137155
record.headers()
138156
.add(new RecordHeader(KafkaHeaders.REPLY_PARTITION, new byte[] { 0, 0, 0, 2 }));
139157
RequestReplyFuture<Integer, String, String> future = template.sendAndReceive(record);
140158
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
141159
ConsumerRecord<Integer, String> consumerRecord = future.get(30, TimeUnit.SECONDS);
142-
assertThat(consumerRecord.value()).isEqualTo("FOO");
160+
assertThat(consumerRecord.value()).isEqualTo("BAZ");
143161
assertThat(consumerRecord.partition()).isEqualTo(2);
144162
}
145163
finally {
@@ -152,7 +170,7 @@ public void testTimeout() throws Exception {
152170
ReplyingKafkaTemplate<Integer, String, String> template = createTemplate(A_REPLY);
153171
try {
154172
template.setReplyTimeout(1);
155-
ProducerRecord<Integer, String> record = new ProducerRecord<>(A_REQUEST, "foo");
173+
ProducerRecord<Integer, String> record = new ProducerRecord<>(A_REQUEST, "fiz");
156174
record.headers().add(new RecordHeader(KafkaHeaders.REPLY_TOPIC, A_REPLY.getBytes()));
157175
RequestReplyFuture<Integer, String, String> future = template.sendAndReceive(record);
158176
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
@@ -214,13 +232,33 @@ public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
214232
containerProperties);
215233
container.setBeanName(this.testName.getMethodName());
216234
ReplyingKafkaTemplate<Integer, String, String> template = new ReplyingKafkaTemplate<>(this.config.pf(), container);
235+
template.setSharedReplyTopic(true);
217236
template.start();
218237
assertThat(latch.await(30, TimeUnit.SECONDS)).isTrue();
219238
assertThat(template.getAssignedReplyTopicPartitions()).hasSize(5);
220239
assertThat(template.getAssignedReplyTopicPartitions().iterator().next().topic()).isEqualTo(topic);
221240
return template;
222241
}
223242

243+
public ReplyingKafkaTemplate<Integer, String, String> createTemplate(TopicPartitionInitialOffset topic)
244+
throws Exception {
245+
246+
ContainerProperties containerProperties = new ContainerProperties(topic);
247+
Map<String, Object> consumerProps = KafkaTestUtils.consumerProps(this.testName.getMethodName(), "false",
248+
embeddedKafka);
249+
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
250+
DefaultKafkaConsumerFactory<Integer, String> cf = new DefaultKafkaConsumerFactory<>(consumerProps);
251+
KafkaMessageListenerContainer<Integer, String> container = new KafkaMessageListenerContainer<>(cf,
252+
containerProperties);
253+
container.setBeanName(this.testName.getMethodName());
254+
ReplyingKafkaTemplate<Integer, String, String> template = new ReplyingKafkaTemplate<>(this.config.pf(), container);
255+
template.setSharedReplyTopic(true);
256+
template.start();
257+
assertThat(template.getAssignedReplyTopicPartitions()).hasSize(1);
258+
assertThat(template.getAssignedReplyTopicPartitions().iterator().next().topic()).isEqualTo(topic.topic());
259+
return template;
260+
}
261+
224262
@Configuration
225263
@EnableKafka
226264
public static class Config {

src/reference/asciidoc/kafka.adoc

Lines changed: 19 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -371,7 +371,6 @@ public class KRequestingApplication {
371371
public ApplicationRunner runner(ReplyingKafkaTemplate<String, String, String> template) {
372372
return args -> {
373373
ProducerRecord<String, String> record = new ProducerRecord<>("kRequests", "foo");
374-
record.headers().add(new RecordHeader(KafkaHeaders.REPLY_TOPIC, "kReplies".getBytes()));
375374
RequestReplyFuture<String, String, String> replyFuture = template.sendAndReceive(record);
376375
SendResult<String, String> sendResult = replyFuture.getSendFuture().get();
377376
System.out.println("Sent ok: " + sendResult.getRecordMetadata());
@@ -384,12 +383,14 @@ public class KRequestingApplication {
384383
public ReplyingKafkaTemplate<String, String, String> kafkaTemplate(
385384
ProducerFactory<String, String> pf,
386385
KafkaMessageListenerContainer<String, String> replyContainer) {
386+
387387
return new ReplyingKafkaTemplate<>(pf, replyContainer);
388388
}
389389
390390
@Bean
391391
public KafkaMessageListenerContainer<String, String> replyContainer(
392392
ConsumerFactory<String, String> cf) {
393+
393394
ContainerProperties containerProperties = new ContainerProperties("kReplies");
394395
return new KafkaMessageListenerContainer<>(cf, containerProperties);
395396
}
@@ -407,7 +408,7 @@ public class KRequestingApplication {
407408
}
408409
----
409410

410-
In addition to the reply topic header set by user code, the template sets a header `KafkaHeaders.CORRELATION_ID` which must be echoed back by the server side.
411+
The template sets a header `KafkaHeaders.CORRELATION_ID` which must be echoed back by the server side.
411412

412413
In this case, simple `@KafkaListener` application responds:
413414

@@ -444,9 +445,23 @@ public class KReplyingApplication {
444445

445446
The `@KafkaListener` infrastructure echoes the correlation id and determines the reply topic.
446447

447-
See <<annotation-send-to>> for more information about sending replies; in this case we use the default header `KafKaHeaders.REPLY_TOPIC` to indicate which topic the reply goes to.
448+
See <<annotation-send-to>> for more information about sending replies; the template uses the default header `KafKaHeaders.REPLY_TOPIC` to indicate which topic the reply goes to.
449+
450+
Starting with version 2.2, the template will attempt to detect the reply topic/partition from the configured reply container.
451+
If the container is configured to listen to a single topic or a single `TopicPartitionInitialOffset`, it will be used to set the reply headers.
452+
If the container is configured otherwise, the user must set up the reply header(s); in this case, an INFO log is written during initialization.
453+
454+
[source, java]
455+
----
456+
record.headers().add(new RecordHeader(KafkaHeaders.REPLY_TOPIC, "kReplies".getBytes()));
457+
----
458+
459+
When configuring with a single reply `TopicPartitionInitialOffset`, you can use the same reply topic for multiple templates, as long as each instance listens on a different partition.
460+
When configuring with a single reply topic, each instance must use a different `group.id` - in this case, all instances will receive each reply, but only the instance that sent the request will find the correlation id.
461+
This may be useful for auto-scaling, but with the overhead of additional network traffic and the small cost of discarding each unwanted reply.
462+
When using this setting, it is recommended that you set the template's `sharedReplyTopic` to true, which will reduce the logging level of unexpected replies to DEBUG instead of the default ERROR.
448463

449-
IMPORTANT: If you have multiple client instances, each will need a dedicated reply topic for each instance.
464+
IMPORTANT: If you have multiple client instances, and you don't configure them as discussed in the paragraphe above, each instance will need a dedicated reply topic.
450465
An alternative is to set the `KafkaHeaders.REPLY_PARTITION` and use a dedicated partition for each instance; the `Header` contains a 4 byte int (Big-endian).
451466
The server must use this header to route the reply to the correct topic (`@KafkaListener` does this).
452467
In this case, though, the reply container must not use Kafka's group management feature and must be configured to listen on a fixed partition (using a `TopicPartitionInitialOffset` in its `ContainerProperties` constructor).

0 commit comments

Comments
 (0)