cadonna commented on a change in pull request #9984:
URL: https://github.com/apache/kafka/pull/9984#discussion_r566722041



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +989,75 @@ 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 boolean callingThreadIsNotCurrentStreamThread = 
!streamThread.getName().equals(Thread.currentThread().getName());
+                    if (streamThread.isAlive() && 
(callingThreadIsNotCurrentStreamThread || threads.size() == 1)) {
+                        log.info("Removing StreamThread " + 
streamThread.getName());
+                        final Optional<String> groupInstanceID = 
streamThread.getGroupInstanceID();
+                        streamThread.requestLeaveGroupDuringShutdown();
                         streamThread.shutdown();
                         if 
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-                            
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+                            if 
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs - begin)) {
+                                log.warn("Thread " + streamThread.getName() + 
" did not shutdown in the allotted time");
+                                timeout = true;
+                            }
                         }
                         threads.remove(streamThread);
                         final long cacheSizePerThread = 
getCacheSizePerThread(threads.size());
                         resizeThreadCache(cacheSizePerThread);
+                        if (groupInstanceID.isPresent() && 
callingThreadIsNotCurrentStreamThread) {
+                            final MemberToRemove memberToRemove = new 
MemberToRemove(groupInstanceID.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) {
+                                log.warn("Rethrowing a java TimeoutException 
as a kafka TimeoutException " + Arrays.toString(e.getStackTrace()));
+                                throw new TimeoutException(e.getMessage());

Review comment:
       nit: I would throw it as an error since I do not see any difference in 
severity compared with the execution exception. I think if you add the java 
timeout exception as the cause of the Kafka timeout exception, it will print 
also the stack trace of the Kafka timeout exception with `Caused by`. However, 
it will print the stack trace later in the logs. But that is OK, IMO, and we do 
that also in other places I guess.  
   
   ```suggestion
                                   log.error("Could not remove static member {} 
from consumer group {} due to a timeout: {}",
                                       groupInstanceID.get(), 
config.getString(StreamsConfig.APPLICATION_ID_CONFIG), e);
                                   throw new TimeoutException(e.getMessage(), 
e);
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +989,75 @@ 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");

Review comment:
       I thought we agreed to not allow non-positive timeouts, i.e., `timeout 
<= Duration.ZERO`, for now. Or did I miss something?

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -983,19 +989,75 @@ 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 boolean callingThreadIsNotCurrentStreamThread = 
!streamThread.getName().equals(Thread.currentThread().getName());
+                    if (streamThread.isAlive() && 
(callingThreadIsNotCurrentStreamThread || threads.size() == 1)) {
+                        log.info("Removing StreamThread " + 
streamThread.getName());
+                        final Optional<String> groupInstanceID = 
streamThread.getGroupInstanceID();
+                        streamThread.requestLeaveGroupDuringShutdown();
                         streamThread.shutdown();
                         if 
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-                            
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+                            if 
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs - begin)) {
+                                log.warn("Thread " + streamThread.getName() + 
" did not shutdown in the allotted time");
+                                timeout = true;
+                            }
                         }
                         threads.remove(streamThread);
                         final long cacheSizePerThread = 
getCacheSizePerThread(threads.size());
                         resizeThreadCache(cacheSizePerThread);
+                        if (groupInstanceID.isPresent() && 
callingThreadIsNotCurrentStreamThread) {
+                            final MemberToRemove memberToRemove = new 
MemberToRemove(groupInstanceID.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) {
+                                log.warn("Rethrowing a java TimeoutException 
as a kafka TimeoutException " + Arrays.toString(e.getStackTrace()));
+                                throw new TimeoutException(e.getMessage());
+                            } catch (final InterruptedException e) {
+                                Thread.currentThread().interrupt();
+                            } catch (final ExecutionException e) {
+                                log.error("Getting the member result threw 
this ExecutionException: " + e.getMessage());
+                                throw new StreamsException("Could not remove 
member from group for the following reason: ", e.getCause());

Review comment:
       nit:
   Just added some more information to the messages.
   
   ```suggestion
                                   log.error("Could not remove static member {} 
from consumer group {} due to: {}",
                                       groupInstanceID.get(), 
config.getString(StreamsConfig.APPLICATION_ID_CONFIG), e);
                                   throw new StreamsException("Could not remove 
static member {} from consumer group {} the following reason: ", e.getCause());
   ```




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to