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 Tambe wrote:
> @Ted, We th
@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.t
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)
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 Tambe wrote:
> @Jun, Until we make idempotent producer the default (kip-185), this kip is
> sensitive to ret
@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 +
ret
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
+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 t
+1. Thanks for the KIP, Sumant and Joel.
On Fri, Sep 8, 2017 at 11:33 AM, Jason Gustafson wrote:
> +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)
+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 val
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
>
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 afte
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 Tambe wrote:
> I added the following: Due to change in the default value of retries from 0
> to MAX_INT and the existing default value of
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 men
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.
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
15 matches
Mail list logo