Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
To me, the validation you mentioned is not user friendly, assuming the values of linger.ms, request.timeout.ms, retry.backoff.ms are from cluster running previous release of Kafka. For fresh installation, the validation is fine. On Mon, Sep 11, 2017 at 2:50 PM, Sumant Tambewrote: > @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 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 > 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 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 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 > > 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 > > > > 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 > > > > > > > > > > > > > > > > > > > > > > > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
@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 Yuwrote: > 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 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 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 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 > 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 > > > 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 > > > > > > > > > > > > > > > > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
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 Tambewrote: > @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 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 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 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 > > 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 > > > > > > > > > > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
Sumant, Thanks for the explanation. Since we default retries to MAX_INT, it seems that by default, the expiration shouldn't be out of order. Jun On Mon, Sep 11, 2017 at 10:06 AM, Sumant Tambewrote: > @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 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 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 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 > > 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 > > > > > > > > > > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
@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 Raowrote: > 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 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 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 > 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 > > > > > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
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 Yuwrote: > +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 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 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 > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
+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 Jumawrote: > 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 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 > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
+1. Thanks for the KIP, Sumant and Joel. On Fri, Sep 8, 2017 at 11:33 AM, Jason Gustafsonwrote: > +1. Thanks for the KIP. > > On Fri, Sep 8, 2017 at 8:17 AM, Sumant Tambe wrote: > > > Updated. > > > > On 8 September 2017 at 02:04, Ismael Juma 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 > 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 > > > > > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
+1. Thanks for the KIP. On Fri, Sep 8, 2017 at 8:17 AM, Sumant Tambewrote: > Updated. > > On 8 September 2017 at 02:04, Ismael Juma 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 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 > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
Updated. On 8 September 2017 at 02:04, Ismael Jumawrote: > 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 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 > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
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 Tambewrote: > 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 >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
Yes, you are right. It is worth mentioning since the default value of the retries config is changing. Thanks, Apurva On Thu, Sep 7, 2017 at 10:25 PM, Sumant Tambewrote: > I added the following: Due to change in the default value of retries from 0 > to MAX_INT and the existing default value of > max.in.flight.request.per.connection==5, reordering becomes a possibility > by default. To prevent reordering, set > max.in.flight.request.per.connection==1. > > It does not hurt to mention it as it's a default behavior change? > > On 7 September 2017 at 22:15, Apurva Mehta wrote: > > > Thanks for the KIP Sumant, +1 from me. > > > > That is the most exhaustive 'Rejected Alternatives' section that I have > > seen :) > > > > One minor point: In the compatibility section, your note on > > 'max.in.flight.requests.per.connection == 5' resulting in out of order > > delivery is true irrespective of these changes. As such, I don't think it > > should be mentioned in the context of this KIP. > > > > Thanks, > > Apurva > > > > On Thu, Sep 7, 2017 at 10:06 PM, Sumant Tambe 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 > > > > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
I added the following: Due to change in the default value of retries from 0 to MAX_INT and the existing default value of max.in.flight.request.per.connection==5, reordering becomes a possibility by default. To prevent reordering, set max.in.flight.request.per.connection==1. It does not hurt to mention it as it's a default behavior change? On 7 September 2017 at 22:15, Apurva Mehtawrote: > Thanks for the KIP Sumant, +1 from me. > > That is the most exhaustive 'Rejected Alternatives' section that I have > seen :) > > One minor point: In the compatibility section, your note on > 'max.in.flight.requests.per.connection == 5' resulting in out of order > delivery is true irrespective of these changes. As such, I don't think it > should be mentioned in the context of this KIP. > > Thanks, > Apurva > > On Thu, Sep 7, 2017 at 10:06 PM, Sumant Tambe 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 > > >
Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
Thanks for the KIP Sumant, +1 from me. That is the most exhaustive 'Rejected Alternatives' section that I have seen :) One minor point: In the compatibility section, your note on 'max.in.flight.requests.per.connection == 5' resulting in out of order delivery is true irrespective of these changes. As such, I don't think it should be mentioned in the context of this KIP. Thanks, Apurva On Thu, Sep 7, 2017 at 10:06 PM, Sumant Tambewrote: > 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 >
[VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer
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