ableegoldman commented on a change in pull request #9695: URL: https://github.com/apache/kafka/pull/9695#discussion_r538923610
########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -924,22 +924,64 @@ private StreamThread createStreamThread(final long cacheSizePerThread, final int * @return name of the added stream thread or empty if a new stream thread could not be added */ public Optional<String> addStreamThread() { - synchronized (changeThreadCount) { - if (isRunningOrRebalancing()) { - final int threadIdx = getNextThreadIndex(); - final long cacheSizePerThread = getCacheSizePerThread(threads.size() + 1); + if (isRunningOrRebalancing()) { + final int threadIdx; + final long cacheSizePerThread; + synchronized (changeThreadCount) { + threadIdx = getNextThreadIndex(); + cacheSizePerThread = getCacheSizePerThread(threads.size() + 1); resizeThreadCache(cacheSizePerThread); - final StreamThread streamThread = createStreamThread(cacheSizePerThread, threadIdx); - synchronized (stateLock) { - if (isRunningOrRebalancing()) { - streamThread.start(); - return Optional.of(streamThread.getName()); - } else { - streamThread.shutdown(); + } + final StreamThread streamThread = createStreamThread(cacheSizePerThread, threadIdx); + + synchronized (stateLock) { + if (isRunningOrRebalancing()) { + streamThread.start(); + return Optional.of(streamThread.getName()); + } else { + streamThread.shutdown(); + threads.remove(streamThread); + resizeThreadCache(getCacheSizePerThread(threads.size())); + return Optional.empty(); + } + } + } + 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}. + * + * @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() { + if (isRunningOrRebalancing()) { + for (final StreamThread streamThread : threads) { + if (streamThread.isAlive()) { + streamThread.shutdown(); + while (streamThread.state() != StreamThread.State.DEAD && !streamThread.getName().equals(Thread.currentThread().getName())) { + try { + synchronized (streamThread.state()) { + streamThread.state().wait(100); + } + } catch (final InterruptedException e) { + e.printStackTrace(); Review comment: Let's continue the discussion about what to do if the thread hasn't finished shutting down [here](https://github.com/apache/kafka/pull/9695#discussion_r538921676), since the same question applies to this case. As for swallowing the InterruptedException, it still makes me uncomfortable but maybe there was/is a good reason that we do it already. Or, maybe you're just the first person to notice that we're doing that. ---------------------------------------------------------------- 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