guozhangwang commented on a change in pull request #11738: URL: https://github.com/apache/kafka/pull/11738#discussion_r809378947
########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java ########## @@ -316,6 +315,14 @@ StreamsProducer threadProducer() { return activeTaskCreator.producerClientIds(); } + Consumer<byte[], byte[]> mainConsumer() { + return mainConsumer; + } + + TopologyMetadata topologyMetadata() { Review comment: nit: maybe we can just pass in the processingMode to the constructor of the `TaskExecutor` since we only need that single flag, and not the whole topology metadata. ########## File path: streams/src/main/java/org/apache/kafka/streams/internals/StreamsConfigUtils.java ########## @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.internals; + +import org.apache.kafka.streams.StreamsConfig; + +public class StreamsConfigUtils { Review comment: Thanks for moving it out of StreamThread, this is much cleaner! ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java ########## @@ -103,20 +102,21 @@ final StandbyTaskCreator standbyTaskCreator, final TopologyMetadata topologyMetadata, final Admin adminClient, - final StateDirectory stateDirectory, - final StreamThread.ProcessingMode processingMode) { + final StateDirectory stateDirectory) { this.time = time; this.changelogReader = changelogReader; this.processId = processId; this.logPrefix = logPrefix; this.topologyMetadata = topologyMetadata; this.adminClient = adminClient; this.stateDirectory = stateDirectory; - this.processingMode = processingMode; - this.tasks = new Tasks(logPrefix, topologyMetadata, streamsMetrics, activeTaskCreator, standbyTaskCreator); + this.processingMode = topologyMetadata.processingMode(); final LogContext logContext = new LogContext(logPrefix); - log = logContext.logger(getClass()); + this.log = logContext.logger(getClass()); + + this.tasks = new Tasks(logContext, topologyMetadata, streamsMetrics, activeTaskCreator, standbyTaskCreator); + this.taskExecutor = new TaskExecutor(tasks, logContext); Review comment: See my other comment: how about just passing in `topologyMetadata.processMode()` here if we do not expect it to be dynamically changable? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java ########## @@ -316,6 +315,14 @@ StreamsProducer threadProducer() { return activeTaskCreator.producerClientIds(); } + Consumer<byte[], byte[]> mainConsumer() { Review comment: Not a modification suggestion: for main consumer though, compared to topologymetadata below I think it's better indeed to expose it instead of passing in to `TaskExecutor` since we may have the consumer to be owned by a different thread in the future and committing would not be executed by the stream thread. So keeping here is preferred. -- 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. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org