wcarlson5 commented on a change in pull request #9615: URL: https://github.com/apache/kafka/pull/9615#discussion_r533656732
########## 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: There will be two more cases of remove. In the replace thread option and in the remove thread option. I'm not really convinced it is necessary but I don't see a problem with re-resizing the cache if we do not start the thread. I don't think there will be any side affects as the client should be shutting down, but if we resize there should be a little extra info in the state and store providers but it would not get used ########## 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: yeah, I didn't realize this was a problem, but when I added more test coverage it showed up ########## 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: I'll remove a few of the unnecessary `+` operations then ########## 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: good catch. I don't think we make sure the thread index starts at 1. But let me fix that ---------------------------------------------------------------- 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