You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/12/01 02:29:57 UTC

[GitHub] [kafka] mjsax commented on a change in pull request #9615: KAFKA-10500: Add thread option

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



##########
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:
       nit. remove unnecessary `this.`

##########
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:
       Nit: can we change the loop to `int = 1; i <= numStreamThreads` and just pass in `i` here?

##########
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:
       Can we create the `StreamStateListener` before we call `createStreamThread` and do `setStateListener` within `createStreamThread` ? If yes, we also don't need to call `setStateListener` within `addStreamThread`

##########
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:
       `shouldNotAddThreadWhenCreated`
   
   Should we also `close()` `KafkaStreams` and check that adding a thread is not possible and/or even put the client into ERROR state and test?

##########
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:
       Why 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) {

Review comment:
       Don't we need to get the `stateLock` as an outer most guard (and not check `isRunningOrRebalancing()` twice)? It seems weird to create a thread but later not start it and throw it away -- especially because we resize the caches (but also don't undo the resizing)?

##########
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.
+     *

Review comment:
       Add missing `<p>` tag

##########
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:
       Why do we compute the names from scratch, but not incrementally maintain them as member variable?

##########
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:
       Just to clarify for myself: if we don't `start()` the thread, no harm is done creating it? Or would we need to do some cleanup even if we don't start the thread?

##########
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:
       As above.

##########
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:
       Should we link to `StreamConfig` instead?

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

Review comment:
       `shouldAddThreadWhenRunning`




----------------------------------------------------------------
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:
users@infra.apache.org