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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -845,67 +856,118 @@ 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(
-                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));
-        }
-
-        ClientMetrics.addNumAliveStreamThreadMetric(streamsMetrics, 
(metricsConfig, now) ->
-            Math.toIntExact(threads.stream().filter(thread -> 
thread.state().isAlive()).count()));
-
-        final StreamStateListener streamStateListener = new 
StreamStateListener(threadState, globalThreadState);
+        threadState = new HashMap<>(numStreamThreads);
+        storeProviders = new ArrayList<>();
+        streamStateListener = new StreamStateListener(threadState, 
globalThreadState);
         if (hasGlobalTopology) {
             globalStreamThread.setStateListener(streamStateListener);
         }
-        for (final StreamThread thread : threads) {
-            thread.setStateListener(streamStateListener);
+        for (int i = 1; i <= numStreamThreads; i++) {
+            createStreamThread(cacheSizePerThread, i);
         }
 
+        ClientMetrics.addNumAliveStreamThreadMetric(streamsMetrics, 
(metricsConfig, now) ->
+            Math.toIntExact(threads.stream().filter(thread -> 
thread.state().isAlive()).count()));
+
         final GlobalStateStoreProvider globalStateStoreProvider = new 
GlobalStateStoreProvider(internalTopologyBuilder.globalStateStores());
         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
+        );
+        streamThread.setStateListener(streamStateListener);
+        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.
+     * <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 (changeThreadCount) {

Review comment:
       We thought it would be better use a separate lock because it is serving 
a different purpose. It will also use used in remove thread. It might be 
simpler to reuse the statelock but I don’t think that would be cleaner. We are 
really locking on the thread cache access and the thread indexes




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