Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-05-03 Thread Frédérik Rouleau
Hi Sophie,

I have updated the KIP and the PR.

Regards,


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-05-02 Thread Sophie Blee-Goldman
Sounds good -- exceptionOrigin makes sense to me.

Let us know when you've updated the KIP. I'll cast a vote once these last
small changes have been made

On Thu, May 2, 2024 at 10:16 AM Frédérik Rouleau
 wrote:

> Hi Sophie,
>
> I agree that the subclasses have limited value and I am not a fan of
> "instance of" usage either.
> I do not see any problem with adding a field but I would rather name it
> something like exceptionOrigin. Any thoughts?
>
> About byteBuffer vs byte[], byteBuffer are more generic and with proper
> doc/example, I do not think it's an issue. I will then remove the byte[]
> returning methods.
>
> Thanks,
>
>
> [image: Confluent] 
> Frederik Rouleau
> Sr Customer Success Technical Architect
> Follow us: [image: Blog]
> <
> https://www.confluent.io/blog?utm_source=footer_medium=email_campaign=ch.email-signature_type.community_content.blog
> >[image:
> Twitter] [image: LinkedIn]
> [image: Slack]
> [image: YouTube]
> 
>
> [image: Try Confluent Cloud for Free]
> <
> https://www.confluent.io/get-started?utm_campaign=tm.fm-apac_cd.inbound_source=gmail_medium=organic
> >
>
>
> On Tue, Apr 30, 2024 at 10:54 PM Sophie Blee-Goldman <
> sop...@responsive.dev>
> wrote:
>
> > Actually one more thing, after thinking a bit about how this would be
> used
> > in practice, I'm inclined to agree that maybe distinguishing between key
> vs
> > value errors via subclasses is not the cleanest way to present the API.
> > Users would still essentially want to catch the general
> > RecordDeserializationException error since in practice, much of the
> > handling is likely to be the same between key and value errors. So then
> to
> > distinguish between these, they would end up doing an "instance of" check
> > on the exception type. Which feels like an awkward way to answer a
> question
> > that could have just been a simple API on the
> > RecordDeserializationException itself. What do you think about getting
> rid
> > of the subclasses and instead adding one more API to the
> > RecordDeserializationException that indicates whether it was a key or
> value
> > error?
> >
> > This could return a simple boolean and be called #isKeyError or
> something,
> > but that feels kind of awkward too. Maybe a better alternative would be
> > something like this:
> >
> > class RecordDeserializationException {
> > enum DeserializationExceptionType {
> > KEY,
> > VALUE
> > }
> >
> >   public DeserializationExceptionType exceptionType();
> > }
> >
> > I also worry that people don't always check for exception subtypes and
> > would easily miss the existence of the KeyDeserializationException and
> > ValueDeserializationException. Simply adding an API to the
> > RecordDeserializationException will make it much easier for users to
> notice
> > and react accordingly, if they care to do something different based on
> > whether the error happened during key or value deserialization.
> >
> > Thoughts?
> >
> > On Tue, Apr 30, 2024 at 1:45 PM Sophie Blee-Goldman <
> sop...@responsive.dev
> > >
> > wrote:
> >
> > > Hey Fred, I think this is looking good but I just want to follow up on
> > > what Kirk asked earlier about having both the ByteBuffer and byte[]
> > forms.
> > > Can't users just use the ByteBuffer versions and convert them into a
> > byte[]
> > > themselves? In some cases it maybe makes sense to offer some additional
> > > APIs if there is complex processing involved in converting between
> > returned
> > > types, but ByteBuffer --> byte[] seems pretty straightforward to me :)
> > >
> > > Generally speaking we try to keep the APIs as tight as possible and
> offer
> > > only what is necessary, and I'd rather leave off "syntactic sugar" type
> > > APIs unless there is a clear need. Put another way: it's easy to add
> > > additional methods if someone wants them, but it's much harder to
> remote
> > > methods since we have to go through a deprecation cycle. So I'd prefer
> to
> > > just keep only the ByteBuffer versions (or only the byte[] -- don't
> > > personally care which of the two)
> > >
> > > One more small nit: since we're deprecating the old exception
> > constructor,
> > > can you list that in the "Compatibility, Deprecation, and Migration
> Plan"
> > > section?
> > >
> > >
> > >
> > > On Wed, Apr 24, 2024 at 5:35 AM Frédérik Rouleau
> > >  wrote:
> > >
> > >> Hi,
> > >>
> > >> I have updated the KIP now and the latest version of PR is available.
> > >>
> > >> About Kirk's questions:
> > >>
> > >> K11: Yes, both can have a deserialization exception but we deserialize
> > the
> > >> key first, so if an error occurs then, we do not try to deserialize
> the
> > >> value. So the exception raised is always for key or value.
> > >>
> > >> K12: Not sure of concrete usage for now, just a sugar feature. I
> suppose
> > >> we
> > >> 

Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-05-02 Thread Frédérik Rouleau
Hi Sophie,

I agree that the subclasses have limited value and I am not a fan of
"instance of" usage either.
I do not see any problem with adding a field but I would rather name it
something like exceptionOrigin. Any thoughts?

About byteBuffer vs byte[], byteBuffer are more generic and with proper
doc/example, I do not think it's an issue. I will then remove the byte[]
returning methods.

Thanks,


[image: Confluent] 
Frederik Rouleau
Sr Customer Success Technical Architect
Follow us: [image: Blog]
[image:
Twitter] [image: LinkedIn]
[image: Slack]
[image: YouTube]


[image: Try Confluent Cloud for Free]



On Tue, Apr 30, 2024 at 10:54 PM Sophie Blee-Goldman 
wrote:

