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



##########
File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
##########
@@ -588,6 +592,29 @@ public void testCloseIsIdempotent() {
             closeCount, MockMetricsReporter.CLOSE_COUNT.get());
     }
 
+    @Test
+    public void testAddThread() {
+        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();
+        try {
+            TestUtils.waitForCondition(() -> streams.state() == 
KafkaStreams.State.RUNNING, 15L, "wait until running");
+        } catch (final InterruptedException e) {
+            e.printStackTrace();
+        }

Review comment:
       You should not use `try-catch` here but just add `throws 
InterruptedException` to the method signature.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,11 +904,72 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
         queryableStoreProvider = new QueryableStoreProvider(storeProviders, 
globalStateStoreProvider);
 
         stateDirCleaner = setupStateDirCleaner();
-        oldHandler = false;
         maybeWarnAboutCodeInRocksDBConfigSetter(log, config);
         rocksDBMetricsRecordingService = 
maybeCreateRocksDBMetricsRecordingService(clientId, config);
     }
 
+    /**
+     * 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() {
+        synchronized (stateLock) {

Review comment:
       Why do we need to synchronize the whole method on `stateLock`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,11 +904,72 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
         queryableStoreProvider = new QueryableStoreProvider(storeProviders, 
globalStateStoreProvider);
 
         stateDirCleaner = setupStateDirCleaner();
-        oldHandler = false;
         maybeWarnAboutCodeInRocksDBConfigSetter(log, config);
         rocksDBMetricsRecordingService = 
maybeCreateRocksDBMetricsRecordingService(clientId, config);
     }
 
+    /**
+     * 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() {
+        synchronized (stateLock) {
+            if (state == State.RUNNING || state == State.REBALANCING) {

Review comment:
       Could we also use `isRunningOrRebalancing()` here?

##########
File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
##########
@@ -588,6 +592,29 @@ public void testCloseIsIdempotent() {
             closeCount, MockMetricsReporter.CLOSE_COUNT.get());
     }
 
+    @Test
+    public void testAddThread() {

Review comment:
       I would prefer to use `shouldAddThread()` as name although the pattern 
is different for the other test methods.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,11 +904,72 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
         queryableStoreProvider = new QueryableStoreProvider(storeProviders, 
globalStateStoreProvider);
 
         stateDirCleaner = setupStateDirCleaner();
-        oldHandler = false;
         maybeWarnAboutCodeInRocksDBConfigSetter(log, config);
         rocksDBMetricsRecordingService = 
maybeCreateRocksDBMetricsRecordingService(clientId, config);
     }
 
+    /**
+     * 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() {
+        synchronized (stateLock) {
+            if (state == State.RUNNING || state == State.REBALANCING) {
+                final int threadIdx = getNextThreadIndex();
+                final long cacheSizePerThread = 
getCacheSizePerThread(threads.size() + 1);
+                resizeThreadCache(threads.size() + 1);
+                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));

Review comment:
       We do something really similar when we start the stream threads at 
startup. Could you try to extract this part to a method?

##########
File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
##########
@@ -588,6 +592,29 @@ public void testCloseIsIdempotent() {
             closeCount, MockMetricsReporter.CLOSE_COUNT.get());
     }
 
+    @Test
+    public void testAddThread() {
+        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();
+        try {
+            TestUtils.waitForCondition(() -> streams.state() == 
KafkaStreams.State.RUNNING, 15L, "wait until running");
+        } catch (final InterruptedException e) {
+            e.printStackTrace();
+        }
+        assertThat(streams.addStreamThread(), 
equalTo(Optional.of("newThread")));
+        assertThat(streams.threads.size(), equalTo(oldSize + 1));
+    }
+
+    @Test
+    public void testAddThreadNotDuringStart() {

Review comment:
       I would prefer to use `shouldNotAddThread()` as name although the 
pattern is different for the other test methods.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -894,11 +904,72 @@ private KafkaStreams(final InternalTopologyBuilder 
internalTopologyBuilder,
         queryableStoreProvider = new QueryableStoreProvider(storeProviders, 
globalStateStoreProvider);
 
         stateDirCleaner = setupStateDirCleaner();
-        oldHandler = false;
         maybeWarnAboutCodeInRocksDBConfigSetter(log, config);
         rocksDBMetricsRecordingService = 
maybeCreateRocksDBMetricsRecordingService(clientId, config);
     }
 
+    /**
+     * 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() {

Review comment:
       Additionally to the unit tests that you wrote, I think we also need 
integration 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