Thanks Bill, Bruno, Sophie, I’ll conclude the vote with your 3 binding votes.
Thank you everyone!
Damien
On Thu, 26 Sept 2024 at 15:21, Bill Bejeck wrote:
>
> Thanks for the KIP, this will be a great addition.
>
> +1(binding)
>
> Regards,
> Bill
>
> On Thu, Sep 26, 2024 at 9:19 AM Bruno Cadonna
ts
> something. It says CONTINUE processing or FAIL the processing. With your
> KIP the response gets an additional instruction, namely
> `andAddToDeadLetterQueue`. I would not sacrifice better readability for
> consistency in this case.
>
>
> Best,
> Bruno
>
> On 9/3/2
es(StandardCharsets.UTF_8),
> "World".getBytes(StandardCharsets.UTF_8)
> ))
> );
> }
>
> I think the code becomes more readable.
>
>
> Best,
> Bruno
>
> On 8/30/24 3:37 PM, Damien Gasparina wrote:
> > Hi everyone,
&g
> This email was screened for spam and malicious content but exercise caution
> anyway.
>
>
>
>
> On 6/13/24 6:16 PM, Damien Gasparina wrote:
> > Hi Bruno,
> >
> > We focused our effort (well, mostly Seb and Loic :)) on KIP-1033,
> > that's why not much
he existing exception handlers, but it can cause similar failures
> > (potentially custom logic, depends on validity input record). There
> > could also be a default implementation as a subclass of
> > `ExtractRecordMetadataTimestamp`.
> >
> > L3. It would be nice
gt; > >>
> > >>> Hi Damien, Sébastien, and Loïc,
> > >>>
> > >>> Thanks for the KIP!
> > >>>
> > >>> +1 (binding)
> > >>>
> > >>> Best,
> > >>> Bruno
> > >>&
Thanks for the feedback, I think we should keep two separate callbacks
for serialization and error handlers. It makes sense for type safety
(ProducerRecord vs POJO) and also for backward
compatibility. On top of that, all metadata provided in the #handle
method would need to be held in memory until
Hi all,
We would like to start a vote for KIP-1033: Add Kafka Streams
exception handler for exceptions occurring during processing
The KIP is available on
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1033%3A+Add+Kafka+Streams+exception+handler+for+exceptions+occurring+during+processing
t
> of sense. Not sure if we would need any extentions in this KIP to get
> this done? I would rather include timestamp extraction issue in the DLQ
> KIP from day one on. The interface is quite different though, so we
> would need to think a little bit about it in more details how to
> 3. exception
>
>
> (6)
> I am wondering where the implementation of ProcessingMetadata gets the
> sourceRawKey/Value from. Do we need additional changes in
> ProcessingContext and implementations?
>
>
> Best,
> Bruno
>
>
> On 4/21/24 2:23 PM, Damien Gaspari
etterQueueRecord(record, "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 handler
on so far I agree with! The
> ProcessingContext thing is the only open question in my mind
>
> On Thu, Apr 11, 2024 at 5:41 AM Damien Gasparina
> wrote:
>
> > Hi Matthias, Bruno,
> >
> > 1.a During my previous comment, by Processor Node ID, I meant
> > Process
7;m worried it will. Can you add a section that
> includes a basic implementation of all the features here? Nothing too
> complicated, just the most bare-bones code needed to actually implement
> forwarding to a dead-letter-queue via the handlers.
>
> Lastly, two super small things:
e the topic and the topic should either be
automatically created, or pre-created.
17. If a DLQ record can not be sent, the exception should go to the
uncaughtExceptionHandler. Let me clearly state it in the KIP.
On Fri, 12 Apr 2024 at 17:25, Damien Gasparina wrote:
>
> Hi Nick,
>
>
rs to discard all custom headers set on the
> source record. Is there a way these can be included? In particular, I'm
> concerned with "schema pointer" headers (like those set by Schema
> Registry), that may need to be propagated, especially if the records are
> fed back int
ead, which statically guarantees that it can't be
> > used to forward records.
> >
> > 4.
> > You mention the KIP-1033 ProcessingExceptionHandler, but what's the plan
> > if KIP-1033 is not adopted, or if KIP-1034 lands before 1033?
> >
> > Regards,
>
i, Apr 12, 2024 at 9:45 AM Damien Gasparina
> wrote:
>
> > 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
ithout
> such a guarantee at the start it seems that managing dead letter queues
> will be fraught with security issues.
>
>
> On Wed, Apr 10, 2024 at 10:34 AM Damien Gasparina
> wrote:
>
> > Hi everyone,
> >
> > To continue on our effort to improve Kafka S
esponse ==
> >> ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) {
> >> > throw new StreamsException("Processing exception handler is set to
> >> fail upon" +
> >> > " a processing error. If you would rather have the streaming
> >> pipeline" +
Damien Gasparina created KAFKA-16505:
Summary: KIP-1034: Dead letter queue in Kafka Streams
Key: KAFKA-16505
URL: https://issues.apache.org/jira/browse/KAFKA-16505
Project: Kafka
Issue
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 they
> which
> > >> has all the metadata and useful info of the ProcessorContext but
> without
> > >> the
> > >> forwarding APIs. This would also lets us sidestep the following issue:
> > >> 2b. If you *do* want the ability to forward records, s
Damien Gasparina created KAFKA-16448:
Summary: Add Kafka Streams exception handler for exceptions
occuring during processing (KIP-1033)
Key: KAFKA-16448
URL: https://issues.apache.org/jira/browse/KAFKA-16448
Hi everyone,
After writing quite a few Kafka Streams applications, me and my colleagues
just created KIP-1033 to introduce a new Exception Handler in Kafka Streams
to simplify error handling.
This feature would allow defining an exception handler to automatically
catch exceptions occurring during
Hi team,
I would like permission to contribute to Kafka.
My wiki ID is "d.gasparina" and my Jira ID is "Dabz".
I would like to propose a KIP to improve Kafka Streams error and exception
handling.
Cheers,
Damien
Damien Gasparina created KAFKA-14302:
Summary: Infinite probing rebalance if a changelog topic got
emptied
Key: KAFKA-14302
URL: https://issues.apache.org/jira/browse/KAFKA-14302
Project: Kafka
Damien Gasparina created KAFKA-13636:
Summary: Committed offsets could be deleted during a rebalance if
a group did not commit for a while
Key: KAFKA-13636
URL: https://issues.apache.org/jira/browse/KAFKA
Damien Gasparina created KAFKA-13109:
Summary: WorkerSourceTask is not enforcing the
errors.retry.timeout and errors.retry.delay.max.ms parameters in case of a
RetriableException during task.poll()
Key: KAFKA-13109
Damien Gasparina created KAFKA-13024:
Summary: Kafka Streams is dropping messages with null key during
repartition
Key: KAFKA-13024
URL: https://issues.apache.org/jira/browse/KAFKA-13024
Project
Damien Gasparina created KAFKA-12951:
Summary: Infinite loop while restoring a GlobalKTable
Key: KAFKA-12951
URL: https://issues.apache.org/jira/browse/KAFKA-12951
Project: Kafka
Issue
Damien Gasparina created KAFKA-12272:
Summary: Kafka Streams metric commit-latency-max and
commit-latency-avg is always 0
Key: KAFKA-12272
URL: https://issues.apache.org/jira/browse/KAFKA-12272
Damien Gasparina created KAFKA-7129:
---
Summary: Dynamic default value for number of thread configuration
Key: KAFKA-7129
URL: https://issues.apache.org/jira/browse/KAFKA-7129
Project: Kafka
32 matches
Mail list logo