Skip to content

GH-4328: Expose native Kafka Streams DLQ configuration#4360

Open
loicgreffier wants to merge 1 commit intospring-projects:mainfrom
loicgreffier:GH-4328
Open

GH-4328: Expose native Kafka Streams DLQ configuration#4360
loicgreffier wants to merge 1 commit intospring-projects:mainfrom
loicgreffier:GH-4328

Conversation

@loicgreffier
Copy link
Copy Markdown

@loicgreffier loicgreffier commented Mar 20, 2026

The PR addresses the 4 points mentioned in issue #4328:

  • Make the RecoveringDeserializationExceptionHandler leverage the new Kafka Streams native DLQ introduced by KIP-1034.
  • Introduce a new NativeDeadLetterDestinationResolver as an alternative to resolve the Kafka Streams native DLQ destination.
  • Make the Kafka Streams native DLQ use the Spring Kafka DLT headers.
  • Introduce new RecoveringProcessingExceptionHandler and RecoveringProductionExceptionHandler.

To answer these points:

  • New DeadLetterRecordManager class. I've introduced a new DeadLetterRecordManager class as a common place to build DLT headers, so the utilities for building headers can be leveraged by both the DeadLetterPublishingRecoverer and exception handlers. I preferred this over a static utility class because of the number of parameters. One instance per handler and for the DeadLetterPublishingRecoverer.

  • New RecoveringProcessingExceptionHandler and RecoveringProductionExceptionHandler implementations with the same recovery logic as RecoveringDeserializationExceptionHandler. I've introduced a new AbstractRecoveringExceptionHandler that holds the commons for all handlers.

  • New NativeDeadLetterDestinationResolver interface. Allow users to define a native DLQ routing logic based on 3 given parameters: the ErrorHandlerContext, the source record as ConsumerRecord and the exception. All handlers leverage it. For now, it is loaded in the same way as the DeadLetterPublishingRecoverer was.

  • I’ve updated the tests to cover the 3 recovering handler implementations. I introduced an AbstractRecoveringExceptionHandlerTests class that contains commons tests. Child classes override some methods to create the corresponding handler and assert the related handling response.

Concerns:

  • The HeaderNames class is still located within the DeadLetterPublishingRecoverer class. It might be better to move it to DeadLetterRecordManager, but this would be a breaking change.

Closes #4328

@loicgreffier loicgreffier force-pushed the GH-4328 branch 8 times, most recently from 6c94761 to a0db242 Compare March 22, 2026 21:14
*
*/
public class RecoveringDeserializationExceptionHandler implements DeserializationExceptionHandler {
public class RecoveringDeserializationExceptionHandler
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Native DLQ in RecoveringDeserializationExceptionHandler (and the shared base logic) matches what we wanted on #4328. We’re on 4.1, so if something needs to change for the right long-term shape, that’s fine as long as we deprecate the old path and make the migration obvious.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did have one concern regarding the DeadLetterPublishingRecoverer#HeaderNames class, whether it should be moved to new DeadLetterRecordManager class. This would require deprecating DeadLetterPublishingRecoverer#HeaderNames in favor of DeadLetterRecordManager#HeaderNames, as well as public accessors like DeadLetterPublishingRecoverer#setHeaderNamesSupplier. But this move might not be necessary.

Regarding the RecoveringDeserializationExceptionHandler, it is currently backward compatible. We could consider dropping support for the Recoverer in favor of the native DLQ, but this would only work if the DeadLetterPublishingRecoverer implementation is used, if a custom implementation is used there would be no migration path.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree on both — HeaderNames move can be a follow-up if we decide it's worth the API churn. On the recoverer: keeping it is the right call since custom implementations have no native DLQ migration path; deprecating the old property name constant is enough for now.

@sobychacko
Copy link
Copy Markdown
Contributor

@loicgreffier Many thanks for the draft PR. I added some initial feedback that I wanted to convey. Please take a look and see what you think. Overall, the PR looks great. Some general things to keep in mind - we need to add a reference docs section for this (and some javadoc comments - see my inline comments). We also need a new entry in the whats-new section. Thanks!

@loicgreffier loicgreffier force-pushed the GH-4328 branch 8 times, most recently from 8411197 to 6ec0036 Compare March 26, 2026 23:05
The default value of `sameIntervalTopicReuseStrategy` in `RetryTopicConfigurationBuilder` has been changed from `MULTIPLE_TOPICS` to `SINGLE_TOPIC` to align with the `@RetryableTopic` annotation default.
See xref:retrytopic/topic-naming.adoc[Topic Naming] for more information.

[[x41-kafka-streams-native-dlq]]
Copy link
Copy Markdown
Contributor

@sobychacko sobychacko Mar 27, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add a link to the relevant ref docs sections from whats-new? Look at the other entries here for examples.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I’ve updated the whats-new section with multiple links to the reference documentation


- If a xref:streams.adoc#dead-letter-destination-resolver[`KafkaStreamsDeadLetterDestinationResolver`] is defined, resume the stream and forward the failed record to the resolved topic-partition using the native Kafka Streams DLQ.
- If `errors.dead.letter.queue.topic.name` is defined and set to a topic name, resume the stream and forward the failed record to that topic using the native Kafka Streams DLQ.
- If a `ConsumerRecordRecoverer` implementation is defined, evaluate it, and resume the stream with no dead-letter records, as it is expected to be handled by the `ConsumerRecordRecoverer`. The framework provides the `DeadLetterPublishingRecoverer` which sends the failed record to a dead-letter topic.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Our preference is one sentence per asciidoc line. If you can change it like that, would be great, but not a big deal, I can do that on merge also.

Copy link
Copy Markdown
Author

@loicgreffier loicgreffier Mar 27, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've updated the section so it now renders 4 bullet points describing the 4 recovery paths one after the other. So this #4360 (comment) is basically fixed.

Explanations regarding bullet point 3, about the ConsumerRecordRecoverer/DeadLetterPublishingRecoverer appear right after bullet point 4... Maybe it would be better to have a dedicated section on how to configure the DeadLetterPublishingRecoverer for an exception handler.


- The Kafka Streams property `errors.dead.letter.queue.topic.name`.
- An implementation of the new `KafkaStreamsDeadLetterDestinationResolver` functional interface for dynamic resolution.
- Through the `setDeadLetterQueueTopic()` of the `StreamsBuilderFactoryBean`.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't it setDeadLetterTopicName?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is, I've fixed it

*/
@FunctionalInterface
public interface KafkaStreamsDeadLetterDestinationResolver {
TopicPartition apply(ErrorHandlerContext errorHandlerContext, ConsumerRecord<?, ?> record, Exception exception);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

apply sounds like a function - how about resolve?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed from apply to resolve


Of course, the `recoverer()` bean can be your own implementation of `ConsumerRecordRecoverer`.

- Fail the stream with no dead-letter records.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you try to render this? Looks like an orphaned bullet item in the list. Might want to align this.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed by #4360 (comment)

@sobychacko
Copy link
Copy Markdown
Contributor

@loicgreffier Some more minor nitpicks. Once you address those, feel free to remove the draft status on the PR.

@loicgreffier loicgreffier marked this pull request as ready for review March 27, 2026 13:26
@loicgreffier
Copy link
Copy Markdown
Author

loicgreffier commented Mar 27, 2026

@sobychacko I've removed the draft status.

Regarding the documentation:

