Apurva,

For the benchmarking, I have a couple of questions:

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.

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.

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.

-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