...
Once the initial ConsumerGroupHeartbeatResponse
is received successfully, the CGMSM will update its local MemberId
and MemberEpoch
based on the returned data. It will then transition to the JOINED
state.
JOINED
When the CGMSM has joined the group, we enter into its namesake state. However, this The JOINED
state simply indicates that the Consumer
instance is known to the coordinator as a member of the group. It does not necessarily imply that it has been assigned any partitions. While in the JOINED
state the CGMSM will periodically send requests to the coordinator at the needed cadence in order to maintain membership.
ASSIGNING
...
The CGMSM should transition back to the JOINING
state if the ConsumerGroupHeartbeatResponse
has an error of UNKNOWN_MEMBER_ID
or FENCED_MEMBER_EPOCH
. If either of those errors occur, the CGMSM will clear its "assigned" partition set (without any revocation), and transition to the JOINING
set so that it rejoins the group with the same MemberId
and the MemberEpoch
of 0.
The CGMSM will transition into the ASSIGNING
state when the ConsumerGroupHeartbeatResponse
contains a non-null
value for Assignment
.
ASSIGNING
The ASSIGNING
state is entered with the intention that the CGMSM will need to perform the assignment reconciliation process. As is done in the JOINED
state, the CGMSM will continue to communicate with the coordinator via the heartbeat mechanism to maintain its membership.
The first action that is performed in this state is to update the CGMSM's value for the member epoch as provided in the ConsumerGroupHeartbeatResponse
.
Next, the CGMSM performs a comparison between its current the assignment and the value of Assignment
contained in the ConsumerGroupHeartbeatResponse
. If the two assignments are equal, the CGMSM has reconciled the assignment successful and will transition back to the JOINED
state. If they are not equal, the reconciliation process begins.
Partition revocation involves:
- Removing the partitions from the CGMSM's "assigned" set
- Commits the offsets for the revoked partitions
- Invokes ConsumerRebalanceListener.onPartitionsRevoked()
QUESTION: do we need to heartbeat between revocation and assignment?
Partition assignment includes:
- Adding the partitions to the CGMSM's "assigned" set
- Invokes ConsumerPartitionAssignor.onAssignment(), if one is set
- Invokes ConsumerRebalanceListener.onPartitionsAssigned()
TERMINATING
TBD
TERMINATED
TBD
...