ity to return:
> >
> > * FAIL
> > * CONTINUE
> > * FAIL.withDeadLetterQueueRecord(record, "dlq-topic")
> > * CONTINUE.withDeadLetterQueueRecord(record, "dlq-topic")
> >
> > A DLQ topic name is currently required using the two l
; > * FAIL.withDeadLetterQueueRecord(record, "dlq-topic")
> > * CONTINUE.withDeadLetterQueueRecord(record, "dlq-topic")
> >
> > A DLQ topic name is currently required using the two last response types.
> > I am wondering if it could benefit users t
"dlq-topic")
>
> A DLQ topic name is currently required using the two last response types.
> I am wondering if it could benefit users to ease the use of the default DLQ
> topic "errors.deadletterqueue.topic.name" when implementing custom handlers,
> with such kind o
rqueue.topic.name" when implementing custom handlers,
with such kind of implementation:
* FAIL.withDefaultDeadLetterQueueRecord(record)
* CONTINUE.withDefaultDeadLetterQueueRecord(record)
Regards,
Loïc
De : Damien Gasparina
Envoyé : dimanche 14 avril 2024 20:24
À : dev@kafka.a
Hi Sophie,
Thanks a lot for your feedback and your detailed comments.
S1.
> I'm confused -- are you saying that we're introducing a new kind of
ProducerRecord class for this?
Sorry for the poor wording, that's not what I meant. While writing the
KIP, I was hesitating between 1. leveraging the
Thanks for the KIP, this will make a lot of people very happy.
Wanted to chime in on a few points that have been raised so far and add
some of my own (numbering with an S to distinguish my points from the
previous ones)
S1.
> 1.a I really meant ProducerRecord, that's the class used to forward
Hi Andrew,
Thanks a lot for your review, plenty of good points!
11. Typo fixed, good cach.
12. I do agree with you and Nick also mentioned it, I updated the KIP
to mention that context headers should be forwarded.
13. Good catch, to be consistent with KIP-298, and without a strong
opinion from
Hi Nick,
1. Good point, that's less impactful than a custom interface, I just
updated the KIP with the new signature.
1.a I really meant ProducerRecord, that's the class used to forward to
downstream processors in the PAPI. The only information missing in
this class is the topic name. I also
Hi Damien, Sebastien and Loic,
Thanks for the KIP. The DLQ pattern is well established and bringing this to
Kafka Streams is a good improvement. I do plan to add DLQ support to share
groups in the future, once KIP-932 is complete. Having broad support in Kafka
for DLQs is great.
I have a few
Hi Damien and Sebastien,
1.
I think you can just add a `String topic` argument to the existing
`withDeadLetterQueueRecord(ProducerRecord
deadLetterQueueRecord)` method, and then the implementation of the
exception handler could choose the topic to send records to using whatever
logic the user
Sébastien
De : Nick Telford
Envoyé : vendredi 12 avril 2024 12:57
À : dev@kafka.apache.org
Objet : [EXT] Re: [DISCUSS] KIP-1034: Dead letter queue in Kafka Streams
Warning External sender Do not click on any links or open any attachments
unless you trust the sender
Hi Nick,
Thanks a lot for your review and your useful comments!
1. It is a good point, as you mentioned, I think it would make sense
in some use cases to have potentially multiple DLQ topics, so we
should provide an API to let users do it.
Thinking out-loud here, maybe it is a better approach to
Oh, and one more thing:
5.
Whenever you take a record out of the stream, and then potentially
re-introduce it at a later date, you introduce the potential for record
ordering issues. For example, that record could have been destined for a
Window that has been closed by the time it's re-processed.
Hi Damien,
Thanks for the KIP! Dead-letter queues are something that I think a lot of
users would like.
I think there are a few points with this KIP that concern me:
1.
It looks like you can only define a single, global DLQ for the entire Kafka
Streams application? What about applications that
In a general way, if the user does not configure the right ACL, that
would be a security issue, but that's true for any topic.
This KIP allows users to configure a Dead Letter Queue without writing
custom Java code in Kafka Streams, not at the topic level.
A lot of applications are already
My concern is that someone would create a dead letter queue on a sensitive
topic and not get the ACL correct from the start. Thus causing potential
confidential data leak. Is there anything in the proposal that would
prevent that from happening? If so I did not recognize it as such.
On Fri,
Hi Claude,
In this KIP, the Dead Letter Queue is materialized by a standard and
independant topic, thus normal ACL applies to it like any other topic.
This should not introduce any security issues, obviously, the right
ACL would need to be provided to write to the DLQ if configured.
Cheers,
I am new to the Kafka codebase so please excuse any ignorance on my part.
When a dead letter queue is established is there a process to ensure that
it at least is defined with the same ACL as the original queue? Without
such a guarantee at the start it seems that managing dead letter queues
will
Hi everyone,
To continue on our effort to improve Kafka Streams error handling, we
propose a new KIP to add out of the box support for Dead Letter Queue.
The goal of this KIP is to provide a default implementation that
should be suitable for most applications and allow users to override
it if
19 matches
Mail list logo