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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,19 +885,126 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
         queryableStoreProvider = new QueryableStoreProvider(storeProviders, 
globalStateStoreProvider);
 
         stateDirCleaner = setupStateDirCleaner();
-        oldHandler = false;
         maybeWarnAboutCodeInRocksDBConfigSetter(log, config);
         rocksDBMetricsRecordingService = 
maybeCreateRocksDBMetricsRecordingService(clientId, config);
     }
 
+    private StreamThread createStreamThread(final long cacheSizePerThread, 
final int threadIdx) {
+        final StreamThread streamThread = StreamThread.create(
+                internalTopologyBuilder,
+                config,
+                clientSupplier,
+                adminClient,
+                processId,
+                clientId,
+                streamsMetrics,
+                time,
+                streamsMetadataState,
+                cacheSizePerThread,
+                stateDirectory,
+                delegatingStateRestoreListener,
+                threadIdx,
+                KafkaStreams.this::closeToError,
+                streamsUncaughtExceptionHandler
+        );
+        threads.add(streamThread);
+        threadState.put(streamThread.getId(), streamThread.state());
+        storeProviders.add(new StreamThreadStateStoreProvider(streamThread));
+        return streamThread;
+    }
+
+    /**
+     * Adds and starts a stream thread in addition to the stream threads that 
are already running in this
+     * Kafka Streams client.
+     *
+     * Since the number of stream threads increases, the sizes of the caches 
in the new stream thread
+     * and the existing stream threads are adapted so that the sum of the 
cache sizes over all stream
+     * threads does not exceed the total cache size specified in configuration
+     * {@code cache.max.bytes.buffering}.
+     *
+     * Stream threads can only be added if this Kafka Streams client is in 
state RUNNING or REBALANCING.
+     *
+     * @return name of the added stream thread or empty if a new stream thread 
could not be added
+     */
+    public Optional<String> addStreamThread() {
+        if (isRunningOrRebalancing()) {
+            final int threadIdx = getNextThreadIndex();
+            final long cacheSizePerThread = 
getCacheSizePerThread(threads.size() + 1);
+            resizeThreadCache(cacheSizePerThread);
+            final StreamThread streamThread = 
createStreamThread(cacheSizePerThread, threadIdx);
+            streamThread.setStateListener(streamStateListener);
+            synchronized (stateLock) {
+                if (isRunningOrRebalancing()) {
+                    streamThread.start();
+                } else {
+                    return Optional.empty();
+                }
+            }
+            return Optional.of(streamThread.getName());
+        } else {
+            return Optional.empty();
+        }
+    }
+
+    /**
+     * Removes one stream thread out of the running stream threads from this 
Kafka Streams client.
+     *
+     * The removed stream thread is gracefully shut down. This method does not 
specify which stream
+     * thread is shut down.
+     *
+     * 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 {@code cache.max.bytes.buffering}.
+     *
+     * @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.isAlive() && 
streamThread.getName().equals(Thread.currentThread().getName())) {

Review comment:
       In the Kip we promise to block until the thread is finished being 
shutdown, I don't think this is best way to do this. However I am not sure we 
will really want to at all. There is also a problem if the calling thread is 
the thread to be shutdown.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,19 +885,126 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
         queryableStoreProvider = new QueryableStoreProvider(storeProviders, 
globalStateStoreProvider);
 
         stateDirCleaner = setupStateDirCleaner();
-        oldHandler = false;
         maybeWarnAboutCodeInRocksDBConfigSetter(log, config);
         rocksDBMetricsRecordingService = 
maybeCreateRocksDBMetricsRecordingService(clientId, config);
     }
 
+    private StreamThread createStreamThread(final long cacheSizePerThread, 
final int threadIdx) {
+        final StreamThread streamThread = StreamThread.create(
+                internalTopologyBuilder,
+                config,
+                clientSupplier,
+                adminClient,
+                processId,
+                clientId,
+                streamsMetrics,
+                time,
+                streamsMetadataState,
+                cacheSizePerThread,
+                stateDirectory,
+                delegatingStateRestoreListener,
+                threadIdx,
+                KafkaStreams.this::closeToError,
+                streamsUncaughtExceptionHandler
+        );
+        threads.add(streamThread);
+        threadState.put(streamThread.getId(), streamThread.state());
+        storeProviders.add(new StreamThreadStateStoreProvider(streamThread));
+        return streamThread;
+    }
+
+    /**
+     * Adds and starts a stream thread in addition to the stream threads that 
are already running in this
+     * Kafka Streams client.
+     *
+     * Since the number of stream threads increases, the sizes of the caches 
in the new stream thread
+     * and the existing stream threads are adapted so that the sum of the 
cache sizes over all stream
+     * threads does not exceed the total cache size specified in configuration
+     * {@code cache.max.bytes.buffering}.
+     *
+     * Stream threads can only be added if this Kafka Streams client is in 
state RUNNING or REBALANCING.
+     *
+     * @return name of the added stream thread or empty if a new stream thread 
could not be added
+     */
+    public Optional<String> addStreamThread() {
+        if (isRunningOrRebalancing()) {
+            final int threadIdx = getNextThreadIndex();
+            final long cacheSizePerThread = 
getCacheSizePerThread(threads.size() + 1);
+            resizeThreadCache(cacheSizePerThread);
+            final StreamThread streamThread = 
createStreamThread(cacheSizePerThread, threadIdx);
+            streamThread.setStateListener(streamStateListener);
+            synchronized (stateLock) {
+                if (isRunningOrRebalancing()) {
+                    streamThread.start();
+                } else {
+                    return Optional.empty();
+                }
+            }
+            return Optional.of(streamThread.getName());
+        } else {
+            return Optional.empty();
+        }
+    }
+
+    /**
+     * Removes one stream thread out of the running stream threads from this 
Kafka Streams client.
+     *
+     * The removed stream thread is gracefully shut down. This method does not 
specify which stream
+     * thread is shut down.
+     *
+     * 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 {@code cache.max.bytes.buffering}.
+     *
+     * @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()) {

Review comment:
       In the kip we say remove a running thread, but that does not really make 
a lot of sense as we should be able to remove an alive thread. Just because all 
the threads are in partition assigned does not mean we should fail this call.




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