> Actually one more thing, after thinking a bit about how this would be used
> in practice, I'm inclined to agree that maybe distinguishing between key vs
> value errors via subclasses is not the cleanest way to present the API.
> Users would still essentially want to catch the general
> RecordDeserializationException error since in practice, much of the
> handling is likely to be the same between key and value errors. So then to
> distinguish between these, they would end up doing an "instance of" check
> on the exception type. Which feels like an awkward way to answer a question
> that could have just been a simple API on the
> RecordDeserializationException itself. What do you think about getting rid
> of the subclasses and instead adding one more API to the
> RecordDeserializationException that indicates whether it was a key or value
> error?
>
> This could return a simple boolean and be called #isKeyError or something,
> but that feels kind of awkward too. Maybe a better alternative would be
> something like this:
>
> class RecordDeserializationException {
> enum DeserializationExceptionType {
> KEY,
> VALUE
> }
>
>   public DeserializationExceptionType exceptionType();
> }
>
> I also worry that people don't always check for exception subtypes and
> would easily miss the existence of the KeyDeserializationException and
> ValueDeserializationException. Simply adding an API to the
> RecordDeserializationException will make it much easier for users to notice
> and react accordingly, if they care to do something different based on
> whether the error happened during key or value deserialization.
>
> Thoughts?
>
> On Tue, Apr 30, 2024 at 1:45 PM Sophie Blee-Goldman  >
> wrote:
>
> > Hey Fred, I think this is looking good but I just want to follow up on
> > what Kirk asked earlier about having both the ByteBuffer and byte[]
> forms.
> > Can't users just use the ByteBuffer versions and convert them into a
> byte[]
> > themselves? In some cases it maybe makes sense to offer some additional
> > APIs if there is complex processing involved in converting between
> returned
> > types, but ByteBuffer --> byte[] seems pretty straightforward to me :)
> >
> > Generally speaking we try to keep the APIs as tight as possible and offer
> > only what is necessary, and I'd rather leave off "syntactic sugar" type
> > APIs unless there is a clear need. Put another way: it's easy to add
> > additional methods if someone wants them, but it's much harder to remote
> > methods since we have to go through a deprecation cycle. So I'd prefer to
> > just keep only the ByteBuffer versions (or only the byte[] -- don't
> > personally care which of the two)
> >
> > One more small nit: since we're deprecating the old exception
> constructor,
> > can you list that in the "Compatibility, Deprecation, and Migration Plan"
> > section?
> >
> >
> >
> > On Wed, Apr 24, 2024 at 5:35 AM Frédérik Rouleau
> >  wrote:
> >
> >> Hi,
> >>
> >> I have updated the KIP now and the latest version of PR is available.
> >>
> >> About Kirk's questions:
> >>
> >> K11: Yes, both can have a deserialization exception but we deserialize
> the
> >> key first, so if an error occurs then, we do not try to deserialize the
> >> value. So the exception raised is always for key or value.
> >>
> >> K12: Not sure of concrete usage for now, just a sugar feature. I suppose
> >> we
> >> can imagine some use case where you need/want only the first bytes and
> do
> >> not want to waste memory allocating the whole payload (SchemaRegistry's
> >> schema Id or something similar).
> >>
> >> K13: The old constructor is not needed anymore. It is just for
> >> compatibility until removed in a major version. As public we might have
> >> some users using it even if I cannot see any valid reason for this.
> >>
> >> Thanks,
> >> Fred
> >>
> >
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-30 Thread Sophie Blee-Goldman
Actually one more thing, after thinking a bit about how this would be used
in practice, I'm inclined to agree that maybe distinguishing between key vs
value errors via subclasses is not the cleanest way to present the API.
Users would still essentially want to catch the general
RecordDeserializationException error since in practice, much of the
handling is likely to be the same between key and value errors. So then to
distinguish between these, they would end up doing an "instance of" check
on the exception type. Which feels like an awkward way to answer a question
that could have just been a simple API on the
RecordDeserializationException itself. What do you think about getting rid
of the subclasses and instead adding one more API to the
RecordDeserializationException that indicates whether it was a key or value
error?

This could return a simple boolean and be called #isKeyError or something,
but that feels kind of awkward too. Maybe a better alternative would be
something like this:

class RecordDeserializationException {
enum DeserializationExceptionType {
KEY,
VALUE
}

  public DeserializationExceptionType exceptionType();
}

I also worry that people don't always check for exception subtypes and
would easily miss the existence of the KeyDeserializationException and
ValueDeserializationException. Simply adding an API to the
RecordDeserializationException will make it much easier for users to notice
and react accordingly, if they care to do something different based on
whether the error happened during key or value deserialization.

Thoughts?

On Tue, Apr 30, 2024 at 1:45 PM Sophie Blee-Goldman 
wrote:

