-
Notifications
You must be signed in to change notification settings - Fork 14.3k
KAFKA-16758: Extend Consumer#close with an option to leave the group or not #17614
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
39a7004
837d846
9906e28
b0bce75
9c7d269
1941081
c03c5f3
840395f
c18babb
a26310d
696d207
87be03e
72f5473
df7afca
f20f6d7
7da92e4
f498a8e
e96c5f5
22138d3
9040cd2
06b51a8
26a62bd
68db56c
3e5e288
7513446
3d4ed4a
9d45444
1e18a34
204cb83
fb4055b
e5c9fa5
ab89078
61a6346
fbc878a
6143d52
17d3ee2
3bf1e26
11fb74f
076ea2a
e946499
8ee016d
7fc4237
f37d223
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,124 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.kafka.clients.consumer; | ||
|
||
import org.apache.kafka.clients.consumer.internals.ConsumerUtils; | ||
|
||
import java.time.Duration; | ||
import java.util.Objects; | ||
import java.util.Optional; | ||
|
||
public class CloseOptions { | ||
/** | ||
* Enum to specify the group membership operation upon leaving group. | ||
* | ||
* <ul> | ||
* <li><b>{@code LEAVE_GROUP}</b>: means the consumer will leave the group.</li> | ||
* <li><b>{@code REMAIN_IN_GROUP}</b>: means the consumer will remain in the group.</li> | ||
* <li><b>{@code DEFAULT}</b>: Applies the default behavior: | ||
* <ul> | ||
* <li>For <b>static members</b>: The consumer will remain in the group.</li> | ||
* <li>For <b>dynamic members</b>: The consumer will leave the group.</li> | ||
* </ul> | ||
* </li> | ||
* </ul> | ||
*/ | ||
public enum GroupMembershipOperation { | ||
LEAVE_GROUP, | ||
REMAIN_IN_GROUP, | ||
DEFAULT | ||
} | ||
|
||
/** | ||
* Specifies the group membership operation upon shutdown. | ||
* By default, {@code GroupMembershipOperation.DEFAULT} will be applied, which follows the consumer's default behavior. | ||
*/ | ||
protected GroupMembershipOperation operation = GroupMembershipOperation.DEFAULT; | ||
|
||
/** | ||
* Specifies the maximum amount of time to wait for the close process to complete. | ||
* This allows users to define a custom timeout for gracefully stopping the consumer. | ||
* If no value is set, the default timeout {@link ConsumerUtils#DEFAULT_CLOSE_TIMEOUT_MS} will be applied. | ||
*/ | ||
protected Optional<Duration> timeout = Optional.empty(); | ||
|
||
private CloseOptions() { | ||
} | ||
|
||
protected CloseOptions(final CloseOptions option) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Excuse me, why to have protected constructor? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Previously, we had an internal class, but we deleted it in development. |
||
this.operation = option.operation; | ||
this.timeout = option.timeout; | ||
} | ||
|
||
/** | ||
* Static method to create a {@code CloseOptions} with a custom timeout. | ||
* | ||
* @param timeout the maximum time to wait for the consumer to close. | ||
* @return a new {@code CloseOptions} instance with the specified timeout. | ||
*/ | ||
public static CloseOptions timeout(final Duration timeout) { | ||
CloseOptions option = new CloseOptions(); | ||
option.timeout = Optional.ofNullable(timeout); | ||
return option; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. return new CloseOptions().withTimeout(timeout); |
||
} | ||
|
||
/** | ||
* Static method to create a {@code CloseOptions} with a specified group membership operation. | ||
* | ||
* @param operation the group membership operation to apply. Must be one of {@code LEAVE_GROUP}, {@code REMAIN_IN_GROUP}, | ||
* or {@code DEFAULT}. | ||
* @return a new {@code CloseOptions} instance with the specified group membership operation. | ||
*/ | ||
public static CloseOptions groupMembershipOperation(final GroupMembershipOperation operation) { | ||
Objects.requireNonNull(operation, "operation should not be null"); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you please consider using |
||
CloseOptions option = new CloseOptions(); | ||
option.operation = operation; | ||
return option; | ||
} | ||
|
||
/** | ||
* Fluent method to set the timeout for the close process. | ||
* | ||
* @param timeout the maximum time to wait for the consumer to close. If {@code null}, the default timeout will be used. | ||
* @return this {@code CloseOptions} instance. | ||
*/ | ||
public CloseOptions withTimeout(final Duration timeout) { | ||
this.timeout = Optional.ofNullable(timeout); | ||
return this; | ||
} | ||
|
||
/** | ||
* Fluent method to set the group membership operation upon shutdown. | ||
* | ||
* @param operation the group membership operation to apply. Must be one of {@code LEAVE_GROUP}, {@code REMAIN_IN_GROUP}, or {@code DEFAULT}. | ||
* @return this {@code CloseOptions} instance. | ||
*/ | ||
public CloseOptions withGroupMembershipOperation(final GroupMembershipOperation operation) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you please add unit test for those helpers? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is tracked by https://issues.apache.org/jira/browse/KAFKA-18267 |
||
Objects.requireNonNull(operation, "operation should not be null"); | ||
this.operation = operation; | ||
return this; | ||
} | ||
|
||
public GroupMembershipOperation groupMembershipOperation() { | ||
return operation; | ||
} | ||
|
||
public Optional<Duration> timeout() { | ||
return timeout; | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please update KIP for this naming change.