Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Kafka-16505: Adding dead letter queue in Kafka Streams #17942

Open
wants to merge 12 commits into
base: trunk
Choose a base branch
from

Conversation

Dabz
Copy link

@Dabz Dabz commented Nov 25, 2024

First Pull Request to implement KIP-1034 to add support of Dead Letter Queue in Kafka Streams.
A second PR will store the source record key and value byte[] in the context to add it to the DLQ records.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@Dabz Dabz force-pushed the KAFKA-16505-Dead-letter-queue-in-Kafka-Streams branch from 4fc7717 to 5698e60 Compare November 25, 2024 16:15
Copy link
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

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

Thanks for the PR, @Dabz !

I started to review it, but did not finish since I need to think about how this KIP uses mutable enums which I guess is an anti-pattern.
Nevertheless, I wanted to give my first comments.

@@ -588,6 +588,10 @@ public class StreamsConfig extends AbstractConfig {
public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." +
" The cluster must have a client metrics subscription which corresponds to a client.";

public static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG = "errors.dead.letter.queue.topic.name";
Copy link
Member

Choose a reason for hiding this comment

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

Could you please add unit tests?

Copy link
Author

Choose a reason for hiding this comment

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

Done :)

Copy link
Member

Choose a reason for hiding this comment

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

You cannot add a class to the public API that was not mentioned in the KIP.

Copy link
Author

Choose a reason for hiding this comment

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

I renamed this class to ExceptionHandlerUtils, make it packaged protected, and made all methods statics to avoid inheritance.

public static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset";


public boolean shouldBuildDeadLetterQueueRecord() {
Copy link
Member

Choose a reason for hiding this comment

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

Why is this public? It is only used within this class.

Copy link
Author

Choose a reason for hiding this comment

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

Changed it to package protected

return this;
}

