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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -1005,11 +1008,60 @@ private StreamThread createAndAddStreamThread(final 
long cacheSizePerThread, fin
                             || threads.size() == 1)) {
                         streamThread.shutdown();
                         if 
(!streamThread.getName().equals(Thread.currentThread().getName())) {
-                            
streamThread.waitOnThreadState(StreamThread.State.DEAD);
+                            
streamThread.waitOnThreadState(StreamThread.State.DEAD, -1);
                         }
                         threads.remove(streamThread);
                         final long cacheSizePerThread = 
getCacheSizePerThread(threads.size());
                         resizeThreadCache(cacheSizePerThread);
+                        final Collection<MemberToRemove> membersToRemove = 
Collections.singletonList(new 
MemberToRemove(streamThread.getGroupInstanceID()));
+                        
adminClient.removeMembersFromConsumerGroup(config.getString(StreamsConfig.APPLICATION_ID_CONFIG),
 new RemoveMembersFromConsumerGroupOptions(membersToRemove));
+                        return Optional.of(streamThread.getName());
+                    }
+                }
+            }
+            log.warn("There are no threads eligible for removal");
+        } else {
+            log.warn("Cannot remove a stream thread when Kafka Streams client 
is in state  " + state());
+        }
+        return Optional.empty();
+    }
+
+    /**
+     * 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 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) throws 
TimeoutException {
+        final String msgPrefix = prepareMillisCheckFailMsgPrefix(timeout, 
"timeout");
+        final long timeoutMs = validateMillisecondDuration(timeout, msgPrefix);
+        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)) {
+                        streamThread.shutdown();
+                        if 
(!streamThread.getName().equals(Thread.currentThread().getName())) {
+                            if 
(!streamThread.waitOnThreadState(StreamThread.State.DEAD, timeoutMs)) {
+                                log.warn("Thread " + streamThread.getName() + 
" did not stop in the allotted time");
+                                throw new TimeoutException("Thread " + 
streamThread.getName() + " did not stop in the allotted time");
+                            }
+                        }
+                        threads.remove(streamThread);
+                        final long cacheSizePerThread = 
getCacheSizePerThread(threads.size());
+                        resizeThreadCache(cacheSizePerThread);
+                        Collection<MemberToRemove> membersToRemove = 
Collections.singletonList(new 
MemberToRemove(streamThread.getGroupInstanceID()));
+                        
adminClient.removeMembersFromConsumerGroup(config.getString(StreamsConfig.APPLICATION_ID_CONFIG),
 new RemoveMembersFromConsumerGroupOptions(membersToRemove));

Review comment:
       Sounds good. How should you handle the `ExecutionException`?




----------------------------------------------------------------
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