Skip to content

Commit 8a23e47

Browse files
garyrussellartembilan
authored andcommitted
GH-2348: Custom Correlation Consumer Side
Resolves #2348 The replying template supports a custom header for correlation for cases when the consumer side is not a Spring app and uses a different header. Support a custom header name on the consumer side, for cases where the client side is not Spring and uses a different header.
1 parent 9066598 commit 8a23e47

File tree

6 files changed

+125
-35
lines changed

6 files changed

+125
-35
lines changed

spring-kafka-docs/src/main/asciidoc/kafka.adoc

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -726,6 +726,9 @@ These header names are used by the `@KafkaListener` infrastructure to route the
726726
Starting with version 2.3, you can customize the header names - the template has 3 properties `correlationHeaderName`, `replyTopicHeaderName`, and `replyPartitionHeaderName`.
727727
This is useful if your server is not a Spring application (or does not use the `@KafkaListener`).
728728

729+
NOTE: Conversely, if the requesting application is not a spring application and puts correlation information in a different header, starting with version 3.0, you can configure a custom `correlationHeaderName` on the listener container factory and that header will be echoed back.
730+
Previously, the listener had to echo custom correlation headers.
731+
729732
[[exchanging-messages]]
730733
====== Request/Reply with `Message<?>` s
731734

spring-kafka-docs/src/main/asciidoc/whats-new.adoc

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,3 +52,9 @@ See <<kafka-template>>.
5252

5353
The futures returned by this class are now `CompletableFuture` s instead of `ListenableFuture` s.
5454
See <<replying-template>> and <<exchanging-messages>>.
55+
56+
[[x30-listener]]
57+
==== `@KafkaListener` Changes
58+
59+
You can now use a custom correlation header which will be echoed in any reply message.
60+
See the note at the end of <<replying-template>> for more information.

spring-kafka/src/main/java/org/springframework/kafka/config/AbstractKafkaListenerContainerFactory.java

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,8 @@ public abstract class AbstractKafkaListenerContainerFactory<C extends AbstractMe
108108

109109
private ContainerCustomizer<K, V, C> containerCustomizer;
110110

111+
private String correlationHeaderName;
112+
111113
@Override
112114
public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
113115
this.applicationContext = applicationContext;
@@ -321,6 +323,17 @@ public void setContainerCustomizer(ContainerCustomizer<K, V, C> containerCustomi
321323
this.containerCustomizer = containerCustomizer;
322324
}
323325