public List<ProducerRecord<byte[], byte[]>> drainDeadLetterQueueRecords() {
Copy link
Member

Choose a reason for hiding this comment

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

According to the KIP this should be named deadLetterQueueReords().

Copy link
Author

Choose a reason for hiding this comment

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

Good point, I renamed it

/**
* {@code CommonExceptionHandler} Contains utilities method that could be used by all exception handlers
*/
public class CommonExceptionHandler implements Configurable {
Copy link
Member

Choose a reason for hiding this comment

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

It would be better to define this class as a immutable utility class in the internal package that provides some utility methods to the default exception handlers. That means, no inheritance.

Copy link
Author

Choose a reason for hiding this comment

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

Done

@lucasbru
Copy link
Member

lucasbru commented Dec 3, 2024

Indeed - using a single collection of dead letter records here that will shared across all stream threads doesn't seem like the right approach to me. Was this intended?

I'm not sure if there is a way to replace the enum by a class of some kind where we can properly define a non-static collection of records and remain binary compatible.

It seems to me that this needs a new design proposal and an updated KIP?

@cadonna
Copy link
Member

cadonna commented Dec 3, 2024

An option to fix this might be to add a new method deadLetterQueueRecords() to the handler interface that returns a list of records to add to the dead letter queue. By default the list is empty.

Copy link
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

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

Thanks for the updates @sebastienviale !

I did a first pass. Here my comments.

Comment on lines 62 to 67
return handle(record, exception);
throw new UnsupportedOperationException();
Copy link
Member

Choose a reason for hiding this comment

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

Shouldn't this still call the other deprecated method?
Imagine a user implemented

handle(final ProducerRecord<byte[], byte[]> record, final Exception exception)

but not

ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, 
                                          final ProducerRecord<byte[], byte[]> record,
                                          final Exception exception)

Streams would throw an UnsupportedOperationException although it did not before upgrading to this version.

Copy link
Member

Choose a reason for hiding this comment

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

Maybe it would be beneficial to add some unit tests that verify this redirection. With such unit tests, this removal would had happened without some thoughts about why the test failed.

Copy link
Contributor

Choose a reason for hiding this comment

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

indeed, I added a Unit Test that failed if an UnsupportedOperationException is thrown

/**
* Represents the result of handling a production exception.
* <p>
* The {@code Response} class encapsulates a {@link ProductionExceptionHandlerResponse},
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
* The {@code Response} class encapsulates a {@link ProductionExceptionHandlerResponse},
* The {@code Response} class encapsulates a {@link Result},

Copy link
Contributor

Choose a reason for hiding this comment

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

done

* {@link ProducerRecord} instances to be sent to a dead letter queue.
* </p>
*/
class Response {
Copy link
Member

Choose a reason for hiding this comment

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

Could you please add some unit tests for this class?

Copy link
Contributor

Choose a reason for hiding this comment

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

done

final ErrorHandlerContext context,
final Exception e) {
if (deadLetterQueueTopicName == null) {
throw new InvalidConfigurationException(String.format("%s can not be null while building DeadLetterQueue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG));
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
throw new InvalidConfigurationException(String.format("%s can not be null while building DeadLetterQueue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG));
throw new InvalidConfigurationException(String.format("%s cannot be null while building dead letter queue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG));

Copy link
Contributor

Choose a reason for hiding this comment

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

done

Comment on lines 74 to 78
static ProducerRecord<byte[], byte[]> buildDeadLetterQueueRecord(final String deadLetterQueueTopicName,
final byte[] key,
final byte[] value,
final ErrorHandlerContext context,
final Exception e) {
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
static ProducerRecord<byte[], byte[]> buildDeadLetterQueueRecord(final String deadLetterQueueTopicName,
final byte[] key,
final byte[] value,
final ErrorHandlerContext context,
final Exception e) {
static ProducerRecord<byte[], byte[]> buildDeadLetterQueueRecord(final String deadLetterQueueTopicName,
final byte[] key,
final byte[] value,
final ErrorHandlerContext context,
final Exception e) {

Copy link
Contributor

Choose a reason for hiding this comment

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

done

@@ -50,6 +54,8 @@ public DeserializationHandlerResponse handle(final ProcessorContext context,
return DeserializationHandlerResponse.CONTINUE;
}

@SuppressWarnings("deprecation")
@Deprecated
@Override
public DeserializationHandlerResponse handle(final ErrorHandlerContext context,
Copy link
Member

Choose a reason for hiding this comment

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

You can remove those deprecated handler methods. They are not called anywhere in the Streams code.

Copy link
Contributor

Choose a reason for hiding this comment

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

done

@Override
public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception) {
public ProcessingHandlerResponse handle(final ErrorHandlerContext context,
Copy link
Member

Choose a reason for hiding this comment

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

You can remove this deprecated handler method. It is not called anywhere in the Streams code.

Copy link
Contributor

Choose a reason for hiding this comment

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

done

public Response handleError(final ErrorHandlerContext context,
final ConsumerRecord<byte[], byte[]> record,
final Exception exception) {
log.warn(
Copy link
Member

Choose a reason for hiding this comment

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

This should be log.error(). It is failing, not resuming.

Copy link
Contributor

Choose a reason for hiding this comment

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

done

@@ -50,6 +54,8 @@ public DeserializationHandlerResponse handle(final ProcessorContext context,
return DeserializationHandlerResponse.FAIL;
}

@SuppressWarnings("deprecation")
@Deprecated
@Override
public DeserializationHandlerResponse handle(final ErrorHandlerContext context,
Copy link
Member

Choose a reason for hiding this comment

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

You can remove those deprecated handler methods. They are not called anywhere in the Streams code.

Copy link
Contributor

Choose a reason for hiding this comment

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

done

Comment on lines 39 to 40
public ProcessingHandlerResponse handle(final ErrorHandlerContext context,
final Record<?, ?> record, final Exception exception) {
Copy link
Member

Choose a reason for hiding this comment

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

You can remove this deprecated handler method. It is not called anywhere in the Streams code.

Copy link
Contributor

Choose a reason for hiding this comment

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

done

@sebastienviale sebastienviale force-pushed the KAFKA-16505-Dead-letter-queue-in-Kafka-Streams branch from 93cd94d to 93c831f Compare January 17, 2025 09:27
@sebastienviale
Copy link
Contributor

Thanks for the updates @sebastienviale !

I did a first pass. Here my comments.

@cadonna thanks for your review, I pushed changes

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants