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

Reply via email to