> in case we later changed the logic to only permit commits on assigned partitions
I experienced this yesterday and was wondering why Kafka allows commits to partitions from other consumers than the assigned one. Does any one know of the reasoning behind this? Martin On 5 Jan 2016 18:29, "Jason Gustafson" <ja...@confluent.io> wrote: > Yes, in this case you should use assign() instead of subscribe(). I'm not > sure it's strictly necessary at the moment to use assign() in this case, > but it would protect your code in case we later changed the logic to only > permit commits on assigned partitions. It also doesn't really cost > anything. > > -Jason > > On Mon, Jan 4, 2016 at 7:49 PM, tao xiao <xiaotao...@gmail.com> wrote: > > > Thanks for the detailed explanation. 'technically commit offsets without > > joining group' I assume it means that I can call assign instead of > > subscribe on consumer which bypasses joining process. > > > > The reason we put the reset offset outside of the consumer process is > that > > we can keep the consumer code as generic as possible since the offset > reset > > process is not needed for all consumer logics. > > > > On Tue, 5 Jan 2016 at 11:18 Jason Gustafson <ja...@confluent.io> wrote: > > > > > Ah, that makes sense if you have to wait to join the group. I think you > > > could technically commit offsets without joining if you were sure that > > the > > > group was dead (i.e. all consumers had either left the group cleanly or > > > their session timeout expired). But if there are still active members, > > then > > > yeah, you have to join the group. Clearly you have to be a little > careful > > > in this case if an active consumer is still trying to read data (it > won't > > > necessarily see the fresh offset commits and could even overwrite > them), > > > but I assume you're handling this. > > > > > > Creating a new instance each time you want to do this seems viable to > me > > > (and likely how we'd end up implementing the command line utility > > anyway). > > > The overhead is just a couple TCP connections. It's probably as good > (or > > as > > > bad) as any other approach. The join latency seems unavoidable if you > > can't > > > be sure that the group is dead since we do not allow non-group members > to > > > commit offsets by design. Any tool we write will be up against the same > > > restriction. We might be able to think of a way to bypass it, but that > > > sounds dangerous. > > > > > > Out of curiosity, what's the advantage in your use case to setting > > offsets > > > out-of-band? I would probably consider options for moving it into the > > > consumer process. > > > > > > -Jason > > > > > > On Mon, Jan 4, 2016 at 6:20 PM, tao xiao <xiaotao...@gmail.com> wrote: > > > > > > > Jason, > > > > > > > > It normally takes a couple of seconds sometimes it takes longer to > > join a > > > > group if the consumer didn't shutdown gracefully previously. > > > > > > > > My use case is to have a command/tool to call to reset offset for a > > list > > > of > > > > partitions and a particular consumer group before the consumer is > > started > > > > or wait until the offset reaches a given number before the consumer > can > > > be > > > > closed. I think https://issues.apache.org/jira/browse/KAFKA-3059 > fits > > my > > > > use case. But for now I need to find out a workaround until this > > feature > > > is > > > > implemented. > > > > > > > > For offset reset one way I can think of is to create a consumer with > > the > > > > same group id that I want to reset the offset for. Then commit the > > offset > > > > for the particular partitions and close the consumer. Is this > solution > > > > viable? > > > > > > > > On Tue, 5 Jan 2016 at 09:56 Jason Gustafson <ja...@confluent.io> > > wrote: > > > > > > > > > Hey Tao, > > > > > > > > > > Interesting that you're seeing a lot of overhead constructing the > new > > > > > consumer instance each time. Granted it does have to fetch topic > > > metadata > > > > > and lookup the coordinator, but I wouldn't have expected that to > be a > > > big > > > > > problem. How long is it typically taking? > > > > > > > > > > -Jason > > > > > > > > > > On Mon, Jan 4, 2016 at 3:26 AM, Marko Bonaći < > > > marko.bon...@sematext.com> > > > > > wrote: > > > > > > > > > > > How are you consuming those topics? > > > > > > > > > > > > IF: I assume you have a consumer, so why not commit from within > > that > > > > > > consumer, after you process the message (whatever "process" means > > to > > > > > you). > > > > > > > > > > > > ELSE: couldn't you have a dedicated consumer for offset commit > > > requests > > > > > > that you don't shut down between requests? > > > > > > > > > > > > FINALLY: tell us more about your use case. > > > > > > > > > > > > Marko Bonaći > > > > > > Monitoring | Alerting | Anomaly Detection | Centralized Log > > > Management > > > > > > Solr & Elasticsearch Support > > > > > > Sematext <http://sematext.com/> | Contact > > > > > > <http://sematext.com/about/contact.html> > > > > > > > > > > > > On Mon, Jan 4, 2016 at 12:18 PM, tao xiao <xiaotao...@gmail.com> > > > > wrote: > > > > > > > > > > > > > Hi team, > > > > > > > > > > > > > > I have a scenario where I want to write new offset for a list > of > > > > topics > > > > > > on > > > > > > > demand. The list of topics is unknown until runtime and the > > > interval > > > > > > > between each commit is undetermined. what would be the best way > > to > > > do > > > > > so? > > > > > > > > > > > > > > One way I can think of is to create a new consumer and call > > > > > > > commitSync(offsets) every time I want to commit. But it seems > > > taking > > > > > too > > > > > > > much time to bootstrap the consumer. is there a lighter way to > > > > achieve > > > > > > > this? > > > > > > > > > > > > > > > > > > > > > > > > > > > >