Hi there, I think what you're asking is how the group protocol can guarantee that each partition is assigned to one and only consumer in the group at any point in time. Is that right? The short answer is that it can't. Because of unexpected pauses on the client (e.g. for garbage collection), there is no way that the coordinator can guarantee that a non-responsive consumer has actually stopped processing messages from partitions it was previously assigned. However, the coordinator does guard offset commits from consumers. If it receives a commit from a consumer that had already been kicked out of the group, it will reject it. So even if a consumer in the group /thinks/ it is assigned a partition, it won't be able to commit offsets.
This might not seem like a very strong guarantee, but it's enough to give you at-least-once or at-most-once delivery semantics. If you process messages first before committing, then you have at-least once. If you commit before processing, you have at-most-once. We are also thinking hard about providing exactly-once guarantees, but Kafka doesn't yet support this. -Jason On Wed, Feb 24, 2016 at 7:49 PM, tuyang <tuyang19890...@126.com> wrote: > Hi, > after reviewed Kafka 0.9.0 code, I have a rebalance question: > coordinator will choose a client as leader to execute assign, > after transfer it to coordinator, when assignments are Propagate to each > member in consumer group,how can it sure one partition is only owned by a > consumer because of consumer’s execute orders.I can not find a owner node > in kafka 0.9 zk. > because the execute order of the two consumers, may be at a fixed time, A > consumer owned a partition at last generation, and B consumer owned this > partition at this partition, when this occurs, how it resolve? >