Also ignore everything I said about Streams earlier. I didn't look closely
enough on my first pass over the KIP and thought this was changing the
DeserializationExceptionHandler in Streams. I see now that this is
actually about the consumer client's DeserializationException so everything
I said about using a ByteArray Deserialization and Kafka Streams doesn't
apply here. The important thing is just that it still deserializes one
record
at a time, and essentially throws this when it fails to convert the Record
type into a ConsumerRecord type. So there's always only one record
at a type to consider.

Sorry for any confusion I caused

On Tue, Apr 16, 2024 at 11:15 AM Sophie Blee-Goldman <sop...@responsive.dev>
wrote:

> Ah, thanks for the additional context. I should have looked at the code
> before I opened my mouth (so to speak)
>
> In that case, I fully agree that using Record instead of ConsumerRecord
> makes sense. It does indeed seem like by definition, if there is a
> DeserializationException then there is no ConsumerRecord since this
> is where/how it gets thrown:
>
> try {
> ...
> return new ConsumerRecord<>(...);
> } catch (RuntimeException e) {
> ...
> throw new RecordDeserializationException(...);
> }
>
> As you mentioned the Record is an input to the method so we definitely have
> one of those, and imo, it makes sense to use. As far as I can tell it's
> just
> a regular public interface so exposing it shouldn't be an issue just based
> on
> the class itself. But I'm still a bit concerned about the checkstyle
> complaint.
>
> I'll try to find someone who can explain why or if we should avoid
> returning
> a Record type here. Other than that, I'd say the KIP LGTM as-is and we
> could kick off voting
>
> On Tue, Apr 16, 2024 at 10:47 AM Frédérik Rouleau
> <froul...@confluent.io.invalid> wrote:
>
>> Thanks Sophie,
>>
>> I can write something in the KIP on how KStreams solves that issue, but as
>> I can't create a Wiki account, I will have to find someone to do this on
>> my
>> behalf (if someone can work on solving that wiki account creation, it
>> would
>> be great).
>>
>> The biggest difference between Record and ConsumerRecord is that data are
>> stored respectively using ByteBuffer and Byte array.
>>
>> For the Record option, the object already exists in the parsing method, so
>> it's roughly just a parameter type change in the Exception. The point is
>> just about exposing the Record class externally. By the way, the name
>> Record is also making some IDE a bit crazy by confusing it with the new
>> Java Record feature. An alternative could be to create another wrapper
>> type
>> of just include key and value ByteBuffer in the
>> RecordDeserializationException itself.
>>
>> For the ConsumerRecord option, it requires to allocate Byte arrays, even
>> if
>> the user does not need it (skip the poison pill for example). This might
>> have some extra cost on GC for some specific use case.
>>
>> Fred
>>
>

Reply via email to