bbejeck commented on code in PR #19955:
URL: https://github.com/apache/kafka/pull/19955#discussion_r2404037884
##########
streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java:
##########
@@ -1588,28 +1598,50 @@ public synchronized boolean close(final Duration
timeout) throws IllegalArgument
throw new IllegalArgumentException("Timeout can't be negative.");
}
- return close(Optional.of(timeoutMs), false);
+ return close(Optional.of(timeoutMs),
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
/**
* Shutdown this {@code KafkaStreams} by signaling all the threads to
stop, and then wait up to the timeout for the
* threads to join.
+ * This method is deprecated and replaced by {@link
#close(org.apache.kafka.streams.CloseOptions)}.
* @param options contains timeout to specify how long to wait for the
threads to shut down, and a flag leaveGroup to
* trigger consumer leave call
* @return {@code true} if all threads were successfully
stopped—{@code false} if the timeout was reached
* before all threads stopped
* Note that this method must not be called in the {@link
StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of
{@link StateListener}.
* @throws IllegalArgumentException if {@code timeout} can't be
represented as {@code long milliseconds}
*/
+ @Deprecated(since = "4.2")
public synchronized boolean close(final CloseOptions options) throws
IllegalArgumentException {
+ final org.apache.kafka.streams.CloseOptions closeOptions =
org.apache.kafka.streams.CloseOptions.timeout(options.timeout)
+ .withGroupMembershipOperation(options.leaveGroup ?
+
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP :
+
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
+ return close(closeOptions);
+ }
+
+ /**
+ * Shutdown this {@code KafkaStreams} by signaling all the threads to
stop, and then wait up to the timeout for the
+ * threads to join.
+ * @param options contains timeout to specify how long to wait for the
threads to shut down, and a flag
+ * {@link
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation#LEAVE_GROUP} to
+ * trigger consumer leave call
Review Comment:
Shouldn't this just specify that users can provide either option for
remaining in the group or leaving vs. explicity showing `LEAVE_GROUP`
##########
streams/src/main/java/org/apache/kafka/streams/CloseOptions.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.streams;
+
+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.
Review Comment:
I think this should say "upon closing closing the Kafka Streams application"
or something similar
##########
streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java:
##########
@@ -1453,7 +1461,9 @@ private Thread shutdownHelper(final boolean error, final
long timeoutMs, final b
return new Thread(() -> {
// notify all the threads to stop; avoid deadlocks by stopping any
// further state reports from the thread since we're shutting down
- int numStreamThreads = processStreamThread(streamThread ->
streamThread.shutdown(leaveGroup));
+ int numStreamThreads = processStreamThread(
+ streamThread -> streamThread.shutdown(operation ==
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP)
Review Comment:
I'm thinking we could update the `StreamThread#shutdown` to take a
`CloseOptions.GroupMembershipOperation` parameter, then change the
`AtomicBoolean` to `AtomicReference<GroupMembershipOperation>` and we can pass
the close option directly to the consumer. WDYT?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]