> Hey Fred, I think this is looking good but I just want to follow up on
> what Kirk asked earlier about having both the ByteBuffer and byte[] forms.
> Can't users just use the ByteBuffer versions and convert them into a byte[]
> themselves? In some cases it maybe makes sense to offer some additional
> APIs if there is complex processing involved in converting between returned
> types, but ByteBuffer --> byte[] seems pretty straightforward to me :)
>
> Generally speaking we try to keep the APIs as tight as possible and offer
> only what is necessary, and I'd rather leave off "syntactic sugar" type
> APIs unless there is a clear need. Put another way: it's easy to add
> additional methods if someone wants them, but it's much harder to remote
> methods since we have to go through a deprecation cycle. So I'd prefer to
> just keep only the ByteBuffer versions (or only the byte[] -- don't
> personally care which of the two)
>
> One more small nit: since we're deprecating the old exception constructor,
> can you list that in the "Compatibility, Deprecation, and Migration Plan"
> section?
>
>
>
> On Wed, Apr 24, 2024 at 5:35 AM Frédérik Rouleau
>  wrote:
>
>> Hi,
>>
>> I have updated the KIP now and the latest version of PR is available.
>>
>> About Kirk's questions:
>>
>> K11: Yes, both can have a deserialization exception but we deserialize the
>> key first, so if an error occurs then, we do not try to deserialize the
>> value. So the exception raised is always for key or value.
>>
>> K12: Not sure of concrete usage for now, just a sugar feature. I suppose
>> we
>> can imagine some use case where you need/want only the first bytes and do
>> not want to waste memory allocating the whole payload (SchemaRegistry's
>> schema Id or something similar).
>>
>> K13: The old constructor is not needed anymore. It is just for
>> compatibility until removed in a major version. As public we might have
>> some users using it even if I cannot see any valid reason for this.
>>
>> Thanks,
>> Fred
>>
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-30 Thread Sophie Blee-Goldman
Hey Fred, I think this is looking good but I just want to follow up on what
Kirk asked earlier about having both the ByteBuffer and byte[] forms. Can't
users just use the ByteBuffer versions and convert them into a byte[]
themselves? In some cases it maybe makes sense to offer some additional
APIs if there is complex processing involved in converting between returned
types, but ByteBuffer --> byte[] seems pretty straightforward to me :)

Generally speaking we try to keep the APIs as tight as possible and offer
only what is necessary, and I'd rather leave off "syntactic sugar" type
APIs unless there is a clear need. Put another way: it's easy to add
additional methods if someone wants them, but it's much harder to remote
methods since we have to go through a deprecation cycle. So I'd prefer to
just keep only the ByteBuffer versions (or only the byte[] -- don't
personally care which of the two)

One more small nit: since we're deprecating the old exception constructor,
can you list that in the "Compatibility, Deprecation, and Migration Plan"
section?



On Wed, Apr 24, 2024 at 5:35 AM Frédérik Rouleau
 wrote:

> Hi,
>
> I have updated the KIP now and the latest version of PR is available.
>
> About Kirk's questions:
>
> K11: Yes, both can have a deserialization exception but we deserialize the
> key first, so if an error occurs then, we do not try to deserialize the
> value. So the exception raised is always for key or value.
>
> K12: Not sure of concrete usage for now, just a sugar feature. I suppose we
> can imagine some use case where you need/want only the first bytes and do
> not want to waste memory allocating the whole payload (SchemaRegistry's
> schema Id or something similar).
>
> K13: The old constructor is not needed anymore. It is just for
> compatibility until removed in a major version. As public we might have
> some users using it even if I cannot see any valid reason for this.
>
> Thanks,
> Fred
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-24 Thread Frédérik Rouleau
Hi,

I have updated the KIP now and the latest version of PR is available.

About Kirk's questions:

K11: Yes, both can have a deserialization exception but we deserialize the
key first, so if an error occurs then, we do not try to deserialize the
value. So the exception raised is always for key or value.

K12: Not sure of concrete usage for now, just a sugar feature. I suppose we
can imagine some use case where you need/want only the first bytes and do
not want to waste memory allocating the whole payload (SchemaRegistry's
schema Id or something similar).

K13: The old constructor is not needed anymore. It is just for
compatibility until removed in a major version. As public we might have
some users using it even if I cannot see any valid reason for this.

Thanks,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-23 Thread Kirk True
Hi Fred,

Thanks for the updates!

Questions:

K11. Can we reconsider the introduction of two new exception subclasses? 
Perhaps I don’t understand the benefit? Technically both the key and the value 
could have deserialization errors, right?

K12. Is there a benefit to exposing both the ByteBuffer and byte[]?

K13. Isn’t it possible for the key() and/or value() calls to throw NPE? (Or 
maybe I don’t understand why we need the old constructor :(

Thanks,
Kirk

> On Apr 23, 2024, at 12:45 AM, Frédérik Rouleau 
>  wrote:
> 
> Hi Andrew,
> 
> A1. I will change the order of arguments to match.
> A2 and A3, Yes the KIP is not updated yet as I do not have a wiki account.
> So I must rely on some help to do those changes, which add some delay. I
> will try to find someone available ASAP.
> A4. I had the same thought. Using keyBuffer and valueBuffer for the
> constructor seems fine for me. If no one disagrees, I will do that change.
> 
> Thanks,
> Fred



Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-23 Thread Frédérik Rouleau
Hi Andrew,

A1. I will change the order of arguments to match.
A2 and A3, Yes the KIP is not updated yet as I do not have a wiki account.
So I must rely on some help to do those changes, which add some delay. I
will try to find someone available ASAP.
A4. I had the same thought. Using keyBuffer and valueBuffer for the
constructor seems fine for me. If no one disagrees, I will do that change.

Thanks,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-23 Thread Andrew Schofield
Hi Fred,
Just reviewing the KIP again now that the discussion has quietened down a 
little.
It will soon be ready for a vote I think. I have a few comments about details.

A1) The signature of the new constructor for RecordDeserializationException
needs to be updated accord to the discussion. I see that you have a PR which
is much closer to what I expected. Personally, I think the arguments for the 
constructor
which represent the portions of the record should match the order for the 
constructor
of ConsumerRecord. We’ve already worked out the order of these things once so I
would go for consistency. I suggest:

public RecordDeserializationException(
TopicPartition partition,
long offset,
long timestamp,
TimestampType timestampType,
ByteBuffer key,
ByteBuffer value,
Headers headers,
String message,
Throwable cause);

A2) There are still references to the Record class in the KIP, but we decided 
not
to use it.

A3) There is also a reference to a getConsumerRecord() method which is now to
be replaced by individual methods for the portions of the record, such as:
byte[] value();

The KIP should have a complete and accurate description of the Java interface
changes so please fill in the details.

