RE: Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-26 Thread Fq Public
I have started the voting thread here: 
https://lists.apache.org/thread/pn846910dovg3d0z3k8pmq5opj0tb9w5 
Please vote :)

Thanks, 
Farooq

On 2023/02/23 07:27:14 "Matthias J. Sax" wrote:
> Thanks for the KIP. Overall LGTM.
> 
> I think you can start a VOTE.
> 
> 
> -Matthias
> 
> On 2/22/23 5:56 PM, Fq Public wrote:
> > Just wanted to bump this thread for visbility.
> > Thanks to everyone who has participated in the discussion so far.
> > 
> > Thanks,
> > Farooq
> > 
> > On 2023/02/14 19:32:53 Guozhang Wang wrote:
> >> Thanks Farooq, that looks good to me.
> >>
> >> Guozhang
> >>
> >> On Sun, Feb 12, 2023 at 9:01 AM Dharin Shah  wrote:
> >>>
> >>> Hello Farooq,
> >>>
> >>> This is actually a great idea, we have dealt with this by using an array
> >>> instead of a set.
> >>> +1 to this :)
> >>>
> >>> Thank you,
> >>> Dharin
> >>>
> >>> On Sun, Feb 12, 2023 at 8:11 PM Fq Public  wrote:
> >>>
>  Hi Guozhang,
> 
>  Thanks for reading over my proposal!
> 
> > Regarding the format, I'm just thinking if we can change the type of
>  "INT newDataLength" to UINT32?
> 
>  Good idea, I've updated the KIP to reflect UINT32 since it makes clear 
>  the
>  value can never be less than zero.
> 
> > `.equals` default implementation on Object is by reference, so if the
>  groupBy did not generate a new object, that may still pass. This means 
>  that
>  even if user does not implement the `.equals` function, if the same 
>  object
>  is returned then this feature would still be triggered, is that correct?
> 
>  Correct, I've updated the KIP to call out this edge-case clearly as
>  follows:
> 
> > Since the default `.equals` implementation for an `Object`  is by
>  reference, if a user's `groupBy` returns the same reference for the key,
>  then the oldKey and the newKey will naturally `.equals`  each other. This
>  will result in a single event being sent to the repartition topic. This
>  change in behaviour should be considered a "bug-fix" rather than a
>  "breaking change" as the semantics of the operation remain unchanged, the
>  only thing that changes for users is they no longer see transient
>  "inconsistent" states.  In the worst case, users in this situation will
>  need to update any strict tests that check specifically for the presence 
>  of
>  transient "inconsistent" states.
> 
>  What do you think?
> 
>  Thanks,
>  Farooq
> 
>  On 2023/02/07 18:02:24 Guozhang Wang wrote:
> > Hello Farooq,
> >
> > Thanks for the very detailed proposal! I think this is a great idea.
> > Just a few thoughts:
> >
> > 1. I regret that we over-optimized the Changed serde format for
> > footprint while making it less extensible. It seems to me that a two
> > rolling bounce migration is unavoidable.. Regarding the format, I'm
> > just thinking if we can change the type of "INT newDataLength" to
> > UINT32?
> >
> > 2. `.equals` default implementation on Object is by reference, so if
> > the groupBy did not generate a new object, that may still pass. This
> > means that even if user does not implement the `.equals` function, if
> > the same object is returned then this feature would still be
> > triggered, is that correct?
> >
> >
> > Guozhang
> >
> > On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:
> >>
> >> Hi everyone,
> >>
> >> I'd like to share a new KIP for discussion:
> >> https://cwiki.apache.org/confluence/x/P5VbDg
> >>
> >> This could be considered mostly as a "bug fix" but we wanted to raise
>  a KIP
> >> for discussion because it involves changes to the serialization format
>  of
> >> an internal topic which raises backward compatibility considerations.
> >>
> >> Please take a look and let me know what you think.
> >>
> >> Thanks,
> >> Farooq
> >
> 

Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-22 Thread Matthias J. Sax

Thanks for the KIP. Overall LGTM.

I think you can start a VOTE.


-Matthias