326+
/**
327+
* Set a custom header name for the correlation id. Default
328+
* {@link org.springframework.kafka.support.KafkaHeaders#CORRELATION_ID}. This header
329+
* will be echoed back in any reply message.
330+
* @param correlationHeaderName the header name.
331+
* @since 3.0
332+
*/
333+
public void setCorrelationHeaderName(String correlationHeaderName) {
334+
this.correlationHeaderName = correlationHeaderName;
335+
}
336+
324337
@SuppressWarnings("deprecation")
325338
@Override
326339
public void afterPropertiesSet() {
@@ -363,7 +376,8 @@ private void configureEndpoint(AbstractKafkaListenerEndpoint<K, V> aklEndpoint)
363376
.acceptIfNotNull(this.ackDiscarded, aklEndpoint::setAckDiscarded)
364377
.acceptIfNotNull(this.replyTemplate, aklEndpoint::setReplyTemplate)
365378
.acceptIfNotNull(this.replyHeadersConfigurer, aklEndpoint::setReplyHeadersConfigurer)
366-
.acceptIfNotNull(this.batchToRecordAdapter, aklEndpoint::setBatchToRecordAdapter);
379+
.acceptIfNotNull(this.batchToRecordAdapter, aklEndpoint::setBatchToRecordAdapter)
380+
.acceptIfNotNull(this.correlationHeaderName, aklEndpoint::setCorrelationHeaderName);
367381
if (aklEndpoint.getBatchListener() == null) {
368382
JavaUtils.INSTANCE
369383
.acceptIfNotNull(this.batchListener, aklEndpoint::setBatchListener);

spring-kafka/src/main/java/org/springframework/kafka/config/AbstractKafkaListenerEndpoint.java

Lines changed: 16 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@
4545
import org.springframework.kafka.listener.adapter.MessagingMessageListenerAdapter;
4646
import org.springframework.kafka.listener.adapter.RecordFilterStrategy;
4747
import org.springframework.kafka.listener.adapter.ReplyHeadersConfigurer;
48+
import org.springframework.kafka.support.JavaUtils;
4849
import org.springframework.kafka.support.TopicPartitionOffset;
4950
import org.springframework.kafka.support.converter.MessageConverter;
5051
import org.springframework.lang.Nullable;
@@ -117,6 +118,8 @@ public abstract class AbstractKafkaListenerEndpoint<K, V>
117118

118119
private byte[] listenerInfo;
119120

121+
private String correlationHeaderName;
122+
120123
@Override
121124
public void setBeanFactory(BeanFactory beanFactory) throws BeansException {
122125
this.beanFactory = beanFactory;
@@ -445,6 +448,16 @@ public void setBatchToRecordAdapter(BatchToRecordAdapter<K, V> batchToRecordAdap
445448
this.batchToRecordAdapter = batchToRecordAdapter;
446449
}
447450

451+
/**
452+
* Set a custom header name for the correlation id. Default
453+
* {@link org.springframework.kafka.support.KafkaHeaders#CORRELATION_ID}. This header
454+
* will be echoed back in any reply message.
455+
* @param correlationHeaderName the header name.
456+
* @since 3.0
457+
*/
458+
public void setCorrelationHeaderName(String correlationHeaderName) {
459+
this.correlationHeaderName = correlationHeaderName;
460+
}
448461

449462
@Override
450463
public void afterPropertiesSet() {
@@ -485,9 +498,9 @@ private void setupMessageListener(MessageListenerContainer container,
485498
@Nullable MessageConverter messageConverter) {
486499

487500
MessagingMessageListenerAdapter<K, V> adapter = createMessageListener(container, messageConverter);
488-
if (this.replyHeadersConfigurer != null) {
489-
adapter.setReplyHeadersConfigurer(this.replyHeadersConfigurer);
490-
}
501+
JavaUtils.INSTANCE
502+
.acceptIfNotNull(this.replyHeadersConfigurer, adapter::setReplyHeadersConfigurer)
503+
.acceptIfNotNull(this.correlationHeaderName, adapter::setCorrelationHeaderName);
491504
adapter.setSplitIterables(this.splitIterables);
492505
Object messageListener = adapter;
493506
boolean isBatchListener = isBatchListener();

spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/MessagingMessageListenerAdapter.java

Lines changed: 27 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -128,11 +128,30 @@ public abstract class MessagingMessageListenerAdapter<K, V> implements ConsumerS
128128

129129
private boolean splitIterables = true;
130130

131+
private String correlationHeaderName = KafkaHeaders.CORRELATION_ID;
132+
133+
/**
134+
* Create an instance with the provided bean and method.
135+
* @param bean the bean.
136+
* @param method the method.
137+
*/
131138
public MessagingMessageListenerAdapter(Object bean, Method method) {
132139
this.bean = bean;
133140
this.inferredType = determineInferredType(method); // NOSONAR = intentionally not final
134141
}
135142

143+
/**
144+
* Set a custom header name for the correlation id. Default
145+
* {@link KafkaHeaders#CORRELATION_ID}. This header will be echoed back in any reply
146+
* message.
147+
* @param correlationHeaderName the header name.
148+
* @since 3.0
149+
*/
150+
public void setCorrelationHeaderName(String correlationHeaderName) {
151+
Assert.notNull(correlationHeaderName, "'correlationHeaderName' cannot be null");
152+
this.correlationHeaderName = correlationHeaderName;
153+
}
154+
136155
/**
137156
* Set the MessageConverter.
138157
* @param messageConverter the converter.
@@ -478,7 +497,7 @@ private Message<?> checkHeaders(Object result, String topic, Object source) { //
478497
MessageHeaders headers = reply.getHeaders();
479498
boolean needsTopic = headers.get(KafkaHeaders.TOPIC) == null;
480499
boolean sourceIsMessage = source instanceof Message;
481-
boolean needsCorrelation = headers.get(KafkaHeaders.CORRELATION_ID) == null && sourceIsMessage;
500+
boolean needsCorrelation = headers.get(this.correlationHeaderName) == null && sourceIsMessage;
482501
boolean needsPartition = headers.get(KafkaHeaders.PARTITION) == null && sourceIsMessage
483502
&& getReplyPartition((Message<?>) source) != null;
484503
if (needsTopic || needsCorrelation || needsPartition) {
@@ -487,8 +506,8 @@ private Message<?> checkHeaders(Object result, String topic, Object source) { //
487506
builder.setHeader(KafkaHeaders.TOPIC, topic);
488507
}
489508
if (needsCorrelation && sourceIsMessage) {
490-
builder.setHeader(KafkaHeaders.CORRELATION_ID,
491-
((Message<?>) source).getHeaders().get(KafkaHeaders.CORRELATION_ID));
509+
builder.setHeader(this.correlationHeaderName,
510+
((Message<?>) source).getHeaders().get(this.correlationHeaderName));
492511
}
493512
if (sourceIsMessage && reply.getHeaders().get(KafkaHeaders.REPLY_PARTITION) == null) {
494513
setPartition(builder, (Message<?>) source);
@@ -503,8 +522,8 @@ private void sendSingleResult(Object result, String topic, @Nullable Object sour
503522
byte[] correlationId = null;
504523
boolean sourceIsMessage = source instanceof Message;
505524
if (sourceIsMessage
506-
&& ((Message<?>) source).getHeaders().get(KafkaHeaders.CORRELATION_ID) != null) {
507-
correlationId = ((Message<?>) source).getHeaders().get(KafkaHeaders.CORRELATION_ID, byte[].class);
525+
&& ((Message<?>) source).getHeaders().get(this.correlationHeaderName) != null) {
526+
correlationId = ((Message<?>) source).getHeaders().get(this.correlationHeaderName, byte[].class);
508527
}
509528
if (sourceIsMessage) {
510529
sendReplyForMessageSource(result, topic, source, correlationId);
@@ -515,15 +534,15 @@ private void sendSingleResult(Object result, String topic, @Nullable Object sour
515534
}
516535

517536
@SuppressWarnings("unchecked")
518-
private void sendReplyForMessageSource(Object result, String topic, Object source, byte[] correlationId) {
537+
private void sendReplyForMessageSource(Object result, String topic, Object source, @Nullable byte[] correlationId) {
519538
MessageBuilder<Object> builder = MessageBuilder.withPayload(result)
520539
.setHeader(KafkaHeaders.TOPIC, topic);
521540
if (this.replyHeadersConfigurer != null) {
522541
Map<String, Object> headersToCopy = ((Message<?>) source).getHeaders().entrySet().stream()
523542
.filter(e -> {
524543
String key = e.getKey();
525544
return !key.equals(MessageHeaders.ID) && !key.equals(MessageHeaders.TIMESTAMP)
526-
&& !key.equals(KafkaHeaders.CORRELATION_ID)
545+
&& !key.equals(this.correlationHeaderName)
527546
&& !key.startsWith(KafkaHeaders.RECEIVED);
528547
})
529548
.filter(e -> this.replyHeadersConfigurer.shouldCopy(e.getKey(), e.getValue()))
@@ -537,7 +556,7 @@ private void sendReplyForMessageSource(Object result, String topic, Object sourc
537556
}
538557
}
539558
if (correlationId != null) {
540-
builder.setHeader(KafkaHeaders.CORRELATION_ID, correlationId);
559+
builder.setHeader(this.correlationHeaderName, correlationId);
541560
}
542561
setPartition(builder, ((Message<?>) source));
543562
this.replyTemplate.send(builder.build());

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

Lines changed: 58 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -467,7 +467,7 @@ public void testGoodWithSimpleMapper() throws Exception {
467467
@Test
468468
public void testAggregateNormal() throws Exception {
469469
AggregatingReplyingKafkaTemplate<Integer, String, String> template = aggregatingTemplate(
470-
new TopicPartitionOffset(D_REPLY, 0), 2, new AtomicInteger());
470+
new TopicPartitionOffset(D_REPLY, 0), 3, new AtomicInteger());
471471
try {
472472
template.setCorrelationHeaderName("customCorrelation");
473473
template.setDefaultReplyTimeout(Duration.ofSeconds(30));
@@ -477,13 +477,17 @@ public void testAggregateNormal() throws Exception {
477477
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
478478
ConsumerRecord<Integer, Collection<ConsumerRecord<Integer, String>>> consumerRecord =
479479
future.get(30, TimeUnit.SECONDS);
480-
assertThat(consumerRecord.value().size()).isEqualTo(2);
480+
assertThat(consumerRecord.value().size()).isEqualTo(3);
481481
Iterator<ConsumerRecord<Integer, String>> iterator = consumerRecord.value().iterator();
482482
String value1 = iterator.next().value();
483-
assertThat(value1).isIn("fOO", "FOO");
483+
assertThat(value1).isIn("fOO", "FOO", "Foo");
484484
String value2 = iterator.next().value();
485-
assertThat(value2).isIn("fOO", "FOO");
485+
assertThat(value2).isIn("fOO", "FOO", "Foo");
486486
assertThat(value2).isNotSameAs(value1);
487+
String value3 = iterator.next().value();
488+
assertThat(value3).isIn("fOO", "FOO", "Foo");
489+
assertThat(value3).isNotSameAs(value1);
490+
assertThat(value3).isNotSameAs(value2);
487491
assertThat(consumerRecord.topic()).isEqualTo(AggregatingReplyingKafkaTemplate.AGGREGATED_RESULTS_TOPIC);
488492
}
489493
finally {
@@ -495,7 +499,7 @@ public void testAggregateNormal() throws Exception {
495499
@Test
496500
public void testAggregateNormalStringCorrelation() throws Exception {
497501
AggregatingReplyingKafkaTemplate<Integer, String, String> template = aggregatingTemplate(
498-
new TopicPartitionOffset(D_REPLY, 0), 2, new AtomicInteger());
502+
new TopicPartitionOffset(D_REPLY, 0), 3, new AtomicInteger());
499503
try {
500504
template.setCorrelationHeaderName("customCorrelation");
501505
template.setBinaryCorrelation(false);
@@ -506,13 +510,17 @@ public void testAggregateNormalStringCorrelation() throws Exception {
506510
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
507511
ConsumerRecord<Integer, Collection<ConsumerRecord<Integer, String>>> consumerRecord =
508512
future.get(30, TimeUnit.SECONDS);
509-
assertThat(consumerRecord.value().size()).isEqualTo(2);
513+
assertThat(consumerRecord.value().size()).isEqualTo(3);
510514
Iterator<ConsumerRecord<Integer, String>> iterator = consumerRecord.value().iterator();
511515
String value1 = iterator.next().value();
512-
assertThat(value1).isIn("fOO", "FOO");
516+
assertThat(value1).isIn("fOO", "FOO", "Foo");
513517
String value2 = iterator.next().value();
514-
assertThat(value2).isIn("fOO", "FOO");
518+
assertThat(value2).isIn("fOO", "FOO", "Foo");
515519
assertThat(value2).isNotSameAs(value1);
520+
String value3 = iterator.next().value();
521+
assertThat(value3).isIn("fOO", "FOO", "Foo");
522+
assertThat(value3).isNotSameAs(value1);
523+
assertThat(value3).isNotSameAs(value2);
516524
assertThat(consumerRecord.topic()).isEqualTo(AggregatingReplyingKafkaTemplate.AGGREGATED_RESULTS_TOPIC);
517525
}
518526
finally {
@@ -526,7 +534,7 @@ public void testAggregateNormalStringCorrelation() throws Exception {
526534
@Disabled("time sensitive")
527535
public void testAggregateTimeout() throws Exception {
528536
AggregatingReplyingKafkaTemplate<Integer, String, String> template = aggregatingTemplate(
529-
new TopicPartitionOffset(E_REPLY, 0), 3, new AtomicInteger());
537+
new TopicPartitionOffset(E_REPLY, 0), 4, new AtomicInteger());
530538
try {
531539
template.setDefaultReplyTimeout(Duration.ofSeconds(5));
532540
template.setCorrelationHeaderName("customCorrelation");
@@ -561,7 +569,7 @@ public void testAggregateTimeout() throws Exception {
561569
public void testAggregateTimeoutPartial() throws Exception {
562570
AtomicInteger releaseCount = new AtomicInteger();
563571
AggregatingReplyingKafkaTemplate<Integer, String, String> template = aggregatingTemplate(
564-
new TopicPartitionOffset(F_REPLY, 0), 3, releaseCount);
572+
new TopicPartitionOffset(F_REPLY, 0), 4, releaseCount);
565573
template.setReturnPartialOnTimeout(true);
566574
try {
567575
template.setDefaultReplyTimeout(Duration.ofSeconds(5));
@@ -572,16 +580,20 @@ public void testAggregateTimeoutPartial() throws Exception {
572580
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
573581
ConsumerRecord<Integer, Collection<ConsumerRecord<Integer, String>>> consumerRecord =
574582
future.get(30, TimeUnit.SECONDS);
575-
assertThat(consumerRecord.value().size()).isEqualTo(2);
583+
assertThat(consumerRecord.value().size()).isEqualTo(3);
576584
Iterator<ConsumerRecord<Integer, String>> iterator = consumerRecord.value().iterator();
577585
String value1 = iterator.next().value();
578-
assertThat(value1).isIn("fOO", "FOO");
586+
assertThat(value1).isIn("fOO", "FOO", "Foo");
579587
String value2 = iterator.next().value();
580-
assertThat(value2).isIn("fOO", "FOO");
588+
assertThat(value2).isIn("fOO", "FOO", "Foo");
581589
assertThat(value2).isNotSameAs(value1);
590+
String value3 = iterator.next().value();
591+
assertThat(value3).isIn("fOO", "FOO", "Foo");
592+
assertThat(value3).isNotSameAs(value1);
593+
assertThat(value3).isNotSameAs(value2);
582594
assertThat(consumerRecord.topic())
583595
.isEqualTo(AggregatingReplyingKafkaTemplate.PARTIAL_RESULTS_AFTER_TIMEOUT_TOPIC);
584-
assertThat(releaseCount.get()).isEqualTo(3);
596+
assertThat(releaseCount.get()).isEqualTo(4);
585597
}
586598
finally {
587599
template.stop();
@@ -593,7 +605,7 @@ public void testAggregateTimeoutPartial() throws Exception {
593605
public void testAggregateTimeoutPartialStringCorrelation() throws Exception {
594606
AtomicInteger releaseCount = new AtomicInteger();
595607
AggregatingReplyingKafkaTemplate<Integer, String, String> template = aggregatingTemplate(
596-
new TopicPartitionOffset(F_REPLY, 0), 3, releaseCount);
608+
new TopicPartitionOffset(F_REPLY, 0), 4, releaseCount);
597609
template.setReturnPartialOnTimeout(true);
598610
template.setBinaryCorrelation(false);
599611
try {
@@ -605,16 +617,20 @@ public void testAggregateTimeoutPartialStringCorrelation() throws Exception {
605617
future.getSendFuture().get(10, TimeUnit.SECONDS); // send ok
606618
ConsumerRecord<Integer, Collection<ConsumerRecord<Integer, String>>> consumerRecord =
607619
future.get(30, TimeUnit.SECONDS);
608-
assertThat(consumerRecord.value().size()).isEqualTo(2);
620+
assertThat(consumerRecord.value().size()).isEqualTo(3);
609621
Iterator<ConsumerRecord<Integer, String>> iterator = consumerRecord.value().iterator();
610622
String value1 = iterator.next().value();
611-
assertThat(value1).isIn("fOO", "FOO");
623+
assertThat(value1).isIn("fOO", "FOO", "Foo");
612624
String value2 = iterator.next().value();
613-
assertThat(value2).isIn("fOO", "FOO");
625+
assertThat(value2).isIn("fOO", "FOO", "Foo");
614626
assertThat(value2).isNotSameAs(value1);
627+
String value3 = iterator.next().value();
628+
assertThat(value3).isIn("fOO", "FOO", "Foo");
629+
assertThat(value3).isNotSameAs(value1);
630+
assertThat(value3).isNotSameAs(value2);
615631
assertThat(consumerRecord.topic())
616632
.isEqualTo(AggregatingReplyingKafkaTemplate.PARTIAL_RESULTS_AFTER_TIMEOUT_TOPIC);
617-
assertThat(releaseCount.get()).isEqualTo(3);
633+
assertThat(releaseCount.get()).isEqualTo(4);
618634
}
619635
finally {
620636
template.stop();
@@ -865,6 +881,17 @@ public ConcurrentKafkaListenerContainerFactory<Integer, String> kafkaListenerCon
865881
return factory;
866882
}
867883

884+
@Bean
885+
public ConcurrentKafkaListenerContainerFactory<Integer, String> customListenerContainerFactory() {
886+
ConcurrentKafkaListenerContainerFactory<Integer, String> factory =
887+
new ConcurrentKafkaListenerContainerFactory<>();
888+
factory.setConsumerFactory(cf());
889+
factory.setReplyTemplate(template());
890+
factory.setCorrelationHeaderName("customCorrelation");
891+
factory.setMissingTopicsFatal(false);
892+
return factory;
893+
}
894+
868895
@Bean
869896
public ConcurrentKafkaListenerContainerFactory<Integer, String> simpleMapperFactory() {
870897
ConcurrentKafkaListenerContainerFactory<Integer, String> factory =
@@ -921,22 +948,30 @@ public HandlerReturn handlerReturn() {
921948
return new HandlerReturn();
922949
}
923950

924-
@KafkaListener(id = "def1", topics = { D_REQUEST, E_REQUEST, F_REQUEST })
951+
@KafkaListener(id = "def1", topics = { D_REQUEST, E_REQUEST, F_REQUEST },
952+
containerFactory = "customListenerContainerFactory")
925953
@SendTo // default REPLY_TOPIC header
926954
public Message<String> dListener1(String in, @Header("customCorrelation") byte[] correlation) {
927955
return MessageBuilder.withPayload(in.toUpperCase())
928956
.setHeader("customCorrelation", correlation)
929957
.build();
930958
}
931959

932-
@KafkaListener(id = "def2", topics = { D_REQUEST, E_REQUEST, F_REQUEST })
960+
@KafkaListener(id = "def2", topics = { D_REQUEST, E_REQUEST, F_REQUEST },
961+
containerFactory = "customListenerContainerFactory")
933962
@SendTo // default REPLY_TOPIC header
934-
public Message<String> dListener2(String in, @Header("customCorrelation") byte[] correlation) {
963+
public Message<String> dListener2(String in) {
935964
return MessageBuilder.withPayload(in.substring(0, 1) + in.substring(1).toUpperCase())
936-
.setHeader("customCorrelation", correlation)
937965
.build();
938966
}
939967

968+
@KafkaListener(id = "def3", topics = { D_REQUEST, E_REQUEST, F_REQUEST },
969+
containerFactory = "customListenerContainerFactory")
970+
@SendTo // default REPLY_TOPIC header
971+
public String dListener3(String in) {
972+
return in.substring(0, 1).toUpperCase() + in.substring(1);
973+
}
974+
940975
@KafkaListener(id = G_REQUEST, topics = G_REQUEST)
941976
public void gListener(Message<String> in) {
942977
String replyTopic = new String(in.getHeaders().get("custom.reply.to", byte[].class));

0 commit comments

Comments
 (0)