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
> 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 G
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.
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 cod
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
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 wa
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
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 oper
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?
F
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
10 matches
Mail list logo