Hi Mickael,

Thanks for the KIP.

Case 5 in the KIP seems a bit weird to me:

> > NULL:v0    mykey    myvalue       (5)
> This will result in a record with key="mykey", value="myvalue" and
> headers={"NULL": "v0"} This is because headers cannot have a null key.
>

To me this feels like it's more suited to an error message (like the case
of collision between the null marker and other flags). With the currently
defined semantics this is the only one of the 5 examples where the null
marker doesn't result in a null being written in the expected place. And
it's not like the user couldn't easily select a different null marker to
avoid a collision with a header key. Was there a reason you went for these
semantics?

Kind regards,

Tom

On Mon, 20 Dec 2021 at 17:06, Mickael Maison <mickael.mai...@gmail.com>
wrote:

> Thanks for the feedback!
>
> Chris,
> I've updated the KIP and added examples to show how the null marker
> will be applied.
>
> Luke,
> Good point, I've updating the KIP to mention the user will get an
> error message if the marker collides with other flags.
>
> Thanks,
> Mickael
>
>
> On Thu, Dec 16, 2021 at 2:00 PM Luke Chen <show...@gmail.com> wrote:
> >
> > Hi Mickael,
> >
> > Thanks for the KIP!
> > This will be a helpful feature for debugging, for sure!
> >
> > I have one question:
> > Will we have some safe net for the collision of `key.separator` and the
> new
> > introduced `null.marker`.
> > That is, what if user set the same or overlapped  `key.separator` and
> > `null.marker`, how would we handle it?
> > Ex: key.separator="-", null.marker="--".
> > Maybe it's corner case, but I think it'd be better we handle it
> gracefully.
> >
> > Thank you.
> > Luke
> >
> >
> >
> > On Wed, Dec 15, 2021 at 11:08 PM Chris Egerton
> <chr...@confluent.io.invalid>
> > wrote:
> >
> > > Hi Mickael,
> > >
> > > Thanks for the KIP. Given how important tombstone records are it's
> hard to
> > > believe that the console producer doesn't already support them!
> > >
> > > I wanted to clarify the intended behavior and how it will play with the
> > > parse.key and the newly-introduced (as of KIP-798 [1]) parse.headers
> > > properties. Is the intention that the null.marker should match the
> entire
> > > line read by the console producer, or that it can match individual
> portions
> > > of a line that correspond to the record's key, value, header key, or
> header
> > > value? I imagine so but think it may be worth calling out (and possibly
> > > illustrating with an example or two) in the KIP.
> > >
> > > [1] -
> > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-798%3A+Add+possibility+to+write+kafka+headers+in+Kafka+Console+Producer
> > >
> > > Cheers,
> > >
> > > Chris
> > >
> > > On Wed, Dec 15, 2021 at 6:08 AM Mickael Maison <
> mickael.mai...@gmail.com>
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I opened a KIP to add the option to produce records with a null value
> > > > using the Console Producer:
> > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-810%3A+Allow+producing+records+with+null+values+in+Kafka+Console+Producer
> > > >
> > > > Let me know if you have any feedback.
> > > >
> > > > Thanks,
> > > > Mickael
> > > >
> > >
>
>

Reply via email to