A4) Given that the key and value are provided to the constructor as ByteBuffer 
but
lazily converted into byte[] as required, I wonder whether the names of the 
methods
and the constructor arguments should be slightly different, just a keyBuffer 
for the
constructure and key() for the getter. Maybe you prefer to keep them the same 
and
I’m happy with that. Just offering a suggestion.


Thanks for the KIP. I think it’s a worthwhile improvement and I expect it’s 
almost there.

Thanks,
Andrew



> On 19 Apr 2024, at 18:59, Frédérik Rouleau  
> wrote:
>
> Hi everyone,
>
> Thanks for all that valuable feedback.
> So we have a consensus not to use Record.
>
> I have updated to PR by creating 2 childs classes
> KeyDeserializationException and ValueDeserializationException. Those
> classes directly embed the required fields. I do not think a wrapper object
> would be useful in that case.
> I still had to update checkstyle as Headers class is not allowed for import
> in the Errors package. I do not think it's an issue to add that
> authorization as Headers is already used in consumerRecord, so already
> public.
>
> The proposed PR https://github.com/apache/kafka/pull/15691/files
>
> If it's ok I will update the KIP.
>
> Regards,
> Fred



Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-19 Thread Frédérik Rouleau
Hi everyone,

Thanks for all that valuable feedback.
So we have a consensus not to use Record.

I have updated to PR by creating 2 childs classes
KeyDeserializationException and ValueDeserializationException. Those
classes directly embed the required fields. I do not think a wrapper object
would be useful in that case.
I still had to update checkstyle as Headers class is not allowed for import
in the Errors package. I do not think it's an issue to add that
authorization as Headers is already used in consumerRecord, so already
public.

The proposed PR https://github.com/apache/kafka/pull/15691/files

If it's ok I will update the KIP.

Regards,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Sophie Blee-Goldman
Thanks for all the context everyone. I think in light of all the
information
that's been shared, I would agree with David that neither the Record
nor ConsumerRecord feels appropriate and that we should just create
a new class/interface that holds the info we want to expose. We
definitely don't want to add each item as a separate parameter since
that will make evolution of this API difficult to do without being a hassle
for users (ie migrating off deprecated APIs and/or blowing up the API
surface area).

I also like the idea of using this to indicate what/where the actual
exception occurred. Alternatively, we could literally extend the
RecordDeserializationException exception class and have a separate
exception type to indicate whether the failure occurred when deserializing
the key or value. Not necessarily saying this is better than just adding
info to the container class/interface, just listing all the options. My
impression is that we tend to favor presenting info in exceptions by
extending the exception type itself rather than adding data to the
exception class. I'm not sure I even agree with that pattern, but that's
been my observation so far.

On Thu, Apr 18, 2024 at 10:47 AM David Arthur
 wrote:

> Hi Fred, thanks for the KIP. Seems like a useful improvement.
>
> As others have mentioned, I think we should avoid exposing Record in this
> way.
>
> Using ConsumerRecord seems okay, but maybe not the best fit for this case
> (for the reasons Matthias gave).
>
> Maybe we could create a new container interface to hold the partially
> deserialized data? This could also indicate to the exception handler
> whether the key, the value, or both had deserialization errors.
>
> Thanks,
> David
>
> On Thu, Apr 18, 2024 at 10:16 AM Frédérik Rouleau
>  wrote:
>
> > Hi,
> >
> > But I guess my main question is really about what metadata we really
> > > want to add to `RecordDeserializationException`? `Record` expose all
> > > kind of internal (serialization) metadata like `keySize()`,
> > > `valueSize()` and many more. For the DLQ use-case it seems we don't
> > > really want any of these? So I am wondering if just adding
> > > key/value/ts/headers would be sufficient?
> > >
> >
> > I think that key/value/ts/headers, topicPartition and offset are all we
> > need. I do not see any usage for other metadata. If someone has a use
> case,
> > I would like to know it.
> >
> > So in that case we can directly add the data into the exception. We can
> > keep ByteBuffer for the local field instead of byte[], that will avoid
> > memory allocation if users do not require it.
> > I wonder if we should return the ByteBuffer or directly the byte[] (or
> both
> > ?) which is more convenient for end users. Any thoughts?
> > Then we can have something like:
> >
> > public RecordDeserializationException(TopicPartition partition,
> >  long offset,
> >  ByteBuffer key,
> >  ByteBuffer value,
> >  Header[] headers,
> >  long timestamp,
> >  String message,
> >  Throwable cause);
> >
> > public TopicPartition topicPartition();
> >
> > public long offset();
> >
> > public long timestamp();
> >
> > public byte[] key(); // Will allocate the array on call
> >
> > public byte[] value(); // Will allocate the array on call
> >
> > public Header[] headers();
> >
> >
> >
> > Regards,
> > Fred
> >
>
>
> --
> -David
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread David Arthur
Hi Fred, thanks for the KIP. Seems like a useful improvement.

As others have mentioned, I think we should avoid exposing Record in this
way.

Using ConsumerRecord seems okay, but maybe not the best fit for this case
(for the reasons Matthias gave).

Maybe we could create a new container interface to hold the partially
deserialized data? This could also indicate to the exception handler
whether the key, the value, or both had deserialization errors.

Thanks,
David

On Thu, Apr 18, 2024 at 10:16 AM Frédérik Rouleau
 wrote:

