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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -870,43 +900,75 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
                 cacheSizePerThread,
                 stateDirectory,
                 delegatingStateRestoreListener,
-                i + 1,
+                threadIdx,
                 KafkaStreams.this::closeToError,
-                this::defaultStreamsUncaughtExceptionHandler
-            );
-            threads.add(streamThread);
-            threadState.put(streamThread.getId(), streamThread.state());
-            storeProviders.add(new 
StreamThreadStateStoreProvider(streamThread));
-        }
+                streamsUncaughtExceptionHandler

Review comment:
       nit: If it happens that you need to push another commit, could you fix 
the indentation here? Sorry that I haven't noticed this before.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -870,43 +900,75 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
                 cacheSizePerThread,
                 stateDirectory,
                 delegatingStateRestoreListener,
-                i + 1,
+                threadIdx,
                 KafkaStreams.this::closeToError,
-                this::defaultStreamsUncaughtExceptionHandler
-            );
-            threads.add(streamThread);
-            threadState.put(streamThread.getId(), streamThread.state());
-            storeProviders.add(new 
StreamThreadStateStoreProvider(streamThread));
-        }
+                streamsUncaughtExceptionHandler
+        );
+        streamThread.setStateListener(streamStateListener);
+        threads.add(streamThread);
+        threadState.put(streamThread.getId(), streamThread.state());
+        storeProviders.add(new StreamThreadStateStoreProvider(streamThread));
+        return streamThread;
+    }
 
-        ClientMetrics.addNumAliveStreamThreadMetric(streamsMetrics, 
(metricsConfig, now) ->
-            Math.toIntExact(threads.stream().filter(thread -> 
thread.state().isAlive()).count()));
+    /**
+     * Adds and starts a stream thread in addition to the stream threads that 
are already running in this
+     * Kafka Streams client.
+     * <p>
+     * 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
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+     * <p>
+     * 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() {
+        synchronized (newThread) {
+            if (isRunningOrRebalancing()) {
+                final int threadIdx = getNextThreadIndex();
+                final long 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();
+                        threads.remove(streamThread);
+                        
resizeThreadCache(getCacheSizePerThread(threads.size()));
+                        return Optional.empty();
+                    }
+                }
+            }
+        }
+        return Optional.empty();
+    }
 
-        final StreamStateListener streamStateListener = new 
StreamStateListener(threadState, globalThreadState);
-        if (hasGlobalTopology) {
-            globalStreamThread.setStateListener(streamStateListener);
+    private int getNextThreadIndex() {
+        final HashSet<String> names = new HashSet<>();
+        for (final StreamThread streamThread: threads) {
+            names.add(streamThread.getName());
         }
-        for (final StreamThread thread : threads) {
-            thread.setStateListener(streamStateListener);
+        final String baseName = clientId + "-StreamThread-";
+        for (int i = 0; i < threads.size(); i++) {

Review comment:
       Shouldn't that be `int i = 1; i <= threads.size(); i++`? Otherwise, we 
would look up `*-StreamThread-0"` and we would not look up `"*-StreamThread-" + 
threads.size()`.
   
   Could you add some tests that check the correct naming as @mjsax suggested? 

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -870,43 +900,75 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
                 cacheSizePerThread,
                 stateDirectory,
                 delegatingStateRestoreListener,
-                i + 1,
+                threadIdx,
                 KafkaStreams.this::closeToError,
-                this::defaultStreamsUncaughtExceptionHandler
-            );
-            threads.add(streamThread);
-            threadState.put(streamThread.getId(), streamThread.state());
-            storeProviders.add(new 
StreamThreadStateStoreProvider(streamThread));
-        }
+                streamsUncaughtExceptionHandler
+        );
+        streamThread.setStateListener(streamStateListener);
+        threads.add(streamThread);
+        threadState.put(streamThread.getId(), streamThread.state());
+        storeProviders.add(new StreamThreadStateStoreProvider(streamThread));
+        return streamThread;
+    }
 
-        ClientMetrics.addNumAliveStreamThreadMetric(streamsMetrics, 
(metricsConfig, now) ->
-            Math.toIntExact(threads.stream().filter(thread -> 
thread.state().isAlive()).count()));
+    /**
+     * Adds and starts a stream thread in addition to the stream threads that 
are already running in this
+     * Kafka Streams client.
+     * <p>
+     * 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
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG}.
+     * <p>
+     * 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() {
+        synchronized (newThread) {
+            if (isRunningOrRebalancing()) {
+                final int threadIdx = getNextThreadIndex();
+                final long cacheSizePerThread = 
getCacheSizePerThread(threads.size() + 1);
+                resizeThreadCache(cacheSizePerThread);
+                final StreamThread streamThread = 
createStreamThread(cacheSizePerThread, threadIdx);
+                synchronized (stateLock) {

Review comment:
       Sorry to bother you again with the synchronization on the `stateLock`, 
but could you explain why we still need it after we synchronize on `newThread`?




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