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 .CloseOptions ;
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 ;
85
86
import java .util .concurrent .TimeUnit ;
86
87
import java .util .function .Supplier ;
87
88
89
+ import static org .apache .kafka .clients .consumer .CloseOptions .GroupMembershipOperation .DEFAULT ;
90
+ import static org .apache .kafka .clients .consumer .CloseOptions .GroupMembershipOperation .LEAVE_GROUP ;
91
+
88
92
/**
89
93
* AbstractCoordinator implements group management for a single group member by interacting with
90
94
* a designated Kafka broker (the coordinator). Group semantics are provided by extending this class.
@@ -1116,23 +1120,21 @@ private boolean isProtocolTypeInconsistent(String protocolType) {
1116
1120
*/
1117
1121
@ Override
1118
1122
public final void close () {
1119
- close (time .timer (0 ));
1123
+ close (time .timer (0 ), DEFAULT );
1120
1124
}
1121
1125
1122
1126
/**
1123
1127
* @throws KafkaException if the rebalance callback throws exception
1124
1128
*/
1125
- protected void close (Timer timer ) {
1129
+ protected void close (Timer timer , CloseOptions . GroupMembershipOperation membershipOperation ) {
1126
1130
try {
1127
1131
closeHeartbeatThread ();
1128
1132
} finally {
1129
1133
// Synchronize after closing the heartbeat thread since heartbeat thread
1130
1134
// needs this lock to complete and terminate after close flag is set.
1131
1135
synchronized (this ) {
1132
- if (rebalanceConfig .leaveGroupOnClose ) {
1133
- onLeavePrepare ();
1134
- maybeLeaveGroup ("the consumer is being closed" );
1135
- }
1136
+ onLeavePrepare ();
1137
+ maybeLeaveGroup (membershipOperation , "the consumer is being closed" );
1136
1138
1137
1139
// At this point, there may be pending commits (async commits or sync commits that were
1138
1140
// interrupted using wakeup) and the leave group request which have been queued, but not
@@ -1153,26 +1155,22 @@ protected void handlePollTimeoutExpiry() {
1153
1155
"either by increasing max.poll.interval.ms or by reducing the maximum size of batches " +
1154
1156
"returned in poll() with max.poll.records." );
1155
1157
1156
- maybeLeaveGroup ("consumer poll timeout has expired." );
1158
+ maybeLeaveGroup (DEFAULT , "consumer poll timeout has expired." );
1157
1159
}
1158
1160
1159
1161
/**
1160
- * Sends LeaveGroupRequest and logs the {@code leaveReason}, unless this member is using static membership or is already
1161
- * not part of the group (ie does not have a valid member id, is in the UNJOINED state, or the coordinator is unknown).
1162
+ * Sends LeaveGroupRequest and logs the {@code leaveReason}, unless this member is using static membership
1163
+ * with the default consumer group membership operation, or is already not part of the group (i.e., does not have a
1164
+ * valid member ID, is in the UNJOINED state, or the coordinator is unknown).
1162
1165
*
1166
+ * @param membershipOperation the operation on consumer group membership that the consumer will perform when closing
1163
1167
* @param leaveReason the reason to leave the group for logging
1164
1168
* @throws KafkaException if the rebalance callback throws exception
1165
1169
*/
1166
- public synchronized RequestFuture <Void > maybeLeaveGroup (String leaveReason ) {
1170
+ public synchronized RequestFuture <Void > maybeLeaveGroup (CloseOptions . GroupMembershipOperation membershipOperation , String leaveReason ) {
1167
1171
RequestFuture <Void > future = null ;
1168
1172
1169
- // Starting from 2.3, only dynamic members will send LeaveGroupRequest to the broker,
1170
- // consumer with valid group.instance.id is viewed as static member that never sends LeaveGroup,
1171
- // and the membership expiration is only controlled by session timeout.
1172
- if (isDynamicMember () && !coordinatorUnknown () &&
1173
- state != MemberState .UNJOINED && generation .hasMemberId ()) {
1174
- // this is a minimal effort attempt to leave the group. we do not
1175
- // attempt any resending if the request fails or times out.
1173
+ if (rebalanceConfig .leaveGroupOnClose && shouldSendLeaveGroupRequest (membershipOperation )) {
1176
1174
log .info ("Member {} sending LeaveGroup request to coordinator {} due to {}" ,
1177
1175
generation .memberId , coordinator , leaveReason );
1178
1176
LeaveGroupRequest .Builder request = new LeaveGroupRequest .Builder (
@@ -1189,6 +1187,14 @@ public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) {
1189
1187
return future ;
1190
1188
}
1191
1189
1190
+ private boolean shouldSendLeaveGroupRequest (CloseOptions .GroupMembershipOperation membershipOperation ) {
1191
+ if (!coordinatorUnknown () && state != MemberState .UNJOINED && generation .hasMemberId ()) {
1192
+ return membershipOperation == LEAVE_GROUP || (isDynamicMember () && membershipOperation == DEFAULT );
1193
+ } else {
1194
+ return false ;
1195
+ }
1196
+ }
1197
+
1192
1198
protected boolean isDynamicMember () {
1193
1199
return rebalanceConfig .groupInstanceId .isEmpty ();
1194
1200
}
0 commit comments