On 2/22/23 5:56 PM, Fq Public wrote:

Just wanted to bump this thread for visbility.
Thanks to everyone who has participated in the discussion so far.

Thanks,
Farooq

On 2023/02/14 19:32:53 Guozhang Wang wrote:

Thanks Farooq, that looks good to me.

Guozhang

On Sun, Feb 12, 2023 at 9:01 AM Dharin Shah  wrote:


Hello Farooq,

This is actually a great idea, we have dealt with this by using an array
instead of a set.
+1 to this :)

Thank you,
Dharin

On Sun, Feb 12, 2023 at 8:11 PM Fq Public  wrote:


Hi Guozhang,

Thanks for reading over my proposal!


Regarding the format, I'm just thinking if we can change the type of

"INT newDataLength" to UINT32?

Good idea, I've updated the KIP to reflect UINT32 since it makes clear the
value can never be less than zero.


`.equals` default implementation on Object is by reference, so if the

groupBy did not generate a new object, that may still pass. This means that
even if user does not implement the `.equals` function, if the same object
is returned then this feature would still be triggered, is that correct?

Correct, I've updated the KIP to call out this edge-case clearly as
follows:


Since the default `.equals` implementation for an `Object`  is by

reference, if a user's `groupBy` returns the same reference for the key,
then the oldKey and the newKey will naturally `.equals`  each other. This
will result in a single event being sent to the repartition topic. This
change in behaviour should be considered a "bug-fix" rather than a
"breaking change" as the semantics of the operation remain unchanged, the
only thing that changes for users is they no longer see transient
"inconsistent" states.  In the worst case, users in this situation will
need to update any strict tests that check specifically for the presence of
transient "inconsistent" states.

What do you think?

Thanks,
Farooq

On 2023/02/07 18:02:24 Guozhang Wang wrote:

Hello Farooq,

Thanks for the very detailed proposal! I think this is a great idea.
Just a few thoughts:

1. I regret that we over-optimized the Changed serde format for
footprint while making it less extensible. It seems to me that a two
rolling bounce migration is unavoidable.. Regarding the format, I'm
just thinking if we can change the type of "INT newDataLength" to
UINT32?

2. `.equals` default implementation on Object is by reference, so if
the groupBy did not generate a new object, that may still pass. This
means that even if user does not implement the `.equals` function, if
the same object is returned then this feature would still be
triggered, is that correct?


Guozhang

On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:


Hi everyone,

I'd like to share a new KIP for discussion:
https://cwiki.apache.org/confluence/x/P5VbDg

This could be considered mostly as a "bug fix" but we wanted to raise

a KIP

for discussion because it involves changes to the serialization format

of

an internal topic which raises backward compatibility considerations.

Please take a look and let me know what you think.

Thanks,
Farooq




RE: Re: Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-22 Thread Fq Public
Just wanted to bump this thread for visbility.
Thanks to everyone who has participated in the discussion so far. 

Thanks,
Farooq