> Hi,
>
> But I guess my main question is really about what metadata we really
> > want to add to `RecordDeserializationException`? `Record` expose all
> > kind of internal (serialization) metadata like `keySize()`,
> > `valueSize()` and many more. For the DLQ use-case it seems we don't
> > really want any of these? So I am wondering if just adding
> > key/value/ts/headers would be sufficient?
> >
>
> I think that key/value/ts/headers, topicPartition and offset are all we
> need. I do not see any usage for other metadata. If someone has a use case,
> I would like to know it.
>
> So in that case we can directly add the data into the exception. We can
> keep ByteBuffer for the local field instead of byte[], that will avoid
> memory allocation if users do not require it.
> I wonder if we should return the ByteBuffer or directly the byte[] (or both
> ?) which is more convenient for end users. Any thoughts?
> Then we can have something like:
>
> public RecordDeserializationException(TopicPartition partition,
>  long offset,
>  ByteBuffer key,
>  ByteBuffer value,
>  Header[] headers,
>  long timestamp,
>  String message,
>  Throwable cause);
>
> public TopicPartition topicPartition();
>
> public long offset();
>
> public long timestamp();
>
> public byte[] key(); // Will allocate the array on call
>
> public byte[] value(); // Will allocate the array on call
>
> public Header[] headers();
>
>
>
> Regards,
> Fred
>


-- 
-David


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Matthias J. Sax
Andrew, thanks for the details about Consumer internals. That's super 
useful for this discussion! -- And it confirms my understanding.


I don't think we want to use ConsumerRecord type thought, 
because for a DLQ the handler wants to write the message into some DLQ 
topic, and thus needs the key and value, so only 
`ConsumerRecord` would work (or maybe `ByteBuffer>`).


While I would be ok with using `ConsumerRecord`, I don't see a huge 
advantage compared to passing in all fields we are interested in 
one-by-one. In the end, if the data is written into a DLQ topic, the 
`ConsumerRecord` object cannot be reused (but the handler will build a 
`ProducerRecord`), and `ConsumerRecord` would "just" be a container -- I 
don't think it would simplify user-code or provide any other benefit, 
but just add an (unnecessary?) level wrapping/indirection?


The only advantage I would see, is for the case that new interesting 
metadata fields get added to the message format -- for this case, using 
`ConsumerRecord` would automatically include these new fields, and we 
don't need to modify the exception class to add them explicitly. But as 
this happens very rarely, it does not seem to provide a huge benefit.


In the end, I would be fine either way. Curious to hear what others think.


-Matthias



On 4/18/24 8:41 AM, Andrew Schofield wrote:

Hi,
Thanks for the KIP. I think it’s an interesting idea and it seems to work 
nicely with how
the clients work today.

Recently, I’ve been digging in to the record deserialization code in the 
consumer as
part of implementing KIP-932. It’s pretty nasty in there.

There are essentially two kinds of problems that can be encountered when
converting the response from a Fetch RPC into ConsumerRecords.

1) The batch might be corrupt, so you can’t even work out what records are in 
there.
2) Individual records cannot be deserialized.

In the second case, the consumer accumulates records from a Fetch response until
it hits a record which it cannot deserialize. If it has any parsed records 
already to return
to the application, it parks the RecordDeserializationException and returns the 
records
so far. Then, on the next poll, because there are no parsed records waiting, it 
throws
the RecordDeserializationException. And so it continues until all fetched 
records are
processed.

I don’t really think of org.apache.kafka.common.record.Record as an external 
interface.
I think it’s a bit close to the wire to be putting on a user-facing interface.

We do know almost everything about the bad record at the point where the
deserialization fails. I wonder whether actually we could use
ConsumerRecord or even ConsumerRecord :)
in the constructor for the RecordDeserializationException. I don’t really like 
having
a long list of each of the individual items of the record’s parts like 
timestamp and so on.
It’s nicer to have an interface to the record that we can evolve without having 
to change
the constructor for this exception.

Thanks,
Andrew


On 18 Apr 2024, at 15:13, Frédérik Rouleau  
wrote:

Hi,

But I guess my main question is really about what metadata we really

want to add to `RecordDeserializationException`? `Record` expose all
kind of internal (serialization) metadata like `keySize()`,
`valueSize()` and many more. For the DLQ use-case it seems we don't
really want any of these? So I am wondering if just adding
key/value/ts/headers would be sufficient?



I think that key/value/ts/headers, topicPartition and offset are all we
need. I do not see any usage for other metadata. If someone has a use case,
I would like to know it.

So in that case we can directly add the data into the exception. We can
keep ByteBuffer for the local field instead of byte[], that will avoid
memory allocation if users do not require it.
I wonder if we should return the ByteBuffer or directly the byte[] (or both
?) which is more convenient for end users. Any thoughts?
Then we can have something like:

public RecordDeserializationException(TopicPartition partition,
 long offset,
 ByteBuffer key,
 ByteBuffer value,
 Header[] headers,
 long timestamp,
 String message,
 Throwable cause);

public TopicPartition topicPartition();

public long offset();

public long timestamp();

public byte[] key(); // Will allocate the array on call

public byte[] value(); // Will allocate the array on call

public Header[] headers();



Regards,
Fred




Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Andrew Schofield
Hi,
Thanks for the KIP. I think it’s an interesting idea and it seems to work 
nicely with how
the clients work today.

Recently, I’ve been digging in to the record deserialization code in the 
consumer as
part of implementing KIP-932. It’s pretty nasty in there.

There are essentially two kinds of problems that can be encountered when
converting the response from a Fetch RPC into ConsumerRecords.

1) The batch might be corrupt, so you can’t even work out what records are in 
there.
2) Individual records cannot be deserialized.

In the second case, the consumer accumulates records from a Fetch response until
it hits a record which it cannot deserialize. If it has any parsed records 
already to return
to the application, it parks the RecordDeserializationException and returns the 
records
so far. Then, on the next poll, because there are no parsed records waiting, it 
throws
the RecordDeserializationException. And so it continues until all fetched 
records are
processed.

I don’t really think of org.apache.kafka.common.record.Record as an external 
interface.
I think it’s a bit close to the wire to be putting on a user-facing interface.

