Thank you Sanjana. I checked the updates on the KIP and it looks good to me too now. I'll vote on the voting thread.
Best, Konstantine On Wed, Mar 25, 2020 at 1:57 PM Sanjana Kaundinya <skaundi...@gmail.com> wrote: > Hi Konstantine, > > Thanks for the insightful feedback. I’ll address it here as well as update > the KIP accordingly. > > I think it is important to call out the fact that we are leaving out > Connect and Streams in the proposed changes, so that it can be addressed in > future KIP/changes. As you pointed out, Kafka Connect does utilize > ConsumerNetworkClient and Metadata for its rebalancing protocol, and as a > result the changes made to exponential backoff would affect the > WorkerGroupMember that utilizes these classes. Any Kafka client that > utilizes these classes would be making use of exponential backoff instead > of the current static backoff. > > That being said, although Kafka Connect will be affected with respect to > those two things, not all of the backoff configs are being replaced here. > As you correctly stated, classes such as AbstractCoordinator, > ConsumerCoordinator, and the Heartbeat thread would be utilizing the static > backoff behavior - no changes will be made with respect to rebalancing. > > With respect to Compatibility, I will add into that section the things > I’ve mentioned above - affects to Kafka Connect as well as no changes to > anything related to the rebalance protocol. In addition, the reason why > retry.backoff.max.ms shouldn’t default to the same value as > retry.backoff.ms is that then if a user isn’t aware of this feature and > doesn’t set this, they wouldn’t enjoy the exponential backoff. Instead it’s > important to ensure that we provide this as a default feature for all Kafka > clients. In addition defaulting the retry.backoff.max.ms to 1000 ms > unconditionally wouldn’t give users the flexibility to tune their clients > to their environments. > > Finally, yes you are correct, in order to have exponential backoff, we > actually do need both configs, with retry.backkoff.ms < > retry.backoff.max.ms. I will update the KIP to reflect that as well as > incorporate the wording change you have suggested. > > Thanks, > Sanjana > > On Mar 25, 2020, 10:50 AM -0700, Konstantine Karantasis < > konstant...@confluent.io>, wrote: > > Hi Sanjana and thanks for the KIP! > > > > Sorry for the late response, but I still have a few questions that you > > might find useful. > > > > The KIP currently does not mention Kafka Connect at all. I have read > > the discussion above where it'd been decided to leave Connect and Streams > > out of the proposed changes, but I feel this should be called out > > explicitly. At the same time, Kafka Connect is also a Kafka client that > > uses ConsumerNetworkClient and Metadata for its rebalancing protocol. > It's > > not clear to me whether changes in those classes will affect Connect > > workers. Do you think it's worth clarifying that? > > > > Additionally, you might also want to add a section specifically to > mention > > how this new config affects the places where the current config > > retry.backoff.ms is used today to back-off during rebalancing. Is > > exponential backoff going to replace the old config in those places as > > well? And if it does, should we add a mention that a very high value of > the > > new retry.backoff.max.ms might affect how quickly a consumer or worker > > rejoins their group after it experiences a temporary network partitioning > > from the broker coordinator? > > > > Places that explicitly use retry.backoff.ms at the moment include the > > AbstractCoordinator, the ConsumerCoordinator and the Heartbeat thread. By > > reading the previous discussion, I understand that these classes might > keep > > using the old static backoff. Even if that's the case, I think it's worth > > mentioning that in the KIP for reference. > > > > In the rejected alternatives section, you mention that "existing behavior > > is always maintained: for reasons explained in the compatibility > section.". > > However, the Compatibility section says that there are no compatibility > > concerns. I'd suggest extending the compatibility section to help a bit > > more in explaining why the alternatives were rejected. Also, in the > > compatibility section you mention that the new config ( > retry.backoff.max.ms) > > will replace the old one (retry.backoff.ms), but from reading at the > > beginning, I understand that in order to have exponential increments, you > > actually need both configs, with retry.backoff.ms < retry.backoff.max.ms > . > > Should the mention around replacement be removed? > > > > Finally, I have a minor suggestion that might help explain the following > > sentence better: > > > > "If retry.backoff.ms is set to be greater than retry.backoff.max.ms, > then > > retry.backoff.max.ms will be used as a **constant backoff from the > > beginning without exponential increase**." (highlighting the difference > > only for reference here). Unless I misunderstood how the new backoff will > > be used when it's smaller than the value of the old config, in which case > > it might help clarifying a bit more as well. > > > > > > Thanks for the KIP! > > Really looking forward to more robust retries in Kafka clients > > > > Konstantine > > > > > > On Tue, Mar 24, 2020 at 9:56 AM Guozhang Wang <wangg...@gmail.com> > wrote: > > > > > In Kafka clients, there are cases where we log a warning when > overriding > > > some conflicting configs and in some other cases we throw and let the > > > brokers to die during startup --- you can check the > > > `postProcessParsedConfig` function in Producer/ConsumerConfig for such > > > logic. > > > > > > I think for this case, it is sufficient to log a warning if we find the > > > `max` < `backoff`. > > > > > > > > > Guozhang > > > > > > On Mon, Mar 23, 2020 at 9:18 PM Boyang Chen < > reluctanthero...@gmail.com> > > > wrote: > > > > > > > Got it, although I would still like to be aware of the actual > backoff I > > > > will be using in production, having the app crash seems like an > > > > over-reaction. I don't think I have further questions :) > > > > > > > > On Mon, Mar 23, 2020 at 7:36 PM Sanjana Kaundinya < > skaundi...@gmail.com> > > > > wrote: > > > > > > > > > Hey Sanjana, > > > > > > > > > > Hey Boyang, > > > > > > > > > > If a user provides no config at all then as you mentioned they > will be > > > > > default be able to make use of the exponential back off feature > > > > introduced > > > > > by the KIP. If the backoff.ms is overriden to 2000 ms, the lesser > of > > > > > either > > > > > the max or the computed back off will be chosen, so in this case > the > > > max > > > > > will be chosen as it is 1000 ms. As Guozhang mentioned if the user > > > > > configures something like this then they would notice the behavior > to > > > not > > > > > be in line what they expect and would see the KIP + Release notes > and > > > > know > > > > > to configure it to be backoff.ms < max backoff.ms. I’m not sure > if its > > > > as > > > > > big of an error to reject the configuration if it’s configured like > > > this, > > > > > as the clients could still run in either case. > > > > > > > > > > To answer your second question, we are making the dynamic backoff > the > > > > > default and not allowing for static backoff (unless they set > > > backoff.ms > > > > > > > > > > max.backof.ms, then that would in a sense be static) We will > include > > > > this > > > > > information in the release notes to make sure users are aware of > this > > > > > behavior change. > > > > > > > > > > Thanks, > > > > > Sanjana > > > > > > > > > > On Mon, Mar 23, 2020 at 6:37 PM Boyang Chen < > > > reluctanthero...@gmail.com> > > > > > wrote: > > > > > > > > > > > Hey Sanjana, > > > > > > > > > > > > my understanding with the update is that if a user provides no > config > > > > at > > > > > > all, a Producer/Consumer/Admin client user would by default > enjoying > > > a > > > > > > starting backoff.ms as 100 ms and max.backoff.ms as 1000 ms? If > I > > > > > already > > > > > > override the backoff.ms to 2000 ms for instance, will I be > choosing > > > > the > > > > > > default max.backoff here? > > > > > > > > > > > > I guess my question would be whether we should just reject a > config > > > > with > > > > > > backoff.ms > max.backoff.ms in the first place, as this looks > like > > > > > > mis-configuration to me. > > > > > > > > > > > > Second question is whether we allow fallback to static backoffs > if > > > the > > > > > user > > > > > > wants to do so, or we should just ship this as an opt-in feature? > > > > > > > > > > > > Let me know your thoughts. > > > > > > > > > > > > Boyang > > > > > > > > > > > > On Mon, Mar 23, 2020 at 11:38 AM Cheng Tan <c...@confluent.io> > > > wrote: > > > > > > > > > > > > > +1 (non-binding) > > > > > > > > > > > > > > > On Mar 19, 2020, at 7:27 PM, Sanjana Kaundinya < > > > > skaundi...@gmail.com > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > Ah yes that makes sense. I’ll update the KIP to reflect this. > > > > > > > > > > > > > > > > Thanks, > > > > > > > > Sanjana > > > > > > > > > > > > > > > > On Thu, Mar 19, 2020 at 5:48 PM Guozhang Wang < > > > wangg...@gmail.com> > > > > > > > wrote: > > > > > > > > > > > > > > > > > Following the formula you have in the KIP, if it is simply: > > > > > > > > > > > > > > > > > > MIN(retry.backoff.max.ms, (retry.backoff.ms * > 2**(failures - > > > 1)) > > > > * > > > > > > > random( > > > > > > > > > 0.8, 1.2)) > > > > > > > > > > > > > > > > > > then the behavior would stay consistent at > retry.backoff.max.ms > > > . > > > > > > > > > > > > > > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > > > > On Thu, Mar 19, 2020 at 5:46 PM Sanjana Kaundinya < > > > > > > skaundi...@gmail.com > > > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > If that’s the case then what should we base the starting > point > > > > as? > > > > > > > > > > Currently in the KIP the starting point is > retry.backoff.ms > > > and > > > > it > > > > > > > > > > exponentially goes up to retry.backoff.max.ms. If > > > > > > retry.backoff.max.ms > > > > > > > > > is > > > > > > > > > > smaller than retry.backoff.ms then that could pose a > bit of a > > > > > > problem > > > > > > > > > > there right? > > > > > > > > > > > > > > > > > > > > On Mar 19, 2020, 5:44 PM -0700, Guozhang Wang < > > > > wangg...@gmail.com > > > > > > , > > > > > > > > > wrote: > > > > > > > > > > > Thanks Sanjana, I did not capture the part that Jason > referred > > > > to, > > > > > > so > > > > > > > > > > > that's my bad :P > > > > > > > > > > > > > > > > > > > > > > Regarding your last statement, I actually feel that > instead of > > > > > take > > > > > > > the > > > > > > > > > > > larger of the two, we should respect " > retry.backoff.max.ms" > > > > even > > > > > if > > > > > > > it > > > > > > > > > > is > > > > > > > > > > > smaller than "retry.backoff.ms". I do not have a very > strong > > > > > > > rationale > > > > > > > > > > > except it is logically more aligned to the config > names. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Thu, Mar 19, 2020 at 5:39 PM Sanjana Kaundinya < > > > > > > > > > skaundi...@gmail.com> > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > Hey Jason and Guozhang, > > > > > > > > > > > > > > > > > > > > > > > > Jason is right, I took this inspiration from KIP-144 > ( > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-144%3A+Exponential+backoff+for+broker+reconnect+attempts > > > > > > > > > > > > ) > > > > > > > > > > > > which had the same logic in order to preserve the > existing > > > > > > behavior. > > > > > > > > > In > > > > > > > > > > > > this case however, if we are thinking to completely > eliminate > > > > the > > > > > > > > > > static > > > > > > > > > > > > backoff behavior, we can do that and as Jason > mentioned put > > > it > > > > in > > > > > > the > > > > > > > > > > > > release notes and not add any special logic. In > addition I > > > > agree > > > > > > that > > > > > > > > > > we > > > > > > > > > > > > should take the larger of the two of ` > retry.backoff.ms` and > > > ` > > > > > > > > > > > > retry.backoff.max.ms`. I'll update the KIP to > reflect this > > > and > > > > > > make > > > > > > > > > it > > > > > > > > > > > > clear that the old static retry backoff is getting > replaced > > > by > > > > > the > > > > > > > > > new > > > > > > > > > > > > dynamic retry backoff. > > > > > > > > > > > > > > > > > > > > > > > > Thanks, > > > > > > > > > > > > Sanjana > > > > > > > > > > > > On Thu, Mar 19, 2020 at 4:23 PM Jason Gustafson < > > > > > > ja...@confluent.io> > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > Hey Guozhang, > > > > > > > > > > > > > > > > > > > > > > > > > > I was referring to this: > > > > > > > > > > > > > > > > > > > > > > > > > > > For users who have not set retry.backoff.ms > explicitly, > > > the > > > > > > > > > > default > > > > > > > > > > > > > behavior will change so that the backoff will grow > up to > > > 1000 > > > > > ms. > > > > > > > > > For > > > > > > > > > > > > users > > > > > > > > > > > > > who have set retry.backoff.ms explicitly, the > behavior will > > > > > > remain > > > > > > > > > > the > > > > > > > > > > > > > same > > > > > > > > > > > > > as they could have specific requirements. > > > > > > > > > > > > > > > > > > > > > > > > > > I took this to mean that for users who have > overridden ` > > > > > > > > > > retry.backoff.ms > > > > > > > > > > > > ` > > > > > > > > > > > > > to 50ms (say), we will change the default ` > > > > retry.backoff.max.ms > > > > > ` > > > > > > > > > to > > > > > > > > > > 50ms > > > > > > > > > > > > > as > > > > > > > > > > > > > well in order to preserve existing backoff > behavior. Is that > > > > not > > > > > > > > > > right? > > > > > > > > > > > > In > > > > > > > > > > > > > any case, I agree that we can use the maximum of > the two > > > > values > > > > > as > > > > > > > > > > the > > > > > > > > > > > > > effective `retry.backoff.max.ms` to handle the > case when > > > the > > > > > > > > > > configured > > > > > > > > > > > > > value of `retry.backoff.ms` is larger than the > default of > > > 1s. > > > > > > > > > > > > > > > > > > > > > > > > > > -Jason > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Thu, Mar 19, 2020 at 3:29 PM Guozhang Wang < > > > > > wangg...@gmail.com > > > > > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > Hey Jason, > > > > > > > > > > > > > > > > > > > > > > > > > > > > My understanding is a bit different here: even > if user has > > > an > > > > > > > > > > explicit > > > > > > > > > > > > > > overridden "retry.backoff.ms", the exponential > mechanism > > > > still > > > > > > > > > > > > triggers > > > > > > > > > > > > > > and > > > > > > > > > > > > > > the backoff would be increased till " > retry.backoff.max.ms > > > "; > > > > > and > > > > > > > > > > if the > > > > > > > > > > > > > > specified "retry.backoff.ms" is already larger > than the " > > > > > > > > > > > > > > retry.backoff.max.ms", we would still take " > > > > > retry.backoff.max.ms > > > > > > > > > ". > > > > > > > > > > > > > > > > > > > > > > > > > > > > So if the user does override the " > retry.backoff.ms" to a > > > > value > > > > > > > > > > larger > > > > > > > > > > > > > than > > > > > > > > > > > > > > 1s and is not aware of the new config, she would > be > > > surprised > > > > > to > > > > > > > > > > see > > > > > > > > > > > > the > > > > > > > > > > > > > > specified value seemingly not being respected, > but she > > > could > > > > > > > > > still > > > > > > > > > > > > learn > > > > > > > > > > > > > > that afterwards by reading the release notes > introducing > > > this > > > > > KIP > > > > > > > > > > > > > anyways. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Thu, Mar 19, 2020 at 3:10 PM Jason Gustafson < > > > > > > > > > > ja...@confluent.io> > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > Hi Sanjana, > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > The KIP looks good to me. I had just one > question about > > > the > > > > > > > > > > default > > > > > > > > > > > > > > > behavior. As I understand, if the user has > specified ` > > > > > > > > > > > > retry.backoff.ms > > > > > > > > > > > > > ` > > > > > > > > > > > > > > > explicitly, then we will not apply the default > max > > > backoff. > > > > As > > > > > > > > > > such, > > > > > > > > > > > > > > > there's no way to get the benefit of this > feature if you > > > are > > > > > > > > > > > > providing > > > > > > > > > > > > > a > > > > > > > > > > > > > > ` > > > > > > > > > > > > > > > retry.backoff.ms` unless you also provide ` > > > > > > > > > retry.backoff.max.ms > > > > > > > > > > `. > > > > > > > > > > > > That > > > > > > > > > > > > > > > makes sense if you assume the user is unaware > of the new > > > > > > > > > > > > configuration, > > > > > > > > > > > > > > but > > > > > > > > > > > > > > > it is surprising otherwise. Since it's not a > semantic > > > change > > > > > > > > > and > > > > > > > > > > > > since > > > > > > > > > > > > > > the > > > > > > > > > > > > > > > default you're proposing of 1s is fairly low > already, I > > > > wonder > > > > > > > > > if > > > > > > > > > > > > it's > > > > > > > > > > > > > > good > > > > > > > > > > > > > > > enough to mention the new configuration in the > release > > > notes > > > > > > > > > and > > > > > > > > > > not > > > > > > > > > > > > > add > > > > > > > > > > > > > > > any special logic. What do you think? > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -Jason > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > On Thu, Mar 19, 2020 at 1:56 PM Sanjana > Kaundinya < > > > > > > > > > > > > > skaundi...@gmail.com> > > > > > > > > > > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > 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 > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > -- Guozhang > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > -- Guozhang > > > >