On 2023/02/14 19:32:53 Guozhang Wang wrote:
> Thanks Farooq, that looks good to me.
> 
> Guozhang
> 
> On Sun, Feb 12, 2023 at 9:01 AM Dharin Shah  wrote:
> >
> > Hello Farooq,
> >
> > This is actually a great idea, we have dealt with this by using an array
> > instead of a set.
> > +1 to this :)
> >
> > Thank you,
> > Dharin
> >
> > On Sun, Feb 12, 2023 at 8:11 PM Fq Public  wrote:
> >
> > > Hi Guozhang,
> > >
> > > Thanks for reading over my proposal!
> > >
> > > > Regarding the format, I'm just thinking if we can change the type of
> > > "INT newDataLength" to UINT32?
> > >
> > > Good idea, I've updated the KIP to reflect UINT32 since it makes clear the
> > > value can never be less than zero.
> > >
> > > > `.equals` default implementation on Object is by reference, so if the
> > > groupBy did not generate a new object, that may still pass. This means 
> > > that
> > > even if user does not implement the `.equals` function, if the same object
> > > is returned then this feature would still be triggered, is that correct?
> > >
> > > Correct, I've updated the KIP to call out this edge-case clearly as
> > > follows:
> > >
> > > > Since the default `.equals` implementation for an `Object`  is by
> > > reference, if a user's `groupBy` returns the same reference for the key,
> > > then the oldKey and the newKey will naturally `.equals`  each other. This
> > > will result in a single event being sent to the repartition topic. This
> > > change in behaviour should be considered a "bug-fix" rather than a
> > > "breaking change" as the semantics of the operation remain unchanged, the
> > > only thing that changes for users is they no longer see transient
> > > "inconsistent" states.  In the worst case, users in this situation will
> > > need to update any strict tests that check specifically for the presence 
> > > of
> > > transient "inconsistent" states.
> > >
> > > What do you think?
> > >
> > > Thanks,
> > > Farooq
> > >
> > > On 2023/02/07 18:02:24 Guozhang Wang wrote:
> > > > Hello Farooq,
> > > >
> > > > Thanks for the very detailed proposal! I think this is a great idea.
> > > > Just a few thoughts:
> > > >
> > > > 1. I regret that we over-optimized the Changed serde format for
> > > > footprint while making it less extensible. It seems to me that a two
> > > > rolling bounce migration is unavoidable.. Regarding the format, I'm
> > > > just thinking if we can change the type of "INT newDataLength" to
> > > > UINT32?
> > > >
> > > > 2. `.equals` default implementation on Object is by reference, so if
> > > > the groupBy did not generate a new object, that may still pass. This
> > > > means that even if user does not implement the `.equals` function, if
> > > > the same object is returned then this feature would still be
> > > > triggered, is that correct?
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:
> > > > >
> > > > > Hi everyone,
> > > > >
> > > > > I'd like to share a new KIP for discussion:
> > > > > https://cwiki.apache.org/confluence/x/P5VbDg
> > > > >
> > > > > This could be considered mostly as a "bug fix" but we wanted to raise
> > > a KIP
> > > > > for discussion because it involves changes to the serialization format
> > > of
> > > > > an internal topic which raises backward compatibility considerations.
> > > > >
> > > > > Please take a look and let me know what you think.
> > > > >
> > > > > Thanks,
> > > > > Farooq
> > > >
> 

Re: Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-14 Thread Guozhang Wang
Thanks Farooq, that looks good to me.

Guozhang

On Sun, Feb 12, 2023 at 9:01 AM Dharin Shah  wrote:
>
> Hello Farooq,
>
> This is actually a great idea, we have dealt with this by using an array
> instead of a set.
> +1 to this :)
>
> Thank you,
> Dharin
>
> On Sun, Feb 12, 2023 at 8:11 PM Fq Public  wrote:
>
> > Hi Guozhang,
> >
> > Thanks for reading over my proposal!
> >
> > > Regarding the format, I'm just thinking if we can change the type of
> > "INT newDataLength" to UINT32?
> >
> > Good idea, I've updated the KIP to reflect UINT32 since it makes clear the
> > value can never be less than zero.
> >
> > > `.equals` default implementation on Object is by reference, so if the
> > groupBy did not generate a new object, that may still pass. This means that
> > even if user does not implement the `.equals` function, if the same object
> > is returned then this feature would still be triggered, is that correct?
> >
> > Correct, I've updated the KIP to call out this edge-case clearly as
> > follows:
> >
> > > Since the default `.equals` implementation for an `Object`  is by
> > reference, if a user's `groupBy` returns the same reference for the key,
> > then the oldKey and the newKey will naturally `.equals`  each other. This
> > will result in a single event being sent to the repartition topic. This
> > change in behaviour should be considered a "bug-fix" rather than a
> > "breaking change" as the semantics of the operation remain unchanged, the
> > only thing that changes for users is they no longer see transient
> > "inconsistent" states.  In the worst case, users in this situation will
> > need to update any strict tests that check specifically for the presence of
> > transient "inconsistent" states.
> >
> > What do you think?
> >
> > Thanks,
> > Farooq
> >
> > On 2023/02/07 18:02:24 Guozhang Wang wrote:
> > > Hello Farooq,
> > >
> > > Thanks for the very detailed proposal! I think this is a great idea.
> > > Just a few thoughts:
> > >
> > > 1. I regret that we over-optimized the Changed serde format for
> > > footprint while making it less extensible. It seems to me that a two
> > > rolling bounce migration is unavoidable.. Regarding the format, I'm
> > > just thinking if we can change the type of "INT newDataLength" to
> > > UINT32?
> > >
> > > 2. `.equals` default implementation on Object is by reference, so if
> > > the groupBy did not generate a new object, that may still pass. This
> > > means that even if user does not implement the `.equals` function, if
> > > the same object is returned then this feature would still be
> > > triggered, is that correct?
> > >
> > >
> > > Guozhang
> > >
> > > On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:
> > > >
> > > > Hi everyone,
> > > >
> > > > I'd like to share a new KIP for discussion:
> > > > https://cwiki.apache.org/confluence/x/P5VbDg
> > > >
> > > > This could be considered mostly as a "bug fix" but we wanted to raise
> > a KIP
> > > > for discussion because it involves changes to the serialization format
> > of
> > > > an internal topic which raises backward compatibility considerations.
> > > >
> > > > Please take a look and let me know what you think.
> > > >
> > > > Thanks,
> > > > Farooq
> > >