We do know almost everything about the bad record at the point where the
deserialization fails. I wonder whether actually we could use
ConsumerRecord or even ConsumerRecord :)
in the constructor for the RecordDeserializationException. I don’t really like 
having
a long list of each of the individual items of the record’s parts like 
timestamp and so on.
It’s nicer to have an interface to the record that we can evolve without having 
to change
the constructor for this exception.

Thanks,
Andrew

> On 18 Apr 2024, at 15:13, Frédérik Rouleau  
> wrote:
> 
> Hi,
> 
> But I guess my main question is really about what metadata we really
>> want to add to `RecordDeserializationException`? `Record` expose all
>> kind of internal (serialization) metadata like `keySize()`,
>> `valueSize()` and many more. For the DLQ use-case it seems we don't
>> really want any of these? So I am wondering if just adding
>> key/value/ts/headers would be sufficient?
>> 
> 
> I think that key/value/ts/headers, topicPartition and offset are all we
> need. I do not see any usage for other metadata. If someone has a use case,
> I would like to know it.
> 
> So in that case we can directly add the data into the exception. We can
> keep ByteBuffer for the local field instead of byte[], that will avoid
> memory allocation if users do not require it.
> I wonder if we should return the ByteBuffer or directly the byte[] (or both
> ?) which is more convenient for end users. Any thoughts?
> Then we can have something like:
> 
> public RecordDeserializationException(TopicPartition partition,
> long offset,
> ByteBuffer key,
> ByteBuffer value,
> Header[] headers,
> long timestamp,
> String message,
> Throwable cause);
> 
> public TopicPartition topicPartition();
> 
> public long offset();
> 
> public long timestamp();
> 
> public byte[] key(); // Will allocate the array on call
> 
> public byte[] value(); // Will allocate the array on call
> 
> public Header[] headers();
> 
> 
> 
> Regards,
> Fred



Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Frédérik Rouleau
Hi,

But I guess my main question is really about what metadata we really
> want to add to `RecordDeserializationException`? `Record` expose all
> kind of internal (serialization) metadata like `keySize()`,
> `valueSize()` and many more. For the DLQ use-case it seems we don't
> really want any of these? So I am wondering if just adding
> key/value/ts/headers would be sufficient?
>

I think that key/value/ts/headers, topicPartition and offset are all we
need. I do not see any usage for other metadata. If someone has a use case,
I would like to know it.

So in that case we can directly add the data into the exception. We can
keep ByteBuffer for the local field instead of byte[], that will avoid
memory allocation if users do not require it.
I wonder if we should return the ByteBuffer or directly the byte[] (or both
?) which is more convenient for end users. Any thoughts?
Then we can have something like:

public RecordDeserializationException(TopicPartition partition,
 long offset,
 ByteBuffer key,
 ByteBuffer value,
 Header[] headers,
 long timestamp,
 String message,
 Throwable cause);

public TopicPartition topicPartition();

public long offset();

public long timestamp();

public byte[] key(); // Will allocate the array on call

public byte[] value(); // Will allocate the array on call

public Header[] headers();



Regards,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-18 Thread Matthias J. Sax

Hi,

I am actually not sure if using `Record` is really the right thing? 
While `Record` is technically public API, it does not seem to be 
intended to be exposed to end users?


But I guess my main question is really about what metadata we really 
want to add to `RecordDeserializationException`? `Record` expose all 
kind of internal (serialization) metadata like `keySize()`, 
`valueSize()` and many more. For the DLQ use-case it seems we don't 
really want any of these? So I am wondering if just adding 
key/value/ts/headers would be sufficient?


The motivation section of the KIP is somewhat spare about DLQ details, 
so it's hard to judge what is needed / useful and would would be a leaky 
abstraction?


About "when we cannot construct a `ConsumerRecord` -- I was not really 
digging into it until know, and was just following Kirks comment 
blindly. But looking into the code, we would only not be able to 
construct a `CosumerRecord` when either key or value deserialization 
fails? But as we would pass in byte[] type it would not matter. -- Kirk 
did also mention a corrupted batch, but it seems for this case we might 
not even hit the deserialization code path, but would error out earlier?


I was also looking into the build setup, and I think the idea of the 
import control is to have some sanity check about import dependencies. I 
currently don't see why we should not add an allow rule for Record.


But if we decide to not pass in Record/ConsumerRecord both questions are 
void anyway. Of course, for this case, we would need to add a getter 
method for each metadata field we add (but I think that would be totally 
ok?)


I also seen know, that the old constructor is deprecated, and thus, I 
think using `Optional` a return type is not required (already reflected 
on the wiki page).


Bottom line seems to be: the motivation about what metadata is needed 
for the DLQ use-case is not described in much detail and thus it's hard 
to judge what the right design might be?


The wiki account thing is unfortunately nothing we can fix on our side. 
We did file a ticket with INFRA team, but need to wait for them to 
address it... In the meantime, if you can provide the missing 
information, and what you want to get edited, I can help to update the 
wiki page accordingly.



-Matthias

On 4/16/24 11:18 AM, Sophie Blee-Goldman wrote:

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 
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
 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

Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-16 Thread Sophie Blee-Goldman
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 
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
>  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
>>
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-16 Thread Sophie Blee-Goldman
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
 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
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-16 Thread Frédérik Rouleau
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


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-16 Thread Sophie Blee-Goldman
As for the ConsumerRecord vs Record thing -- I personally think the
other alternative that Kirk mentioned would make more sense here,
that is, returning a Optional> rather
than changing the type from ConsumerRecord to Record.

I'm not sure why checkstyle is saying we shouldn't use the Record
class, but I'm a bit uncomfortable with ignoring this unless we have
someone who can explain why it's complaining and whether this
applies to our situation or not. I'm worried the Record class has
something to do with the old legacy records and using it here
would be a step backward. Note that most of the classes that
implement the Record interface have Legacy in their name, and
neither the ConsumerRecord nor ProducerRecord that most people
are familiar with extend the Record class.

