Thanks for the comments, Ewen. Responses inline.

> 1. Re: the mention of exactly once, this is within a producer session,
> right? And so really only idempotent. Applications still need to take extra
> steps for exactly once if they, e.g., are producing data from some other
> log like a DB txn log.
>
>
Yes, this is really idempotence. What it means that a single acknowledged
send will result in a single copy of the message in the log. Further, a
sequence of sends to a partition will appear in the log in the order they
were sent.

I think it is worth clarifying this in the KIP, and will do so.


> 2.
>
> > Further, the results above show that there is a large improvement in
> throughput and latency when we go from max.in.flight=1 to max.in.flight=2,
> but then there no discernible difference for higher values of this setting.
>
> If in the tests there's no difference with higher values, perhaps leaving
> it alone is better. There are a bunch of other configs we expose and this
> test only evaluates one environment. Without testing things like cross-DC
> traffic, I'd be wary of jumping to the conclusion that max.in.flight > 2
> never makes a difference and that some people aren't already relying on a
> larger default OOTB.
>
>
Are you suggesting to leave the default at 5? I think that is fair.
However, we would need to bound the value of this variable, in order to
make idempotence actually work. Particularly, if max.inflight = N, then we
need to preserve the record metadata (offset, timestamp, sequence) of the
last N appends per partition in order to be able to de-duplicate .

The proposal is to to default this value to 2, but allow any value up to 5.

I would also be interested to learn if there are actual cross-dc use cases
which set max.inflight > 1 today since any higher value means that you lose
ordering guarantees, meaning the two DC's are not guaranteed to be exact
replicas of each other.


> 3. The acks=all change is actually unrelated to the title of the KIP and
> orthogonal to all the other changes. It's also the most risky since
> acks=all needs more network round trips. And while I think it makes sense
> to have the more durable default, this seems like it's actually fairly
> likely to break things for some people (even if a minority of people). This
> one seems like a setting change that needs more sensitive handling, e.g.
> both release notes and log notification that the default is going to
> change, followed by the actual change later.
>
>
Not sure I follow. The KIP proposes to make exactly once delivery the
default behavior. As mentioned above, this means that each acknowledged
send should result in exactly one copy of the message in the log. With
acks=1, we can only ever have at-most once delivery, ie. an acknowledged
send could result in 0 copies of the message in the log. Please let me know
if I have missed something.

Thanks,
Apurva


> -Ewen
>
> On Tue, Aug 8, 2017 at 5:23 PM, Apurva Mehta <apu...@confluent.io> wrote:
>
> > Hi,
> >
> > I've put together a new KIP which proposes to ship Kafka with its
> strongest
> > delivery guarantees by default.
> >
> > We currently ship with at most once semantics and don't provide any
> > ordering guarantees per partition. The proposal is is to provide exactly
> > once in order delivery per partition by default in the upcoming 1.0.0
> > release.
> >
> > The KIP linked to below also outlines the performance characteristics of
> > the proposed default.
> >
> > The KIP is here:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 185%3A+Make+exactly+once+in+order+delivery+per+partition+
> > the+default+producer+setting
> >
> > Please have a look, I would love your feedback!
> >
> > Thanks,
> > Apurva
> >
>

Reply via email to