19
19
import org .apache .kafka .clients .ClientResponse ;
20
20
import org .apache .kafka .clients .CommonClientConfigs ;
21
21
import org .apache .kafka .clients .GroupRebalanceConfig ;
22
+ import org .apache .kafka .clients .consumer .GroupMembershipOperation ;
22
23
import org .apache .kafka .common .KafkaException ;
23
24
import org .apache .kafka .common .Node ;
24
25
import org .apache .kafka .common .errors .AuthenticationException ;
@@ -1114,22 +1115,26 @@ private boolean isProtocolTypeInconsistent(String protocolType) {
1114
1115
*/
1115
1116
@ Override
1116
1117
public final void close () {
1117
- close (time .timer (0 ));
1118
+ close (time .timer (0 ), GroupMembershipOperation . DEFAULT );
1118
1119
}
1119
1120
1120
1121
/**
1121
1122
* @throws KafkaException if the rebalance callback throws exception
1122
1123
*/
1123
- protected void close (Timer timer ) {
1124
+ protected void close (Timer timer , GroupMembershipOperation membershipOperation ) {
1124
1125
try {
1125
1126
closeHeartbeatThread ();
1126
1127
} finally {
1127
1128
// Synchronize after closing the heartbeat thread since heartbeat thread
1128
1129
// needs this lock to complete and terminate after close flag is set.
1129
1130
synchronized (this ) {
1130
- if (rebalanceConfig .leaveGroupOnClose ) {
1131
+ // If membershipOperation is REMAIN_IN_GROUP, never send leave group request.
1132
+ // If membershipOperation is DEFAULT, leave group based on rebalanceConfig.leaveGroupOnClose.
1133
+ // Otherwise, leave group only if membershipOperation is LEAVE_GROUP.
1134
+ if (GroupMembershipOperation .REMAIN_IN_GROUP != membershipOperation &&
1135
+ (GroupMembershipOperation .LEAVE_GROUP == membershipOperation || rebalanceConfig .leaveGroupOnClose )) {
1131
1136
onLeavePrepare ();
1132
- maybeLeaveGroup ("the consumer is being closed" );
1137
+ maybeLeaveGroup (membershipOperation , "the consumer is being closed" );
1133
1138
}
1134
1139
1135
1140
// At this point, there may be pending commits (async commits or sync commits that were
@@ -1151,31 +1156,28 @@ protected void handlePollTimeoutExpiry() {
1151
1156
"either by increasing max.poll.interval.ms or by reducing the maximum size of batches " +
1152
1157
"returned in poll() with max.poll.records." );
1153
1158
1154
- maybeLeaveGroup ("consumer poll timeout has expired." );
1159
+ maybeLeaveGroup (GroupMembershipOperation . DEFAULT , "consumer poll timeout has expired." );
1155
1160
}
1156
1161
1157
1162
/**
1158
- * Sends LeaveGroupRequest and logs the {@code leaveReason}, unless this member is using static membership or is already
1159
- * not part of the group (ie does not have a valid member id, is in the UNJOINED state, or the coordinator is unknown).
1163
+ * Sends LeaveGroupRequest and logs the {@code leaveReason}, unless this member is using static membership
1164
+ * with the default consumer group membership operation, or is already not part of the group (i.e., does not have a
1165
+ * valid member ID, is in the UNJOINED state, or the coordinator is unknown).
1160
1166
*
1167
+ * @param membershipOperation the operation on consumer group membership that the consumer will perform when closing
1161
1168
* @param leaveReason the reason to leave the group for logging
1162
1169
* @throws KafkaException if the rebalance callback throws exception
1163
1170
*/
1164
- public synchronized RequestFuture <Void > maybeLeaveGroup (String leaveReason ) {
1171
+ public synchronized RequestFuture <Void > maybeLeaveGroup (GroupMembershipOperation membershipOperation , String leaveReason ) {
1165
1172
RequestFuture <Void > future = null ;
1166
1173
1167
- // Starting from 2.3, only dynamic members will send LeaveGroupRequest to the broker,
1168
- // consumer with valid group.instance.id is viewed as static member that never sends LeaveGroup,
1169
- // and the membership expiration is only controlled by session timeout.
1170
- if (isDynamicMember () && !coordinatorUnknown () &&
1171
- state != MemberState .UNJOINED && generation .hasMemberId ()) {
1172
- // this is a minimal effort attempt to leave the group. we do not
1173
- // attempt any resending if the request fails or times out.
1174
+ if (shouldSendLeaveGroupRequest (membershipOperation )) {
1174
1175
log .info ("Member {} sending LeaveGroup request to coordinator {} due to {}" ,
1175
1176
generation .memberId , coordinator , leaveReason );
1177
+
1176
1178
LeaveGroupRequest .Builder request = new LeaveGroupRequest .Builder (
1177
1179
rebalanceConfig .groupId ,
1178
- Collections . singletonList (new MemberIdentity ().setMemberId (generation .memberId ).setReason (JoinGroupRequest .maybeTruncateReason (leaveReason )))
1180
+ List . of (new MemberIdentity ().setMemberId (generation .memberId ).setReason (JoinGroupRequest .maybeTruncateReason (leaveReason )))
1179
1181
);
1180
1182
1181
1183
future = client .send (coordinator , request ).compose (new LeaveGroupResponseHandler (generation ));
@@ -1187,6 +1189,17 @@ public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) {
1187
1189
return future ;
1188
1190
}
1189
1191
1192
+ private boolean shouldSendLeaveGroupRequest (GroupMembershipOperation membershipOperation ) {
1193
+ // According to KIP-1092, static members can leave the group if the consumer group membership operation is LEAVE_GROUP.
1194
+ // If the operation is REMAIN_IN_GROUP, this method "maybeLeaveGroup" will not be invoked.
1195
+ return membershipOperation == GroupMembershipOperation .LEAVE_GROUP ||
1196
+ // Below is the default behavior for consumer group membership:
1197
+ // Starting from 2.3, only dynamic members will send LeaveGroupRequest to the broker,
1198
+ // consumer with valid group.instance.id is viewed as static member that never sends LeaveGroup,
1199
+ // and the membership expiration is only controlled by session timeout.
1200
+ (isDynamicMember () && !coordinatorUnknown () && state != MemberState .UNJOINED && generation .hasMemberId ());
1201
+ }
1202
+
1190
1203
protected boolean isDynamicMember () {
1191
1204
return rebalanceConfig .groupInstanceId .isEmpty ();
1192
1205
}
0 commit comments