Re: Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-12 Thread Dharin Shah
Hello Farooq,

This is actually a great idea, we have dealt with this by using an array
instead of a set.
+1 to this :)

Thank you,
Dharin

On Sun, Feb 12, 2023 at 8:11 PM Fq Public  wrote:

> Hi Guozhang,
>
> Thanks for reading over my proposal!
>
> > Regarding the format, I'm just thinking if we can change the type of
> "INT newDataLength" to UINT32?
>
> Good idea, I've updated the KIP to reflect UINT32 since it makes clear the
> value can never be less than zero.
>
> > `.equals` default implementation on Object is by reference, so if the
> groupBy did not generate a new object, that may still pass. This means that
> even if user does not implement the `.equals` function, if the same object
> is returned then this feature would still be triggered, is that correct?
>
> Correct, I've updated the KIP to call out this edge-case clearly as
> follows:
>
> > Since the default `.equals` implementation for an `Object`  is by
> reference, if a user's `groupBy` returns the same reference for the key,
> then the oldKey and the newKey will naturally `.equals`  each other. This
> will result in a single event being sent to the repartition topic. This
> change in behaviour should be considered a "bug-fix" rather than a
> "breaking change" as the semantics of the operation remain unchanged, the
> only thing that changes for users is they no longer see transient
> "inconsistent" states.  In the worst case, users in this situation will
> need to update any strict tests that check specifically for the presence of
> transient "inconsistent" states.
>
> What do you think?
>
> Thanks,
> Farooq
>
> On 2023/02/07 18:02:24 Guozhang Wang wrote:
> > Hello Farooq,
> >
> > Thanks for the very detailed proposal! I think this is a great idea.
> > Just a few thoughts:
> >
> > 1. I regret that we over-optimized the Changed serde format for
> > footprint while making it less extensible. It seems to me that a two
> > rolling bounce migration is unavoidable.. Regarding the format, I'm
> > just thinking if we can change the type of "INT newDataLength" to
> > UINT32?
> >
> > 2. `.equals` default implementation on Object is by reference, so if
> > the groupBy did not generate a new object, that may still pass. This
> > means that even if user does not implement the `.equals` function, if
> > the same object is returned then this feature would still be
> > triggered, is that correct?
> >
> >
> > Guozhang
> >
> > On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:
> > >
> > > Hi everyone,
> > >
> > > I'd like to share a new KIP for discussion:
> > > https://cwiki.apache.org/confluence/x/P5VbDg
> > >
> > > This could be considered mostly as a "bug fix" but we wanted to raise
> a KIP
> > > for discussion because it involves changes to the serialization format
> of
> > > an internal topic which raises backward compatibility considerations.
> > >
> > > Please take a look and let me know what you think.
> > >
> > > Thanks,
> > > Farooq
> >


