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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -846,18 +856,37 @@ 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<>();
-        for (int i = 0; i < numStreamThreads; i++) {
-            final StreamThread streamThread = StreamThread.create(
+        threadState = new HashMap<>(numStreamThreads);
+        storeProviders = new ArrayList<>();
+        streamStateListener = new StreamStateListener(threadState, 
globalThreadState);
+        if (hasGlobalTopology) {
+            globalStreamThread.setStateListener(streamStateListener);
+        }
+        for (int i = 1; i <= numStreamThreads; i++) {
+            createStreamThread(cacheSizePerThread, i + 1);

Review comment:
       `i + 1` -> `i` 
   
   Wondering why this does not result in a test failure? (Or does it; Jenkins 
is still running.) -- Maybe we want to add a small test that verifies that we 
name threads correctly.

##########
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:
       I see -- so it's a bug fix on the side (not directly related to this RP) 
-- the original PR that added this method should have added this check.

##########
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:
       That is not what I meant. But it might not matter much anyway.
   
   While we need to loop over all used names in L951 below to reuse, we don't 
need to compute `names` from scratch but would just modify `names` each time we 
add/remove a thread. But it's not perf-critical so re-doing the computation is 
fine, too.

##########
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:
       Maybe. We should ensure that we do proper cleanup for all cases.
   
   What make we wonder: I don't see any code (except the `remove` you added 
below) that would remove a `StreamThread` from the list? Will this be done in a 
different PR?

##########
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:
       I see. So we exploit that possible state transitions are limited. Thanks 
for explaining. Makes sense.




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