I can't speak to all use-cases, but for the database one, I think pause-resume will be necessary in any case, and therefore dynamic batch sizes are not needed.
Databases are really unexpected regarding response times - load and locking can affect this. I'm not sure there's a good way to know you are going into rebalance hell before it is too late. So if I were writing code that updates an RDBMS based on Kafka, I'd pick a reasonable batch size (say 5000 records), and basically pause, batch-insert all records, commit and resume. Does that make sense? On Mon, Jan 4, 2016 at 10:37 AM, Jason Gustafson <ja...@confluent.io> wrote: > Gwen and Ismael, > > I agree the configuration option is probably the way to go, but I was > wondering whether there would be cases where it made sense to let the > consumer dynamically set max messages to adjust for downstream slowness. > For example, if the consumer is writing consumed records to another > database, and that database is experiencing heavier than expected load, > then the consumer could halve its current max messages in order to adapt > without risking rebalance hell. It could then increase max messages as the > load on the database decreases. It's basically an easier way to handle flow > control than we provide with pause/resume. > > -Jason > > On Mon, Jan 4, 2016 at 9:46 AM, Gwen Shapira <g...@confluent.io> wrote: > > > The wiki you pointed to is no longer maintained and fell out of sync with > > the code and protocol. > > > > You may want to refer to: > > > > > https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol > > > > On Mon, Jan 4, 2016 at 4:38 AM, Jens Rantil <jens.ran...@tink.se> wrote: > > > > > Hi guys, > > > > > > I realized I never thanked yall for your input - thanks! > > > Jason: I apologize for assuming your stance on the issue! Feels like we > > all > > > agreed on the solution. +1 > > > > > > Follow-up: Jason made a point about defining prefetch and fairness > > > behaviour in the KIP. I am now working on putting that down in writing. > > To > > > do be able to do this I think I need to understand the current prefetch > > > behaviour in the new consumer API (0.9) a bit better. Some specific > > > questions: > > > > > > - How does a specific consumer balance incoming messages from > multiple > > > partitions? Is the consumer simply issuing Multi-Fetch requests[1] > for > > > the > > > consumed assigned partitions of the relevant topics? Or is the > > consumer > > > fetching from one partition at a time and balancing between them > > > internally? That is, is the responsibility of partition balancing > (and > > > fairness) on the broker side or consumer side? > > > - Is the above documented somewhere? > > > > > > [1] > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Writing+a+Driver+for+Kafka > > > , > > > see "Multi-Fetch". > > > > > > Thanks, > > > Jens > > > > > > On Wed, Dec 23, 2015 at 2:44 AM, Ismael Juma <ism...@juma.me.uk> > wrote: > > > > > > > On Wed, Dec 23, 2015 at 1:24 AM, Gwen Shapira <g...@confluent.io> > > wrote: > > > > > > > > > Given the background, it sounds like you'll generally want each > call > > to > > > > > poll() to return the same number of events (which is the number you > > > > planned > > > > > on having enough memory / time for). It also sounds like tuning the > > > > number > > > > > of events will be closely tied to tuning the session timeout. That > > is - > > > > if > > > > > I choose to lower the session timeout for some reason, I will have > to > > > > > modify the number of records returning too. > > > > > > > > > > If those assumptions are correct, I think a configuration makes > more > > > > sense. > > > > > 1. We are unlikely to want this parameter to be change at the > > lifetime > > > of > > > > > the consumer > > > > > 2. The correct value is tied to another configuration parameter, so > > > they > > > > > will be controlled together. > > > > > > > > > > > > > I was thinking the same thing. > > > > > > > > Ismael > > > > > > > > > > > > > > > > -- > > > Jens Rantil > > > Backend engineer > > > Tink AB > > > > > > Email: jens.ran...@tink.se > > > Phone: +46 708 84 18 32 > > > Web: www.tink.se > > > > > > Facebook <https://www.facebook.com/#!/tink.se> Linkedin > > > < > > > > > > http://www.linkedin.com/company/2735919?trk=vsrp_companies_res_photo&trkInfo=VSRPsearchId%3A1057023381369207406670%2CVSRPtargetId%3A2735919%2CVSRPcmpt%3Aprimary > > > > > > > Twitter <https://twitter.com/tink> > > > > > >