Thank you for the comments Guozhang. I’ll leave this KIP out for discussion till the end of the week and then start a vote for this early next week.
Sanjana On Mar 18, 2020, 3:38 PM -0700, Guozhang Wang <wangg...@gmail.com>, wrote: > Hello Sanjana, > > Thanks for the proposed KIP, I think that makes a lot of sense -- as you > mentioned in the motivation, we've indeed seen many issues with regard to > the frequent retries, with bounded exponential backoff in the scenario > where there's a long connectivity issue we would effectively reduce the > request load by 10 given the default configs. > > For higher-level Streams client and Connect frameworks, today we also have > a retry logic but that's used in a slightly different way. For example in > Streams, we tend to handle the retry logic at the thread-level and hence > very likely we'd like to change that mechanism in KIP-572 anyways. For > producer / consumer / admin clients, I think just applying this behavioral > change across these clients makes lot of sense. So I think can just leave > the Streams / Connect out of the scope of this KIP to be addressed in > separate discussions. > > I do not have further comments about this KIP :) LGTM. > > Guozhang > > > On Wed, Mar 18, 2020 at 12:09 AM Sanjana Kaundinya <skaundi...@gmail.com> > wrote: > > > Thanks for the feedback Boyang. > > > > If there’s anyone else who has feedback regarding this KIP, would really > > appreciate it hearing it! > > > > Thanks, > > Sanjana > > > > On Tue, Mar 17, 2020 at 11:38 PM Boyang Chen <bche...@outlook.com> wrote: > > > > > Sounds great! > > > > > > Get Outlook for iOS<https://aka.ms/o0ukef> > > > ________________________________ > > > From: Sanjana Kaundinya <skaundi...@gmail.com> > > > Sent: Tuesday, March 17, 2020 5:54:35 PM > > > To: dev@kafka.apache.org <dev@kafka.apache.org> > > > Subject: Re: [DISCUSS] KIP-580: Exponential Backoff for Kafka Clients > > > > > > Thanks for the explanation Boyang. One of the most common problems that > > we > > > have in Kafka is with respect to metadata fetches. For example, if there > > is > > > a broker failure, all clients start to fetch metadata at the same time > > and > > > it often takes a while for the metadata to converge. In a high load > > > cluster, there are also issues where the volume of metadata has made > > > convergence of metadata slower. > > > > > > For this case, exponential backoff helps as it reduces the retry rate and > > > spaces out how often clients will retry, thereby bringing down the time > > for > > > convergence. Something that Jason mentioned that would be a great > > addition > > > here would be if the backoff should be “jittered” as it was in KIP-144 > > with > > > respect to exponential reconnect backoff. This would help prevent the > > > clients from being synchronized on when they retry, thereby spacing out > > the > > > number of requests being sent to the broker at the same time. > > > > > > I’ll add this example to the KIP and flush out more of the details - so > > > it’s more clear. > > > > > > On Mar 17, 2020, 1:24 PM -0700, Boyang Chen <reluctanthero...@gmail.com > > > , > > > wrote: > > > > Thanks for the reply Sanjana. I guess I would like to rephrase my > > > question > > > > 2 and 3 as my previous response is a little bit unactionable. > > > > > > > > My specific point is that exponential backoff is not a silver bullet > > and > > > we > > > > should consider using it to solve known problems, instead of making the > > > > holistic changes to all clients in Kafka ecosystem. I do like the > > > > exponential backoff idea and believe this would be of great value, but > > > > maybe we should focus on proposing some existing modules that are > > > suffering > > > > from static retry, and only change them in this first KIP. If in the > > > > future, some other component users believe they are also suffering, we > > > > could get more minor KIPs to change the behavior as well. > > > > > > > > Boyang > > > > > > > > On Sun, Mar 15, 2020 at 12:07 AM Sanjana Kaundinya < > > skaundi...@gmail.com > > > > > > > > wrote: > > > > > > > > > Thanks for the feedback Boyang, I will revise the KIP with the > > > > > mathematical relations as per your suggestion. To address your > > > feedback: > > > > > > > > > > 1. Currently, with the default of 100 ms per retry backoff, in 1 > > second > > > > > we would have 10 retries. In the case of using an exponential > > backoff, > > > we > > > > > would have a total of 4 retries in 1 second. Thus we have less than > > > half of > > > > > the amount of retries in the same timeframe and can lessen broker > > > pressure. > > > > > This calculation is done as following (using the formula laid out in > > > the > > > > > KIP: > > > > > > > > > > Try 1 at time 0 ms, failures = 0, next retry in 100 ms (default retry > > > ms > > > > > is initially 100 ms) > > > > > Try 2 at time 100 ms, failures = 1, next retry in 200 ms > > > > > Try 3 at time 300 ms, failures = 2, next retry in 400 ms > > > > > Try 4 at time 700 ms, failures = 3, next retry in 800 ms > > > > > Try 5 at time 1500 ms, failures = 4, next retry in 1000 ms (default > > max > > > > > retry ms is 1000 ms) > > > > > > > > > > For 2 and 3, could you elaborate more about what you mean with > > respect > > > to > > > > > client timeouts? I’m not very familiar with the Streams framework, so > > > would > > > > > love to get more insight to how that currently works, with respect to > > > > > producer transactions, so I can appropriately update the KIP to > > address > > > > > these scenarios. > > > > > On Mar 13, 2020, 7:15 PM -0700, Boyang Chen < > > > reluctanthero...@gmail.com>, > > > > > wrote: > > > > > > Thanks for the KIP Sanjana. I think the motivation is good, but > > lack > > > of > > > > > > more quantitative analysis. For instance: > > > > > > > > > > > > 1. How much retries we are saving by applying the exponential retry > > > vs > > > > > > static retry? There should be some mathematical relations between > > the > > > > > > static retry ms, the initial exponential retry ms, the max > > > exponential > > > > > > retry ms in a given time interval. > > > > > > 2. How does this affect the client timeout? With exponential retry, > > > the > > > > > > client shall be getting easier to timeout on a parent level caller, > > > for > > > > > > instance stream attempts to retry initializing producer > > transactions > > > with > > > > > > given 5 minute interval. With exponential retry this mechanism > > could > > > > > > experience more frequent timeout which we should be careful with. > > > > > > 3. With regards to #2, we should have more detailed checklist of > > all > > > the > > > > > > existing static retry scenarios, and adjust the initial exponential > > > retry > > > > > > ms to make sure we won't get easily timeout in high level due to > > too > > > few > > > > > > attempts. > > > > > > > > > > > > Boyang > > > > > > > > > > > > On Fri, Mar 13, 2020 at 4:38 PM Sanjana Kaundinya < > > > skaundi...@gmail.com> > > > > > > wrote: > > > > > > > > > > > > > Hi Everyone, > > > > > > > > > > > > > > I’ve written a KIP about introducing exponential backoff for > > Kafka > > > > > > > clients. Would appreciate any feedback on this. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients > > > > > > > > > > > > > > Thanks, > > > > > > > Sanjana > > > > > > > > > > > > > > > > > > > > -- > -- Guozhang