Hi Luke,

Thank you for your feedback.  I've updated the KIP with your suggestions.

1. Updated with a better example.
2. I removed the reference to ClassicDefaultPartitioner, it was probably
confusing.
3. The logic doesn't rely on checking batches, I've updated the proposal to
make it more explicit.
4. The primary issue (uneven distribution) is described in the linked jira,
copied an example from jira into the KIP as well.

-Artem


On Thu, Nov 4, 2021 at 8:34 PM Luke Chen <show...@gmail.com> wrote:

> Hi Artem,
> Thanks for the KIP! And thanks for reminding me to complete KIP-782, soon.
> :)
>
> Back to the KIP, I have some comments:
> 1. You proposed to have a new config: "partitioner.sticky.batch.size", but
> I can't see how we're going to use it to make the partitioner better.
> Please explain more in KIP (with an example will be better as suggestion
> (4))
> 2. In the "Proposed change" section, you take an example to use
> "ClassicDefaultPartitioner", is that referring to the current default
> sticky partitioner? I think it'd better you name your proposed partition
> with a different name for distinguish between the default one and new one.
> (Although after implementation, we are going to just use the same name)
> 3. So, if my understanding is correct, you're going to have a "batch"
> switch, and before the in-flight is full, it's disabled. Otherwise, we'll
> enable it. Is that right? Sorry, I don't see any advantage of having this
> batch switch. Could you explain more?
> 4. I think it should be more clear if you can have a clear real example in
> the motivation section, to describe what issue we faced using current
> sticky partitioner. And in proposed changes section, using the same
> example, to describe more detail about how you fix this issue with your
> way.
>
> Thank you.
> Luke
>
> On Fri, Nov 5, 2021 at 1:38 AM Artem Livshits
> <alivsh...@confluent.io.invalid> wrote:
>
> > Hello,
> >
> > This is the discussion thread for
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner
> > .
> >
> > The proposal is a bug fix for
> > https://issues.apache.org/jira/browse/KAFKA-10888, but it does include a
> > client config change, therefore we have a KIP to discuss.
> >
> > -Artem
> >
>

Reply via email to