Just want to clarify that regarding 1), I'm fine with changing it to `all`
but just wanted to argue it is not necessarily correlate with the
exactly-once semantics, but rather on persistence v.s. availability
trade-offs, so I'd like to discuss them separately.

Regarding 2), one minor concern I had is that the enforcement is on the
client side while the parts it affects is on the broker side. I.e. the
broker code would assume at most 5 in.flight when idempotent is turned on,
but this is not enforced at the broker but relying at the client side's
sanity. So other implementations of the client that may not obey this may
likely break the broker code. If we do enforce this we'd better enforce it
at the broker side. Also, I'm wondering if we have considered the approach
for brokers to read the logs in order to get the starting offset when it
does not about it in its snapshot, that whether it is worthwhile if we
assume that such issues are very rare to happen?


Guozhang



On Mon, Aug 14, 2017 at 11:01 AM, Apurva Mehta <apu...@confluent.io> wrote:

> Hello,
>
> I just want to summarize where we are in this discussion
>
> There are two major points of contention: should we have acks=1 or acsk=all
> by default? and how to cap max.in.flight.requests.per.connection?
>
> 1) acks=1 vs acks=all1
>
> Here are the tradeoffs of each:
>
> If you have replication-factor=N, your data is resilient N-1 to disk
> failures. For N>1, here is the tradeoff between acks=1 and acks=all.
>
> With proposed defaults and acks=all, the stock Kafka producer and the
> default broker settings would guarantee that ack'd messages would be in the
> log exactly once.
>
> With the proposed defaults and acks=1, the stock Kafka producer and the
> default broker settings would guarantee that 'retained ack'd messages would
> be in the log exactly once. But all ack'd messages may not be retained'.
>
> If you leave replication-factor=1, acks=1 and acks=all have identical
> semantics and performance, but you are resilient to 0 disk failures.
>
> I think the measured cost (again the performance details are in the wiki)
> of acks=all is well worth the much clearer semantics. What does the rest of
> the community think?
>
> 2) capping max.in.flight at 5 when idempotence is enabled.
>
> We need to limit the max.in.flight for the broker to de-duplicate messages
> properly. The limitation would only apply when idempotence is enabled. The
> shared numbers show that when the client-broker latency is low, there is no
> performance gain for max.inflight > 2.
>
> Further, it is highly debatable that max.in.flight=500 is significantly
> better than max.in.flight=5  for a really high latency client-broker link,
> and so far there are no hard numbers one way or another. However, assuming
> that max.in.flight=500 is significantly better than max.inflight=5 in some
> niche use case, the user would have to sacrifice idempotence for
> throughput. In this extreme corner case, I think it is an acceptable
> tradeoff.
>
> What does the community think?
>
> Thanks,
> Apurva
>



-- 
-- Guozhang

Reply via email to