@Ted, We throw a ConfigException when user-configured values of linger.ms, request.timeout.ms, retry.backoff.ms add up to more than delivery.timeout.ms . The kip mentions this in the Validation section.
On 11 September 2017 at 14:31, Ted Yu <yuzhih...@gmail.com> wrote: > bq. larger than default (linger.ms + request.timeout.ms + retry.backoff.ms > ) > > I was not referring to the sum of default values for the above parameters. > I was referring to the sum of user configured values for these parameters > (since we don't know whether that sum is higher than 120 seconds or not) . > > On Mon, Sep 11, 2017 at 10:06 AM, Sumant Tambe <suta...@gmail.com> wrote: > > > @Jun, Until we make idempotent producer the default (kip-185), this kip > is > > sensitive to retries. I.e., we expire batches either delivery.timeout.ms > > passes or all retries are exhausted, whichever comes first. In cases > where > > retries exhaust first due to linger.ms + retries * (request.timeout.ms + > > retry.backoff.ms) being much smaller than delivery.timeout.ms, multiple > > failed requests (due to pipelining) will cause batches to expire > > out-of-order. Right? > > > > @Ted, The idea is to have the default value of delivery.timeout.ms=120 > > sec, > > which is much larger than default (linger.ms + request.timeout.ms + > > retry.backoff.ms). If a user configures them incorrectly, report a > > ConfigException. > > > > > > On 11 September 2017 at 09:11, Jun Rao <j...@confluent.io> wrote: > > > > > Hi, Sumant, > > > > > > Thanks for the KIP. +1. > > > > > > Just a minor clarification. The KIP says "Batches expire in order > > > when max.in.flight.request.per.connection==1". Is that true? It seems > > that > > > even with max.in.flight.request.per.connection > 1, batches should > still > > > expire in order. > > > > > > Jun > > > > > > On Sat, Sep 9, 2017 at 6:15 PM, Ted Yu <yuzhih...@gmail.com> wrote: > > > > > > > +1 for the KIP. > > > > > > > > For delivery.timeout.ms , since it should be >= linger.ms + > > > > request.timeout.ms + retry.backoff.ms , it seems the default value > > > should > > > > be max(120 seconds, linger.ms + request.timeout.ms + > retry.backoff.ms > > ). > > > > > > > > Cheers > > > > > > > > On Fri, Sep 8, 2017 at 2:04 AM, Ismael Juma <ism...@juma.me.uk> > wrote: > > > > > > > > > Thanks for the KIP. +1 (binding) from me. Just a minor suggestion, > I > > > > would > > > > > mention the following under "Public Interfaces": > > > > > > > > > > Default value of delivery.timeout.ms = 120 seconds > > > > > Default value of retries will be changed to MAX_INT > > > > > request.timeout.ms – current meaning, but messages are not expired > > > after > > > > > this time. I.e., request.timeout.ms is no longer relevant for > batch > > > > > expiry. > > > > > > > > > > The compatibility impact of such changes can remain in the > > > compatibility > > > > > section. Also, I agree about keeping your "reordering" text > although > > it > > > > > seems like the wiki wasn't updated to match what you posted in the > > > > > discussion thread. > > > > > > > > > > Ismael > > > > > > > > > > On Fri, Sep 8, 2017 at 6:06 AM, Sumant Tambe <suta...@gmail.com> > > > wrote: > > > > > > > > > > > Hi all, > > > > > > > > > > > > I would like to open the vote for KIP-91: > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-91+ > > > > > > Provide+Intuitive+User+Timeouts+in+The+Producer > > > > > > > > > > > > Thank you all for your input on the kip so far. > > > > > > > > > > > > Regards, > > > > > > Sumant > > > > > > > > > > > > > > > > > > > > >