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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -846,43 +856,24 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
                 time,
                 globalThreadId,
                 delegatingStateRestoreListener,
-                this::defaultStreamsUncaughtExceptionHandler
+                streamsUncaughtExceptionHandler
             );
             globalThreadState = globalStreamThread.state();
         }
 
         // use client id instead of thread client id since this admin client 
may be shared among threads
         adminClient = 
clientSupplier.getAdmin(config.getAdminConfigs(ClientUtils.getSharedAdminClientId(clientId)));
 
-        final Map<Long, StreamThread.State> threadState = new 
HashMap<>(numStreamThreads);
-        final ArrayList<StreamThreadStateStoreProvider> storeProviders = new 
ArrayList<>();
+        threadState = new HashMap<>(numStreamThreads);
+        storeProviders = new ArrayList<>();
         for (int i = 0; i < numStreamThreads; i++) {
-            final StreamThread streamThread = StreamThread.create(
-                internalTopologyBuilder,
-                config,
-                clientSupplier,
-                adminClient,
-                processId,
-                clientId,
-                streamsMetrics,
-                time,
-                streamsMetadataState,
-                cacheSizePerThread,
-                stateDirectory,
-                delegatingStateRestoreListener,
-                i + 1,
-                KafkaStreams.this::closeToError,
-                this::defaultStreamsUncaughtExceptionHandler
-            );
-            threads.add(streamThread);
-            threadState.put(streamThread.getId(), streamThread.state());
-            storeProviders.add(new 
StreamThreadStateStoreProvider(streamThread));
+            createStreamThread(cacheSizePerThread, i + 1);

Review comment:
       I think so

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -398,6 +407,7 @@ public void setUncaughtExceptionHandler(final 
StreamsUncaughtExceptionHandler st
         final Consumer<Throwable> handler = exception -> 
handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler);
         synchronized (stateLock) {
             if (state == State.CREATED) {
+                this.streamsUncaughtExceptionHandler = handler;

Review comment:
       the `this.` is necessary. the parameter is the same name

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
##########
@@ -76,6 +76,9 @@ public void resize(final long newCacheSizeBytes) {
         final boolean shrink = newCacheSizeBytes < maxCacheSizeBytes;
         maxCacheSizeBytes = newCacheSizeBytes;
         if (shrink) {
+            if (caches.values().isEmpty()) {

Review comment:
       Apparently `CircularIterators` throw an error if they are made on empty 
lists. And if there are no caches to resize we don't need to bother with the 
rest of this.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,19 +885,88 @@ 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) {

Review comment:
       If it is not running or rebalancing we after it was already running or 
rebalancing on line 930 we know the client has stopped. The number of threads 
will be reset to the config and everything will be rebuilt anyways, so changing 
the cache size should not matter.
   
   The state lock is so that in between the second check and starting the 
thread the state does not change to pending shutdown or something else. I don't 
think its necessary to guard the whole method as the cache should be thrown out 
if it's not being started.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,19 +885,88 @@ 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}.

Review comment:
       I think that makes more sense.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -846,43 +856,24 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
                 time,
                 globalThreadId,
                 delegatingStateRestoreListener,
-                this::defaultStreamsUncaughtExceptionHandler
+                streamsUncaughtExceptionHandler
             );
             globalThreadState = globalStreamThread.state();
         }
 
         // use client id instead of thread client id since this admin client 
may be shared among threads
         adminClient = 
clientSupplier.getAdmin(config.getAdminConfigs(ClientUtils.getSharedAdminClientId(clientId)));
 
-        final Map<Long, StreamThread.State> threadState = new 
HashMap<>(numStreamThreads);
-        final ArrayList<StreamThreadStateStoreProvider> storeProviders = new 
ArrayList<>();
+        threadState = new HashMap<>(numStreamThreads);
+        storeProviders = new ArrayList<>();
         for (int i = 0; i < numStreamThreads; i++) {
-            final StreamThread streamThread = StreamThread.create(
-                internalTopologyBuilder,
-                config,
-                clientSupplier,
-                adminClient,
-                processId,
-                clientId,
-                streamsMetrics,
-                time,
-                streamsMetadataState,
-                cacheSizePerThread,
-                stateDirectory,
-                delegatingStateRestoreListener,
-                i + 1,
-                KafkaStreams.this::closeToError,
-                this::defaultStreamsUncaughtExceptionHandler
-            );
-            threads.add(streamThread);
-            threadState.put(streamThread.getId(), streamThread.state());
-            storeProviders.add(new 
StreamThreadStateStoreProvider(streamThread));
+            createStreamThread(cacheSizePerThread, i + 1);
         }
 
         ClientMetrics.addNumAliveStreamThreadMetric(streamsMetrics, 
(metricsConfig, now) ->
             Math.toIntExact(threads.stream().filter(thread -> 
thread.state().isAlive()).count()));
 
-        final StreamStateListener streamStateListener = new 
StreamStateListener(threadState, globalThreadState);
+        streamStateListener = new StreamStateListener(threadState, 
globalThreadState);

Review comment:
       good idea. I don't know why the `SteamStateListener` is created after 
the stream threads are made but it seems to work.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,19 +885,88 @@ 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();
+                    return Optional.of(streamThread.getName());
+                } else {
+                    return Optional.empty();
+                }
+            }
+        } else {
+            return Optional.empty();
+        }
+    }
+
+    private int getNextThreadIndex() {
+        final HashSet<String> names = new HashSet<>();

Review comment:
       As threads are removed we want to reuse those names, so incrementing 
would not work for us. Maybe there is away to store a next name, but then the 
logic would have to be spread out in a few places and I prefer to just compute 
a few names.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,19 +885,88 @@ 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();

Review comment:
       I think I explain this above. But we can remove from the thread list.

##########
File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
##########
@@ -588,6 +592,25 @@ public void testCloseIsIdempotent() {
             closeCount, MockMetricsReporter.CLOSE_COUNT.get());
     }
 
+    @Test
+    public void shouldAddThread() throws InterruptedException {
+        props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
+        final KafkaStreams streams = new 
KafkaStreams(getBuilderWithSource().build(), props, supplier, time);
+        streams.start();
+        final int oldSize = streams.threads.size();
+        TestUtils.waitForCondition(() -> streams.state() == 
KafkaStreams.State.RUNNING, 15L, "wait until running");
+        assertThat(streams.addStreamThread(), 
equalTo(Optional.of("newThread")));
+        assertThat(streams.threads.size(), equalTo(oldSize + 1));
+    }
+
+    @Test
+    public void shouldNotAddThread() {

Review comment:
       Good idea I'll add some tests.




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