[ 
https://issues.apache.org/jira/browse/KAFKA-1740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14620038#comment-14620038
 ] 

Onur Karaman commented on KAFKA-1740:
-------------------------------------

There's a discrepancy between https://reviews.apache.org/r/35231/ and what's 
checked in.

We used to bump the generationId immediately after transitioning to 
PreparingRebalance. Commit 3f8480ccfb011eb43da774737597c597f703e11b moved the 
generationId bump to be immediately after the transition to Rebalancing.

I think the intent was to support the following rebalance pattern: consumers 
stop consuming, commit offsets, and then do the join group to get the new 
partitions they own (which KafkaConsumer does not seem to currently follow). 
Since the group is already PreparingRebalance by the time the consumer finds 
out about the rebalance, we need to allow offset commits during the Preparing 
Rebalance state. I think you can do this in one of two ways:
1. postpone the generationId bump to the Rebalancing state and have 
handleHeartbeat additionally check for the group being in the Stable state.
2. revert back to having the generationId bump happen in the PreparingRebalance 
state and have handleCommitOffsets allow offset commits either during Stable 
state when the request's generationId == group.generationId or when 
PreparingRebalance and request's generationId == group.generationId - 1.

I prefer option 2 because it makes more sense (to me at least) to log a 
generationId increase as soon as the rebalancing process begins.

This commit chose option 1 but didn't fix handleHeartbeat. It currently allows 
consumers to heartbeat even during the PreparingRebalance group state, which is 
incorrect. Heartbeats should only be accepted during the Stable state. The 
problem here is that a consumer gets notified of the rebalance from a 
HeartbeatResponse when generation ids don't match, yet the generationId bump 
only happens after the Rebalancing state. By the time we reach the Rebalancing 
state, PreparingRebalance state has finished and the consumer is already 
considered failed.

> Merge Offset manager into Coordinator
> -------------------------------------
>
>                 Key: KAFKA-1740
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1740
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: consumer
>            Reporter: Guozhang Wang
>            Assignee: Guozhang Wang
>            Priority: Critical
>             Fix For: 0.8.3
>
>         Attachments: KAFKA-1740_2015-06-29_18:21:42.patch, 
> KAFKA-1740_2015-06-29_18:44:54.patch
>
>
> This JIRA involves refactoring offset manager and merge it into coordinator, 
> including adding the logic for consumer-id / generation-id checking.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to