Skip to content

Commit eb3e8f6

Browse files
committed
spring-projectsGH-3696: DeserializationEx support for KafkaMS
Fixes spring-projects#3696 * Add an internal logic into `KafkaMessageSource` to react properly for the `ErrorHandlingDeserializer` configuration and re-throw `DeserializationException`
1 parent bd013e0 commit eb3e8f6

File tree

4 files changed

+178
-3
lines changed

4 files changed

+178
-3
lines changed

spring-integration-kafka/src/main/java/org/springframework/integration/kafka/inbound/KafkaMessageSource.java

Lines changed: 86 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import java.util.LinkedHashSet;
2727
import java.util.List;
2828
import java.util.Map;
29+
import java.util.Properties;
2930
import java.util.Set;
3031
import java.util.TreeSet;
3132
import java.util.concurrent.ConcurrentHashMap;
@@ -46,6 +47,7 @@
4647
import org.apache.kafka.common.TopicPartition;
4748
import org.apache.kafka.common.errors.WakeupException;
4849

50+
import org.springframework.beans.factory.BeanClassLoaderAware;
4951
import org.springframework.core.log.LogAccessor;
5052
import org.springframework.integration.IntegrationMessageHeaderAccessor;
5153
import org.springframework.integration.acks.AcknowledgmentCallback;
@@ -58,6 +60,7 @@
5860
import org.springframework.kafka.core.DefaultKafkaConsumerFactory;
5961
import org.springframework.kafka.listener.ConsumerAwareRebalanceListener;
6062
import org.springframework.kafka.listener.ConsumerProperties;
63+
import org.springframework.kafka.listener.ListenerUtils;
6164
import org.springframework.kafka.listener.LoggingCommitCallback;
6265
import org.springframework.kafka.support.Acknowledgment;
6366
import org.springframework.kafka.support.DefaultKafkaHeaderMapper;
@@ -69,9 +72,13 @@
6972
import org.springframework.kafka.support.converter.KafkaMessageHeaders;
7073
import org.springframework.kafka.support.converter.MessagingMessageConverter;
7174
import org.springframework.kafka.support.converter.RecordMessageConverter;
75+
import org.springframework.kafka.support.serializer.DeserializationException;
76+
import org.springframework.kafka.support.serializer.ErrorHandlingDeserializer;
77+
import org.springframework.kafka.support.serializer.SerializationUtils;
7278
import org.springframework.lang.Nullable;
7379
import org.springframework.messaging.Message;
7480
import org.springframework.util.Assert;
81+
import org.springframework.util.ClassUtils;
7582
import org.springframework.util.ObjectUtils;
7683
import org.springframework.util.StringUtils;
7784