That said, I literally have no context on the history of Record or why
it's not supposed to be used, so I welcome someone with more info
here to chime in. Without additional context, I'd say we should just
use the ConsumerRecord type as initially proposed, and make the
getter API return an Optional>

I'm also personally unaware of what might cause us to be unable to
form a ConsumerRecord. Matthias and/or Kirk, can you elaborate
on the specific cases we're worried about here? We should really
highlight those in the getter javadocs to explain why it's an Optional.
What fields would/could be missing?

On Tue, Apr 16, 2024 at 10:11 AM Sophie Blee-Goldman 
wrote:

> I think some missing context here (which can maybe be added in the
> Motivation section as background) is that the deserialization is actually
> done within Streams, not within the Consumer. Since the consumers
> in Kafka Streams might be subscribed to multiple topics with different
> data types, it has to use a ByteArrayDeserializer so that the consumer
> just hands back the plain bytes and then Kafka Streams can do all the
> deserialization based on its knowledge of the topic->serde mapping.
>
> Streams will just store the set of records returned by each #poll as plain
> bytes, and only when they are dequeued does it actually attempt to
> deserialize the record. So there's only one record being deserialized at
> a time, and each one can be handled separately by the deserialization
> exception handler. Which is what KIP-334 relies on.
>
> I do think it's worth saying something like that in the KIP somewhere,
> since it's somewhat obscure knowledge of Kafka Streams internals
> that not everyone will immediately know about. Feel free to just copy
> what I wrote or write something more succinct 
>
> By the way, there are some minor formatting typos in the code snippet
> for the KIP. Not a big deal, but if you're editing the KIP anyways you
> might as well fix that
>
> On Tue, Apr 16, 2024 at 1:33 AM Frédérik Rouleau
>  wrote:
>
>> Hi Almog,
>>
>> I think you do not understand the behavior that was introduced with the
>> KIP-334.
>> When you have a DeserializationException, if you set the proper seek call
>> to skip the faulty record, the next poll call will return the remaining
>> records to process and not a new list of records. When the KIP was
>> released, I made a demo project
>> https://github.com/fred-ro/kafka-poison-pills not sure it's still
>> working,
>> I should spend time maintaining it. The only issue with the initial KIP is
>> that you do not have access to the data of the faulty record which makes
>> DLQ implementation quite difficult.
>>
>> Regards,
>> Fred
>>
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-16 Thread Sophie Blee-Goldman
I think some missing context here (which can maybe be added in the
Motivation section as background) is that the deserialization is actually
done within Streams, not within the Consumer. Since the consumers
in Kafka Streams might be subscribed to multiple topics with different
data types, it has to use a ByteArrayDeserializer so that the consumer
just hands back the plain bytes and then Kafka Streams can do all the
deserialization based on its knowledge of the topic->serde mapping.

Streams will just store the set of records returned by each #poll as plain
bytes, and only when they are dequeued does it actually attempt to
deserialize the record. So there's only one record being deserialized at
a time, and each one can be handled separately by the deserialization
exception handler. Which is what KIP-334 relies on.

I do think it's worth saying something like that in the KIP somewhere,
since it's somewhat obscure knowledge of Kafka Streams internals
that not everyone will immediately know about. Feel free to just copy
what I wrote or write something more succinct 

By the way, there are some minor formatting typos in the code snippet
for the KIP. Not a big deal, but if you're editing the KIP anyways you
might as well fix that

On Tue, Apr 16, 2024 at 1:33 AM Frédérik Rouleau
 wrote:

> Hi Almog,
>
> I think you do not understand the behavior that was introduced with the
> KIP-334.
> When you have a DeserializationException, if you set the proper seek call
> to skip the faulty record, the next poll call will return the remaining
> records to process and not a new list of records. When the KIP was
> released, I made a demo project
> https://github.com/fred-ro/kafka-poison-pills not sure it's still working,
> I should spend time maintaining it. The only issue with the initial KIP is
> that you do not have access to the data of the faulty record which makes
> DLQ implementation quite difficult.
>
> Regards,
> Fred
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-16 Thread Frédérik Rouleau
Hi Almog,

I think you do not understand the behavior that was introduced with the
KIP-334.
When you have a DeserializationException, if you set the proper seek call
to skip the faulty record, the next poll call will return the remaining
records to process and not a new list of records. When the KIP was
released, I made a demo project
https://github.com/fred-ro/kafka-poison-pills not sure it's still working,
I should spend time maintaining it. The only issue with the initial KIP is
that you do not have access to the data of the faulty record which makes
DLQ implementation quite difficult.

Regards,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-15 Thread Almog Gavra
Hi Frederik - thanks for the KIP, this will be a fantastic and elegant
addition to Kafka Streams.

I have a higher level question about this, which is that the `poll`
interface returns multiple records and yet the DeserializationException
will be thrown if any record in the batch cannot be processed. I suspect
that makes your example of a DLQ challenging, since it would skip all
records up until the record that could not be deserialized (even if they
were valid). People would have to be careful of polling up until the offset
of the "bad" record...

