cadonna commented on a change in pull request #9984:
URL: https://github.com/apache/kafka/pull/9984#discussion_r565991800
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -1074,6 +1089,9 @@ private void completeShutdown(final boolean cleanRun) {
streamsMetrics.removeAllThreadLevelSensors(getName());
setState(State.DEAD);
+ if (leaveGroup.get()) {
+ mainConsumer.unsubscribe();
Review comment:
It is illegal to unsubscribe a closed consumer.
##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +988,68 @@ private StreamThread createAndAddStreamThread(final long
cacheSizePerThread, fin
* no stream threads are alive
*/
public Optional<String> removeStreamThread() {
+ return removeStreamThread(Long.MAX_VALUE);
+ }
+
+ /**
+ * Removes one stream thread out of the running stream threads from this
Kafka Streams client.
+ * <p>
+ * The removed stream thread is gracefully shut down. This method does not
specify which stream
+ * thread is shut down.
+ * <p>
+ * Since the number of stream threads decreases, the sizes of the caches
in the remaining stream
+ * threads are adapted so that the sum of the cache sizes over all stream
threads equals the total
+ * cache size specified in configuration {@link
StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+ *
+ * @param timeout The the length of time to wait for the thread to shutdown
+ * @throws org.apache.kafka.common.errors.TimeoutException if the thread
does not stop in time
+ * @return name of the removed stream thread or empty if a stream thread
could not be removed because
+ * no stream threads are alive
+ */
+ public Optional<String> removeStreamThread(final Duration timeout) {
+ final String msgPrefix = prepareMillisCheckFailMsgPrefix(timeout,
"timeout");
+ final long timeoutMs = validateMillisecondDuration(timeout, msgPrefix);
+ return removeStreamThread(timeoutMs);
+ }
+
+ private Optional<String> removeStreamThread(final long timeoutMs) throws
TimeoutException {
+ final long begin = time.milliseconds();
+ boolean timeout = false;
if (isRunningOrRebalancing()) {
synchronized (changeThreadCount) {
// make a copy of threads to avoid holding lock
for (final StreamThread streamThread : new
ArrayList<>(threads)) {
if (streamThread.isAlive() &&
(!streamThread.getName().equals(Thread.currentThread().getName())
|| threads.size() == 1)) {
+ final Optional<String> groupInstanceID =
streamThread.getGroupInstanceID();
+ streamThread.leaveGroup();
streamThread.shutdown();
if
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+ if
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs)) {
+ log.warn("Thread " + streamThread.getName() +
" did not stop in the allotted time");
+ timeout = true;
+ }
}
threads.remove(streamThread);
final long cacheSizePerThread =
getCacheSizePerThread(threads.size());
resizeThreadCache(cacheSizePerThread);
+ if (groupInstanceID.isPresent() &&
!streamThread.getName().equals(Thread.currentThread().getName())) {
+ final MemberToRemove memberToRemove = new
MemberToRemove(streamThread.getGroupInstanceID().get());
Review comment:
```suggestion
final MemberToRemove memberToRemove = new
MemberToRemove(groupInstanceID.get());
```
##########
File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
##########
@@ -330,12 +331,12 @@ private void prepareStreamThread(final StreamThread
thread, final boolean termin
Collections.emptySet()
)
).anyTimes();
- EasyMock.expect(thread.threadMetadata()).andStubReturn(threadMetadata);
- thread.waitOnThreadState(StreamThread.State.DEAD);
- EasyMock.expectLastCall().anyTimes();
+ EasyMock.expect(thread.waitOnThreadState(StreamThread.State.DEAD,
Long.MAX_VALUE)).andReturn(true).anyTimes();
Review comment:
```suggestion
EasyMock.expect(thread.waitOnThreadState(StreamThread.State.DEAD,
Long.MAX_VALUE)).andStubReturn(true);
```
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -1158,6 +1176,14 @@ public String toString(final String indent) {
return indent + "\tStreamsThread threadId: " + getName() + "\n" +
taskManager.toString(indent);
}
+ public Optional<String> getGroupInstanceID() {
+ return getGroupInstanceID;
+ }
+
+ public void leaveGroup() {
+ this.leaveGroup.set(true);
Review comment:
Could you please rename the method and the field to something like
`requestLeaveGroupDuringShutdown()` and `leaveGroupRequested`?
##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +988,68 @@ private StreamThread createAndAddStreamThread(final long
cacheSizePerThread, fin
* no stream threads are alive
*/
public Optional<String> removeStreamThread() {
+ return removeStreamThread(Long.MAX_VALUE);
+ }
+
+ /**
+ * Removes one stream thread out of the running stream threads from this
Kafka Streams client.
+ * <p>
+ * The removed stream thread is gracefully shut down. This method does not
specify which stream
+ * thread is shut down.
+ * <p>
+ * Since the number of stream threads decreases, the sizes of the caches
in the remaining stream
+ * threads are adapted so that the sum of the cache sizes over all stream
threads equals the total
+ * cache size specified in configuration {@link
StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+ *
+ * @param timeout The the length of time to wait for the thread to shutdown
+ * @throws org.apache.kafka.common.errors.TimeoutException if the thread
does not stop in time
+ * @return name of the removed stream thread or empty if a stream thread
could not be removed because
+ * no stream threads are alive
+ */
+ public Optional<String> removeStreamThread(final Duration timeout) {
+ final String msgPrefix = prepareMillisCheckFailMsgPrefix(timeout,
"timeout");
+ final long timeoutMs = validateMillisecondDuration(timeout, msgPrefix);
+ return removeStreamThread(timeoutMs);
+ }
+
+ private Optional<String> removeStreamThread(final long timeoutMs) throws
TimeoutException {
+ final long begin = time.milliseconds();
+ boolean timeout = false;
if (isRunningOrRebalancing()) {
synchronized (changeThreadCount) {
// make a copy of threads to avoid holding lock
for (final StreamThread streamThread : new
ArrayList<>(threads)) {
if (streamThread.isAlive() &&
(!streamThread.getName().equals(Thread.currentThread().getName())
|| threads.size() == 1)) {
+ final Optional<String> groupInstanceID =
streamThread.getGroupInstanceID();
+ streamThread.leaveGroup();
streamThread.shutdown();
if
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+ if
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs)) {
+ log.warn("Thread " + streamThread.getName() +
" did not stop in the allotted time");
+ timeout = true;
+ }
}
threads.remove(streamThread);
final long cacheSizePerThread =
getCacheSizePerThread(threads.size());
resizeThreadCache(cacheSizePerThread);
+ if (groupInstanceID.isPresent() &&
!streamThread.getName().equals(Thread.currentThread().getName())) {
+ final MemberToRemove memberToRemove = new
MemberToRemove(streamThread.getGroupInstanceID().get());
+ final Collection<MemberToRemove> membersToRemove =
Collections.singletonList(memberToRemove);
+ final RemoveMembersFromConsumerGroupResult
removeMembersFromConsumerGroupResult =
adminClient.removeMembersFromConsumerGroup(config.getString(StreamsConfig.APPLICATION_ID_CONFIG),
new RemoveMembersFromConsumerGroupOptions(membersToRemove));
Review comment:
This line is a bit long.
```suggestion
final RemoveMembersFromConsumerGroupResult
removeMembersFromConsumerGroupResult =
adminClient.removeMembersFromConsumerGroup(
config.getString(StreamsConfig.APPLICATION_ID_CONFIG),
new
RemoveMembersFromConsumerGroupOptions(membersToRemove)
);
```
##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +988,68 @@ private StreamThread createAndAddStreamThread(final long
cacheSizePerThread, fin
* no stream threads are alive
*/
public Optional<String> removeStreamThread() {
+ return removeStreamThread(Long.MAX_VALUE);
+ }
+
+ /**
+ * Removes one stream thread out of the running stream threads from this
Kafka Streams client.
+ * <p>
+ * The removed stream thread is gracefully shut down. This method does not
specify which stream
+ * thread is shut down.
+ * <p>
+ * Since the number of stream threads decreases, the sizes of the caches
in the remaining stream
+ * threads are adapted so that the sum of the cache sizes over all stream
threads equals the total
+ * cache size specified in configuration {@link
StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+ *
+ * @param timeout The the length of time to wait for the thread to shutdown
+ * @throws org.apache.kafka.common.errors.TimeoutException if the thread
does not stop in time
+ * @return name of the removed stream thread or empty if a stream thread
could not be removed because
+ * no stream threads are alive
+ */
+ public Optional<String> removeStreamThread(final Duration timeout) {
+ final String msgPrefix = prepareMillisCheckFailMsgPrefix(timeout,
"timeout");
+ final long timeoutMs = validateMillisecondDuration(timeout, msgPrefix);
+ return removeStreamThread(timeoutMs);
+ }
+
+ private Optional<String> removeStreamThread(final long timeoutMs) throws
TimeoutException {
+ final long begin = time.milliseconds();
+ boolean timeout = false;
if (isRunningOrRebalancing()) {
synchronized (changeThreadCount) {
// make a copy of threads to avoid holding lock
for (final StreamThread streamThread : new
ArrayList<>(threads)) {
if (streamThread.isAlive() &&
(!streamThread.getName().equals(Thread.currentThread().getName())
|| threads.size() == 1)) {
+ final Optional<String> groupInstanceID =
streamThread.getGroupInstanceID();
+ streamThread.leaveGroup();
streamThread.shutdown();
if
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+ if
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs)) {
+ log.warn("Thread " + streamThread.getName() +
" did not stop in the allotted time");
Review comment:
nit:
```suggestion
log.warn("Thread " + streamThread.getName()
+ " did not shutdown in the allotted time");
```
##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +988,68 @@ private StreamThread createAndAddStreamThread(final long
cacheSizePerThread, fin
* no stream threads are alive
*/
public Optional<String> removeStreamThread() {
+ return removeStreamThread(Long.MAX_VALUE);
+ }
+
+ /**
+ * Removes one stream thread out of the running stream threads from this
Kafka Streams client.
+ * <p>
+ * The removed stream thread is gracefully shut down. This method does not
specify which stream
+ * thread is shut down.
+ * <p>
+ * Since the number of stream threads decreases, the sizes of the caches
in the remaining stream
+ * threads are adapted so that the sum of the cache sizes over all stream
threads equals the total
+ * cache size specified in configuration {@link
StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+ *
+ * @param timeout The the length of time to wait for the thread to shutdown
+ * @throws org.apache.kafka.common.errors.TimeoutException if the thread
does not stop in time
+ * @return name of the removed stream thread or empty if a stream thread
could not be removed because
+ * no stream threads are alive
+ */
+ public Optional<String> removeStreamThread(final Duration timeout) {
+ final String msgPrefix = prepareMillisCheckFailMsgPrefix(timeout,
"timeout");
+ final long timeoutMs = validateMillisecondDuration(timeout, msgPrefix);
+ return removeStreamThread(timeoutMs);
+ }
+
+ private Optional<String> removeStreamThread(final long timeoutMs) throws
TimeoutException {
+ final long begin = time.milliseconds();
+ boolean timeout = false;
if (isRunningOrRebalancing()) {
synchronized (changeThreadCount) {
// make a copy of threads to avoid holding lock
for (final StreamThread streamThread : new
ArrayList<>(threads)) {
if (streamThread.isAlive() &&
(!streamThread.getName().equals(Thread.currentThread().getName())
|| threads.size() == 1)) {
+ final Optional<String> groupInstanceID =
streamThread.getGroupInstanceID();
+ streamThread.leaveGroup();
streamThread.shutdown();
if
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+ if
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs)) {
+ log.warn("Thread " + streamThread.getName() +
" did not stop in the allotted time");
+ timeout = true;
+ }
}
threads.remove(streamThread);
final long cacheSizePerThread =
getCacheSizePerThread(threads.size());
resizeThreadCache(cacheSizePerThread);
+ if (groupInstanceID.isPresent() &&
!streamThread.getName().equals(Thread.currentThread().getName())) {
+ final MemberToRemove memberToRemove = new
MemberToRemove(streamThread.getGroupInstanceID().get());
+ final Collection<MemberToRemove> membersToRemove =
Collections.singletonList(memberToRemove);
+ final RemoveMembersFromConsumerGroupResult
removeMembersFromConsumerGroupResult =
adminClient.removeMembersFromConsumerGroup(config.getString(StreamsConfig.APPLICATION_ID_CONFIG),
new RemoveMembersFromConsumerGroupOptions(membersToRemove));
+ try {
+
removeMembersFromConsumerGroupResult.memberResult(memberToRemove).get(timeoutMs
- begin, TimeUnit.MILLISECONDS);
+ } catch (final
java.util.concurrent.TimeoutException e) {
+ throw new TimeoutException(e.getMessage());
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ } catch (final ExecutionException e) {
+ e.printStackTrace();
Review comment:
Shall we throw the cause of the `ExecutionException` and log an error?
##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +988,68 @@ private StreamThread createAndAddStreamThread(final long
cacheSizePerThread, fin
* no stream threads are alive
*/
public Optional<String> removeStreamThread() {
+ return removeStreamThread(Long.MAX_VALUE);
+ }
+
+ /**
+ * Removes one stream thread out of the running stream threads from this
Kafka Streams client.
+ * <p>
+ * The removed stream thread is gracefully shut down. This method does not
specify which stream
+ * thread is shut down.
+ * <p>
+ * Since the number of stream threads decreases, the sizes of the caches
in the remaining stream
+ * threads are adapted so that the sum of the cache sizes over all stream
threads equals the total
+ * cache size specified in configuration {@link
StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+ *
+ * @param timeout The the length of time to wait for the thread to shutdown
+ * @throws org.apache.kafka.common.errors.TimeoutException if the thread
does not stop in time
+ * @return name of the removed stream thread or empty if a stream thread
could not be removed because
+ * no stream threads are alive
+ */
+ public Optional<String> removeStreamThread(final Duration timeout) {
+ final String msgPrefix = prepareMillisCheckFailMsgPrefix(timeout,
"timeout");
+ final long timeoutMs = validateMillisecondDuration(timeout, msgPrefix);
+ return removeStreamThread(timeoutMs);
+ }
+
+ private Optional<String> removeStreamThread(final long timeoutMs) throws
TimeoutException {
+ final long begin = time.milliseconds();
+ boolean timeout = false;
if (isRunningOrRebalancing()) {
synchronized (changeThreadCount) {
// make a copy of threads to avoid holding lock
for (final StreamThread streamThread : new
ArrayList<>(threads)) {
if (streamThread.isAlive() &&
(!streamThread.getName().equals(Thread.currentThread().getName())
Review comment:
Since we use this check mutliple times, could you please extract
`!streamThread.getName().equals(Thread.currentThread().getName()` to a variable
named `callingThreadIsNotCurrentStreamThread` or similar.
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -1158,6 +1176,14 @@ public String toString(final String indent) {
return indent + "\tStreamsThread threadId: " + getName() + "\n" +
taskManager.toString(indent);
}
+ public Optional<String> getGroupInstanceID() {
+ return getGroupInstanceID;
+ }
+
+ public void leaveGroup() {
+ this.leaveGroup.set(true);
Review comment:
Or shall we simply add the following overload `shutdown(final boolean
shouldUnsubscribe)` instead of a new method? If yes, I would still rename the
field as proposed above.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]