You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by tao xiao <xi...@gmail.com> on 2016/01/04 12:18:51 UTC

Best way to commit offset on demand

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?

Re: Best way to commit offset on demand

Posted by Jason Gustafson <ja...@confluent.io>.
Not sure there's a great reason. In the initial design, the server itself
only permitted commits from consumers that were assigned the respective
partitions, but we lost this when we generalized the group coordination
protocol. It seems like it still makes sense to do it on the client though,
so it's possible that this was just forgotten in all the noise. I'll open a
JIRA and see what others think.

-Jason

On Wed, Jan 6, 2016 at 6:49 AM, Martin Skøtt <martin.skoett@falconsocial.com
> wrote:

> > 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 <xi...@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 <xi...@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.bonaci@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 <
> xiaotao183@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?
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Best way to commit offset on demand

Posted by Martin Skøtt <ma...@falconsocial.com>.
> 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 <xi...@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 <xi...@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.bonaci@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 <xi...@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?
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Best way to commit offset on demand

Posted by Jason Gustafson <ja...@confluent.io>.
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 <xi...@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 <xi...@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.bonaci@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 <xi...@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?
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Best way to commit offset on demand

Posted by tao xiao <xi...@gmail.com>.
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 <xi...@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.bonaci@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 <xi...@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?
> > > > >
> > > >
> > >
> >
>

Re: Best way to commit offset on demand

Posted by Jason Gustafson <ja...@confluent.io>.
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 <xi...@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 <ma...@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 <xi...@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?
> > > >
> > >
> >
>

Re: Best way to commit offset on demand

Posted by tao xiao <xi...@gmail.com>.
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 <ma...@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 <xi...@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?
> > >
> >
>

Re: Best way to commit offset on demand

Posted by Jason Gustafson <ja...@confluent.io>.
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 <ma...@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 <xi...@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?
> >
>

Re: Best way to commit offset on demand

Posted by tao xiao <xi...@gmail.com>.
My use case is to reset offset to a certain number for a particular
consumer group before I start the consumer so that I can control where to
start consuming. It is not ideal put the reset offset logic inside the
consumer application code as this is an out-of-band process and it is part
of our operational procedure. I'd like the keep the consumer as generic as
possible. I am looking for a solution where I can reset/fetch offset for a
list of topic partitions and a particular consumer group which I can invoke
at any time as a standalone procedure.

On Mon, 4 Jan 2016 at 19:27 Marko Bonaći <ma...@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 <xi...@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?
> >
>

Re: Best way to commit offset on demand

Posted by Marko Bonaći <ma...@sematext.com>.
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 <xi...@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?
>