I don't have a great suggestion for an API that could address this, here
are a few suggestions that come to mind:
1. add an optional callback to `poll` that could specify behavior for
records that fail to deserialize (that way the callback could specify a
return value of "fail" or "ignore" individual failing records within the
batch)
2. have a version of consumer#poll that returns a version of
ConsumerRecords that has two lists: all successfully polled records and all
failed records (the obvious downside is that devs might forget to check the
failed records)
3. have the serialization exception contain all successful and all failed
records (that's just not very elegant).

Anyway, there are many people much smarter than I watching this thread --
they may have better suggestions! (Or I may have misunderstood anything, in
which case please carry on...)

Cheers,
Almog

On Fri, Apr 12, 2024 at 1:26 PM Sophie Blee-Goldman 
wrote:

> I think the bigger question here is: why is checkstyle complaining about
> this import? Does anyone know?
>
> On Thu, Apr 11, 2024 at 11:12 AM Frédérik Rouleau
>  wrote:
>
> > Hi everyone,
> >
> > I have made some changes to take in account comments. I have replaced the
> > ConsumerRecord by Record. As it was not allowed by checkstyle, I have
> > modified its configuration. I hope that's ok.
> > I find this new version better. Thanks for your help.
> >
> > Regards,
> > Fred
> >
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-12 Thread Sophie Blee-Goldman
I think the bigger question here is: why is checkstyle complaining about
this import? Does anyone know?

On Thu, Apr 11, 2024 at 11:12 AM Frédérik Rouleau
 wrote:

> Hi everyone,
>
> I have made some changes to take in account comments. I have replaced the
> ConsumerRecord by Record. As it was not allowed by checkstyle, I have
> modified its configuration. I hope that's ok.
> I find this new version better. Thanks for your help.
>
> Regards,
> Fred
>


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-11 Thread Frédérik Rouleau
Hi everyone,

I have made some changes to take in account comments. I have replaced the
ConsumerRecord by Record. As it was not allowed by checkstyle, I have
modified its configuration. I hope that's ok.
I find this new version better. Thanks for your help.

Regards,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-11 Thread Frédérik Rouleau
Hi Kirk,

I have made the test and I confirm that checkstyle is complaining
(Disallowed import - org.apache.kafka.common.record.Record.) if I use
Record class in the RecordDeserialisationException.
An alternative might be to add key(), value(), headers() etc methods
directly in the exception.

Regards,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-11 Thread Frédérik Rouleau
Thanks for your feedback.

Kirk, that's a good point. I will check if there are other ways of raising
an exception than the deserialisation itself.
About Record, I agree I think it would be a better choice and my initial
version was using it. But then I realised that this class might not be
exposed, at least I had some errors from checkstyle. That solution would
also improve GC pressure if you do not use the record by avoiding the
allocation of useless byte arrays.
Maybe someone can confirm that there is no issue by using the Record class.
Matthias, thanks for your comment. Unfortunately I will have to find
someone to do the changes for me as I was not able to create an account on
the wiki.

Regards,
Fred


Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-10 Thread Matthias J. Sax

Thanks for the KIP Fred.

Couple of nits: it's not clear from the "Public API" section what is new 
and what is existing API w/o going back to the code. For existing 
methods which are not changed, it's also best to actually omit them. -- 
It would also be best to only put the interface itself down, but not the 
implementation (ie, no private members and no method body).


Thus, it might be better to do something like this:

+

public class RecordDeserializationException extends SerializationException {

   // newly added
   public RecordDeserializationException(TopicPartition partition,
 ConsumerRecord 
record,

 String message,
 Throwable cause);

   public ConsumerRecord getConsumerRecord();
}

+

From the description it's not clear to me if you propose to change the 
existing constructor, or propose to add a new constructor. From a 
compatibility POV, we cannot really change the existing constructor (but 
we could deprecate it and remove in the future (and add a new one in 
parallel). But I also agree with Kirk that there could be cases for 
which we cannot pass in a `ConsumerRecord` and thus keeping the old 
constructor could make sense (and change the new getter to return an 
`Optinal`).


Another small thing: in Kafka, getter methods are not using a `get` 
prefix, and thus it should be `consumerRecord()` w/o the "get".




-Matthias


On 4/10/24 4:21 PM, Kirk True wrote:

Hi Fred,

Thanks for the KIP!

Questions/comments:

How do we handle the case where CompletedFetch.parseRecord isn’t able to 
construct a well-formed ConsumerRecord (i.e. the values it needs are 
missing/corrupted/etc.)?
Please change RecordDeserializationException’s getConsumerRecord() method to be 
named consumerRecord() to be consistent.
Should we change the return type of consumerRecord() to be 
Optional> in the cases where even a “raw” 
ConsumerRecord can’t be created?
To avoid the above, does it make sense to include a Record object instead of a 
ConsumerRecord? The former doesn’t include the leaderEpoch or TimestampType, 
but maybe that’s OK?

Thanks,
Kirk


On Apr 10, 2024, at 8:47 AM, Frédérik Rouleau  
wrote:

Hi everyone,

To make implementation of DLQ in consumer easier, I would like to add the
raw ConsumerRecord into the RecordDeserializationException.

Details are in KIP-1036

.

Thanks for your feedback.

Regards,
Fred





Re: [DISCUSS] KIP-1036: Extend RecordDeserializationException exception

2024-04-10 Thread Kirk True
Hi Fred,

Thanks for the KIP!

Questions/comments:

How do we handle the case where CompletedFetch.parseRecord isn’t able to 
construct a well-formed ConsumerRecord (i.e. the values it needs are 
missing/corrupted/etc.)?
Please change RecordDeserializationException’s getConsumerRecord() method to be 
named consumerRecord() to be consistent.   
Should we change the return type of consumerRecord() to be 
Optional> in the cases where even a “raw” 
ConsumerRecord can’t be created?
To avoid the above, does it make sense to include a Record object instead of a 
ConsumerRecord? The former doesn’t include the leaderEpoch or TimestampType, 
but maybe that’s OK?

Thanks,
Kirk

> On Apr 10, 2024, at 8:47 AM, Frédérik Rouleau  
> wrote:
> 
> Hi everyone,
> 
> To make implementation of DLQ in consumer easier, I would like to add the
> raw ConsumerRecord into the RecordDeserializationException.
> 
> Details are in KIP-1036
> 
> .
> 
> Thanks for your feedback.
> 
> Regards,
> Fred