  • I've updated the whats-new section with information about the new handlers, the activation of the native DLQ, and the deprecated property.
  • For the streams documentation:
    • I've created 3 distincts sections for RecoveringDeserializationExceptionHandler, RecoveringProcessingExceptionHandler and RecoveringProductionExceptionHandler.
    • I've added a Recovery Strategies section that explains the recovery paths and options used by all handlers. Each handler section points to this Recovery Strategies section.
    • I've added a section for the new dead-letter destination resolver.
    • I've added a section for the Kafka Streams dead-letter topic name property and the StreamsBuilderFactoryBean.

If any sentence or wording isn't clear and need to be revised, please don't hesitate to let me know.

EDIT: I've added a word about the headers that are being attached to the dead-letter record.

@loicgreffier loicgreffier force-pushed the GH-4328 branch 2 times, most recently from e4ecab1 to 4e1dc01 Compare March 27, 2026 15:25
- If xref:streams.adoc#dead-letter-queue-topic-name-property[`errors.dead.letter.queue.topic.name`] is defined and set to a topic name, resume the stream and forward the failed record to that topic using the native Kafka Streams DLQ.
- If a `ConsumerRecordRecoverer` implementation is defined, invoke it and resume the stream without producing dead-letter records, as handling is delegated to the `ConsumerRecordRecoverer`.
- Fail the stream without producing dead-letter records.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The DeadLetterPublishingRecoverer explanation and code example currently appear after the "Fail" bullet above, which makes it look like it's related to the fail case. Since it's really elaborating on bullet 3 (the ConsumerRecordRecoverer path), could you either:

• Move it to right before the "Fail" bullet so it immediately follows the bullet it explains, or
• Pull it into a dedicated sub-section right after the bullet list, with a note in bullet 3 pointing to it — similar to how the resolver and DLQ topic have their own sections below. (This option might be the cleanest). What do you think?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull it into a dedicated sub-section right after the bullet list, with a note in bullet 3 pointing to it — similar to how the resolver and DLQ topic have their own sections below. (This option might be the cleanest). What do you think?

It aligns with #4360 (comment). I've updated this part to include a dedicated section for DeadLetterPublishingRecoverer.


- The Kafka Streams property xref:streams.adoc#dead-letter-queue-topic-name-property[`errors.dead.letter.queue.topic.name`].
- An implementation of the new xref:streams.adoc#dead-letter-destination-resolver[`KafkaStreamsDeadLetterDestinationResolver`] functional interface for dynamic resolution.
- The `setDeadLetterTopicName()` method of the `StreamsBuilderFactoryBean`.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there an xref we can add for this? since we link for everything else.

Copy link
Copy Markdown
Author

@loicgreffier loicgreffier Mar 27, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The StreamsBuilderFactoryBean updates are documented in the new Dead Letter Queue Topic Name Property section. I've added an xref to that section.

I've wondered whether a dedicated section for StreamsBuilderFactoryBean would be worth it, but there's not that much to say


Version 4.1 introduces the `RecoveringProcessingExceptionHandler`, which can take some action when an exception occurs during stream processing.
It implements the `ProcessingExceptionHandler` interface (refer to the Kafka documentation for details), introduced by https://cwiki.apache.org/confluence/display/KAFKA/KIP-1033%3A+Add+Kafka+Streams+exception+handler+for+exceptions+occurring+during+processing[KIP-1033]
and follows the xref:streams.adoc#recovery-strategies[Spring for Apache Kafka recovery strategies].
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Each handler says "follows the recovery strategies" with a forward xref. Flipping the order — Recovery Strategies first, then the handlers — would read more naturally for someone going top to bottom. What do you think?

Copy link
Copy Markdown
Author

@loicgreffier loicgreffier Mar 27, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Correct, especially since the Recovery Strategies section begins by introducing the 3 handlers provided by the framework:

The framework provides the following exception handlers, which follow the same recovery strategies:

- xref:streams.adoc#streams-deser-recovery[`RecoveringDeserializationExceptionHandler`]
- xref:streams.adoc#streams-processing-recovery[`RecoveringProcessingExceptionHandler`]
- xref:streams.adoc#streams-production-recovery[`RecoveringProductionExceptionHandler`]

Fixed!

*/
@FunctionalInterface
public interface KafkaStreamsDeadLetterDestinationResolver {
TopicPartition resolve(ErrorHandlerContext errorHandlerContext, ConsumerRecord<?, ?> record, Exception exception);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A method-level Javadoc on resolve would be a nice addition. For example, in the docs we show return new TopicPartition("dlqTopic1", -1) — it would help to briefly explain what the -1 partition value means. It's a Kafka concept rather than a Spring one, but a short note like "use -1 to let the producer's default partitioner choose the partition" would save users from having to dig into Kafka docs to understand it.

Copy link
Copy Markdown
Author

@loicgreffier loicgreffier Mar 27, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've added the Javadoc.

Regarding the new TopicPartition("dlqTopic1", -1), this originally comes from this current documentation section: https://docs.spring.io/spring-kafka/reference/streams.html#streams-deser-recovery.

-1 is currently handled by the DeadLetterPublishingRecoverer:

topicPartition.partition() < 0 ? null : topicPartition.partition(),
and is converted to null before constructing the ProducerRecord.

This is documented here already: https://docs.spring.io/spring-kafka/reference/kafka/annotation-error-handling.html#dead-letters. The new Streams > Dead Letter Publishing Recoverer section links to it.

Still worth reiterating this in Javadoc and new Dead Letter Destination Resolver section.

@sobychacko
Copy link
Copy Markdown
Contributor

sobychacko commented Mar 27, 2026

@loicgreffier Thanks for all the updates. I added a final round of PR review comments. Hopefully, once that is addressed, we can take it for a final spin of review and merge.

Btw - can you update the commit message with some more details about the rationale for the changes. We usually follow these guidelines for commit messages: https://cbea.ms/git-commit/

Thanks!

@sobychacko
Copy link
Copy Markdown
Contributor

Could you take care of the commit message? (See my comment above). Also, please sign your commits following DCO - https://spring.io/blog/2025/01/06/hello-dco-goodbye-cla-simplifying-contributions-to-spring

@loicgreffier loicgreffier changed the title GH-4328: Add support for native Kafka Streams DLQ GH-4328: Expose native Kafka Streams DLQ configuration Mar 27, 2026
@loicgreffier loicgreffier force-pushed the GH-4328 branch 2 times, most recently from 62aa2f5 to 7f933c1 Compare March 27, 2026 21:10
Extract the DLT record header-building logic into a new DeadLetterRecordManager
class so it can be used outside of DeadLetterPublishingRecoverer.

Add two new handlers, RecoveringProcessingExceptionHandler and
RecoveringProductionExceptionHandler, to handle processing and production errors, respectively.

Add a common AbstractRecoveringExceptionHandler that centralizes
the shared error-handling logic for all provided recovering exception handlers.

Add a KafkaStreamsDeadLetterDestinationResolver to allow users to define
dead-letter routing logic used by the Kafka Streams native DLQ in the provided
exception handler implementations.

Update StreamsBuilderFactoryBean to expose a dead-letter queue topic name
as a possible destination for all provided recovering exception handler implementations.

Fixes: spring-projects#4328
Signed-off-by: Loïc Greffier <loic.greffier@michelin.com>
@loicgreffier
Copy link
Copy Markdown
Author

Commit message has been changed, accordingly to the original issue title

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Expose native Kafka Streams DLQ configuration (KIP-1034)

2 participants