RE: Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-12 Thread Fq Public
Hi Guozhang, 

Thanks for reading over my proposal!

> Regarding the format, I'm just thinking if we can change the type of "INT 
> newDataLength" to UINT32?

Good idea, I've updated the KIP to reflect UINT32 since it makes clear the 
value can never be less than zero. 

> `.equals` default implementation on Object is by reference, so if the groupBy 
> did not generate a new object, that may still pass. This means that even if 
> user does not implement the `.equals` function, if the same object is 
> returned then this feature would still be triggered, is that correct?

Correct, I've updated the KIP to call out this edge-case clearly as follows: 

> Since the default `.equals` implementation for an `Object`  is by reference, 
> if a user's `groupBy` returns the same reference for the key, then the oldKey 
> and the newKey will naturally `.equals`  each other. This will result in a 
> single event being sent to the repartition topic. This change in behaviour 
> should be considered a "bug-fix" rather than a "breaking change" as the 
> semantics of the operation remain unchanged, the only thing that changes for 
> users is they no longer see transient "inconsistent" states.  In the worst 
> case, users in this situation will need to update any strict tests that check 
> specifically for the presence of transient "inconsistent" states.

What do you think? 

Thanks, 
Farooq

On 2023/02/07 18:02:24 Guozhang Wang wrote:
> Hello Farooq,
> 
> Thanks for the very detailed proposal! I think this is a great idea.
> Just a few thoughts:
> 
> 1. I regret that we over-optimized the Changed serde format for
> footprint while making it less extensible. It seems to me that a two
> rolling bounce migration is unavoidable.. Regarding the format, I'm
> just thinking if we can change the type of "INT newDataLength" to
> UINT32?
> 
> 2. `.equals` default implementation on Object is by reference, so if
> the groupBy did not generate a new object, that may still pass. This
> means that even if user does not implement the `.equals` function, if
> the same object is returned then this feature would still be
> triggered, is that correct?
> 
> 
> Guozhang
> 
> On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:
> >
> > Hi everyone,
> >
> > I'd like to share a new KIP for discussion:
> > https://cwiki.apache.org/confluence/x/P5VbDg
> >
> > This could be considered mostly as a "bug fix" but we wanted to raise a KIP
> > for discussion because it involves changes to the serialization format of
> > an internal topic which raises backward compatibility considerations.
> >
> > Please take a look and let me know what you think.
> >
> > Thanks,
> > Farooq
> 

Re: [DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-07 Thread Guozhang Wang
Hello Farooq,

Thanks for the very detailed proposal! I think this is a great idea.
Just a few thoughts:

1. I regret that we over-optimized the Changed serde format for
footprint while making it less extensible. It seems to me that a two
rolling bounce migration is unavoidable.. Regarding the format, I'm
just thinking if we can change the type of "INT newDataLength" to
UINT32?

2. `.equals` default implementation on Object is by reference, so if
the groupBy did not generate a new object, that may still pass. This
means that even if user does not implement the `.equals` function, if
the same object is returned then this feature would still be
triggered, is that correct?


Guozhang

On Mon, Feb 6, 2023 at 5:05 AM Fq Public  wrote:
>
> Hi everyone,
>
> I'd like to share a new KIP for discussion:
> https://cwiki.apache.org/confluence/x/P5VbDg
>
> This could be considered mostly as a "bug fix" but we wanted to raise a KIP
> for discussion because it involves changes to the serialization format of
> an internal topic which raises backward compatibility considerations.
>
> Please take a look and let me know what you think.
>
> Thanks,
> Farooq


[DISCUSS] KIP-904 Kafka Streams - Guarantee subtractor is called before adder if key has not changed

2023-02-06 Thread Fq Public
Hi everyone,

I'd like to share a new KIP for discussion:
https://cwiki.apache.org/confluence/x/P5VbDg

This could be considered mostly as a "bug fix" but we wanted to raise a KIP
for discussion because it involves changes to the serialization format of
an internal topic which raises backward compatibility considerations.

Please take a look and let me know what you think.

Thanks,
Farooq