@@ -102,7 +109,8 @@
102109
* @since 5.4
103110
*
104111
*/
105-
public class KafkaMessageSource<K, V> extends AbstractMessageSource<Object> implements Pausable {
112+
public class KafkaMessageSource<K, V> extends AbstractMessageSource<Object>
113+
implements Pausable, BeanClassLoaderAware {
106114

107115
private static final long MIN_ASSIGN_TIMEOUT = 2000L;
108116

@@ -146,6 +154,10 @@ public class KafkaMessageSource<K, V> extends AbstractMessageSource<Object> impl
146154

147155
private Duration closeTimeout = Duration.ofSeconds(DEFAULT_CLOSE_TIMEOUT);
148156

157+
private boolean checkNullKeyForExceptions;
158+
159+
private boolean checkNullValueForExceptions;
160+
149161
private volatile Consumer<K, V> consumer;
150162

151163
private volatile boolean pausing;
@@ -158,6 +170,8 @@ public class KafkaMessageSource<K, V> extends AbstractMessageSource<Object> impl
158170

159171
public volatile boolean newAssignment; // NOSONAR - direct access from inner
160172

173+
private ClassLoader classLoader;
174+
161175
/**
162176
* Construct an instance with the supplied parameters. Fetching multiple
163177
* records per poll will be disabled.
@@ -257,11 +271,68 @@ public Collection<TopicPartition> getAssignedPartitions() {
257271
return Collections.unmodifiableCollection(this.assignedPartitions);
258272
}
259273

274+
@Override
275+
public void setBeanClassLoader(ClassLoader classLoader) {
276+
this.classLoader = classLoader;
277+
}
278+
260279
@Override
261280
protected void onInit() {
262281
if (!StringUtils.hasText(this.consumerProperties.getClientId())) {
263282
this.consumerProperties.setClientId(getComponentName());
264283
}
284+
285+
Map<String, Object> props = this.consumerFactory.getConfigurationProperties();
286+
Properties kafkaConsumerProperties = this.consumerProperties.getKafkaConsumerProperties();
287+
this.checkNullKeyForExceptions =
288+
this.consumerProperties.isCheckDeserExWhenKeyNull() ||
289+
checkDeserializer(findDeserializerClass(props, kafkaConsumerProperties, false));
290+
this.checkNullValueForExceptions =
291+
this.consumerProperties.isCheckDeserExWhenValueNull() ||
292+
checkDeserializer(findDeserializerClass(props, kafkaConsumerProperties, true));
293+
}
294+
295+
@Nullable
296+
private Object findDeserializerClass(Map<String, Object> props, Properties consumerOverrides, boolean isValue) {
297+
Object configuredDeserializer =
298+
isValue
299+
? this.consumerFactory.getValueDeserializer()
300+
: this.consumerFactory.getKeyDeserializer();
301+
if (configuredDeserializer == null) {
302+
Object deser = consumerOverrides.get(
303+
isValue
304+
? ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG
305+
: ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
306+
if (deser == null) {
307+
deser = props.get(
308+
isValue
309+
? ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG
310+
: ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
311+
}
312+
return deser;
313+
}
314+
else {
315+
return configuredDeserializer.getClass();
316+
}
317+
}
318+
319+
private boolean checkDeserializer(@Nullable Object deser) {
320+
Class<?> deserializer = null;
321+
if (deser instanceof Class<?> deserClass) {
322+
deserializer = deserClass;
323+
}
324+
else if (deser instanceof String str) {
325+
try {
326+
deserializer = ClassUtils.forName(str, this.classLoader);
327+
}
328+
catch (ClassNotFoundException | LinkageError e) {
329+
throw new IllegalStateException(e);
330+
}
331+
}
332+
else if (deser != null) {
333+
throw new IllegalStateException("Deserializer must be a class or class name, not a " + deser.getClass());
334+
}
335+
return deserializer != null && ErrorHandlingDeserializer.class.isAssignableFrom(deserializer);
265336
}
266337

267338
/**
@@ -609,6 +680,13 @@ record = this.recordsIterator.next();
609680
}
610681

611682
private Object recordToMessage(ConsumerRecord<K, V> record) {
683+
if (record.value() == null && this.checkNullValueForExceptions) {
684+
checkDeserializationException(record, SerializationUtils.VALUE_DESERIALIZER_EXCEPTION_HEADER);
685+
}
686+
if (record.key() == null && this.checkNullKeyForExceptions) {
687+
checkDeserializationException(record, SerializationUtils.KEY_DESERIALIZER_EXCEPTION_HEADER);
688+
}
689+
612690
TopicPartition topicPartition = new TopicPartition(record.topic(), record.partition());
613691
KafkaAckInfo<K, V> ackInfo = new KafkaAckInfoImpl(record, topicPartition);
614692
AcknowledgmentCallback ackCallback = this.ackCallbackFactory.createCallback(ackInfo);
@@ -639,6 +717,13 @@ private Object recordToMessage(ConsumerRecord<K, V> record) {
639717
}
640718
}
641719

720+
private void checkDeserializationException(ConsumerRecord<K, V> cRecord, String headerName) {
721+
DeserializationException exception = ListenerUtils.getExceptionFromHeader(cRecord, headerName, this.logger);
722+
if (exception != null) {
723+
throw exception;
724+
}
725+
}
726+
642727
@Override
643728
public void destroy() {
644729
this.lock.lock();

spring-integration-kafka/src/test/java/org/springframework/integration/kafka/inbound/MessageSourceIntegrationTests.java

Lines changed: 79 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,8 @@
2424
import org.apache.kafka.clients.consumer.ConsumerConfig;
2525
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
2626
import org.apache.kafka.common.TopicPartition;
27+
import org.apache.kafka.common.serialization.Deserializer;
28+
import org.junit.jupiter.api.BeforeAll;
2729
import org.junit.jupiter.api.Test;
2830

2931
import org.springframework.integration.channel.NullChannel;
@@ -32,11 +34,17 @@
3234
import org.springframework.kafka.core.DefaultKafkaProducerFactory;
3335
import org.springframework.kafka.core.KafkaTemplate;
3436
import org.springframework.kafka.listener.ConsumerProperties;
37+
import org.springframework.kafka.support.serializer.DeserializationException;
38+
import org.springframework.kafka.support.serializer.ErrorHandlingDeserializer;
3539
import org.springframework.kafka.test.utils.KafkaTestUtils;
3640
import org.springframework.messaging.Message;
3741
import org.springframework.messaging.support.GenericMessage;
42+
import org.springframework.util.ClassUtils;
3843

3944
import static org.assertj.core.api.Assertions.assertThat;
45+
import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
46+
import static org.awaitility.Awaitility.await;
47+
import static org.mockito.Mockito.mock;
4048

4149
/**
4250
* @author Gary Russell
@@ -50,9 +58,17 @@ class MessageSourceIntegrationTests {
5058

5159
static final String TOPIC1 = "MessageSourceIntegrationTests1";
5260

61+
static final String TOPIC2 = "MessageSourceIntegrationTests2";
62+
63+
static String brokers;
64+
65+
@BeforeAll
66+
static void setup() {
67+
brokers = System.getProperty("spring.global.embedded.kafka.brokers");
68+
}
69+
5370
@Test
5471
void testSource() throws Exception {
55-
String brokers = System.getProperty("spring.global.embedded.kafka.brokers");
5672
Map<String, Object> consumerProps = KafkaTestUtils.consumerProps(brokers, "testSource", "false");
5773
consumerProps.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 2);
5874
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
@@ -122,4 +138,66 @@ public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
122138
template.destroy();
123139
}
124140

141+
@Test
142+
void deserializationErrorIsThrownFromSource() throws Exception {
143+
Map<String, Object> consumerProps = KafkaTestUtils.consumerProps(brokers, "testErrorChannelSource", "false");
144+
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
145+
consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ErrorHandlingDeserializer.class);
146+
consumerProps.put(ErrorHandlingDeserializer.VALUE_DESERIALIZER_CLASS, FailingDeserializer.class);
147+
148+
DefaultKafkaConsumerFactory<Integer, String> consumerFactory = new DefaultKafkaConsumerFactory<>(consumerProps);
149+
ConsumerProperties consumerProperties = new ConsumerProperties(TOPIC2);
150+
CountDownLatch assigned = new CountDownLatch(1);
151+
consumerProperties.setConsumerRebalanceListener(
152+
new ConsumerRebalanceListener() {
153+
154+
@Override
155+
public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
156+
}
157+
158+
@Override
159+
public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
160+
assigned.countDown();
161+
}
162+
163+
});
164+
165+
consumerProperties.setPollTimeout(10);
166+
167+
KafkaMessageSource<Integer, String> source = new KafkaMessageSource<>(consumerFactory, consumerProperties);
168+
source.setBeanClassLoader(ClassUtils.getDefaultClassLoader());
169+
source.setBeanFactory(mock());
170+
source.afterPropertiesSet();
171+
source.start();
172+
173+
Map<String, Object> producerProps = KafkaTestUtils.producerProps(brokers);
174+
DefaultKafkaProducerFactory<Object, Object> producerFactory = new DefaultKafkaProducerFactory<>(producerProps);
175+
KafkaTemplate<Object, Object> template = new KafkaTemplate<>(producerFactory);
176+
177+
String testData = "test data";
178+
template.send(TOPIC2, testData);
179+
180+
source.receive(); // Trigger Kafka Consumer creation and poll()
181+
assertThat(assigned.await(10, TimeUnit.SECONDS)).isTrue();
182+
183+
await().untilAsserted(() ->
184+
assertThatExceptionOfType(DeserializationException.class)
185+
.isThrownBy(source::receive)
186+
.hasFieldOrPropertyWithValue("data", testData.getBytes())
187+
.withMessage("failed to deserialize")
188+
.withStackTraceContaining("failed deserialization"));
189+
190+
source.destroy();
191+
template.destroy();
192+
}
193+
194+
public static class FailingDeserializer implements Deserializer<String> {
195+
196+
@Override
197+
public String deserialize(String topic, byte[] data) {
198+
throw new RuntimeException("failed deserialization");
199+
}
200+
201+
}
202+
125203
}

src/reference/asciidoc/kafka.adoc

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -436,6 +436,11 @@ If you set `allowMultiFetch` to `true` you must process all the retrieved record
436436

437437
Messages emitted by this adapter contain a header `kafka_remainingRecords` with a count of records remaining from the previous poll.
438438

439+
Starting with version `6.2`, the `KafkaMessageSource` supports an `ErrorHandlingDeserializer` provided in the consumer properties.
440+
A `DeserializationException` is extracted from record headers and thrown to the called.
441+
With a `SourcePollingChannelAdapter` this exception is wrapped into an `ErrorMessage` and published to its `errorChannel`.
442+
See https://docs.spring.io/spring-kafka/reference/html/#error-handling-deserializer[`ErrorHandlingDeserializer`] documentation for more information.
443+
439444
[[kafka-outbound-gateway]]
440445
=== Outbound Gateway
441446

@@ -448,7 +453,7 @@ It is suggested that you add a `ConsumerRebalanceListener` to the template's rep
448453

449454
The `KafkaProducerMessageHandler` `sendTimeoutExpression` default is `delivery.timeout.ms` Kafka producer property `+ 5000` so that the actual Kafka error after a timeout is propagated to the application, instead of a timeout generated by this framework.
450455
This has been changed for consistency because you may get unexpected behavior (Spring may time out the `send()`, while it is actually, eventually, successful).
451-
IMPORTANT: That timeout is 120 seconds by default so you may wish to reduce it to get more timely failures.
456+
IMPORTANT: That timeout is 120 seconds by default, so you may wish to reduce it to get more timely failures.
452457

453458
[[kafka-outbound-gateway-configuration]]
454459
==== Configuration

src/reference/asciidoc/whats-new.adoc

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,3 +38,10 @@ See, for example, `transformWith()`, `splitWith()` in <<./dsl.adoc#java-dsl, Jav
3838
- For the server and client WebSocket containers, the send buffer overflow strategy is now configurable in `IntegrationWebSocketContainer` and in XML via `send-buffer-overflow-strategy`.
3939
This strategy determines the behavior when a session's outbound message buffer has reached the configured limit.
4040
See <<./web-sockets.adoc#websocket-client-container-attributes, WebSockets Support>> for more information.
41+
42+
43+
[[x6.2-kafka]]
44+
=== Apache Kafka Support Changes
45+
46+
The `KafkaMessageSource` now extracts an `ErrorHandlingDeserializer` configuration from the consumer properties and re-throws `DeserializationException` extracted from failed record headers.
47+
See <<./kafka.adoc#kafka-inbound-pollable, Kafka Inbound Channel Adapter>> for more information.

0 commit comments

Comments
 (0)