guozhangwang commented on a change in pull request #11787:
URL: https://github.com/apache/kafka/pull/11787#discussion_r812260659



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadata.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.processor.internals;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.errors.UnknownTopologyException;
+import org.apache.kafka.streams.processor.TaskId;
+
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import org.slf4j.Logger;
+
+import static 
org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
+
+/**
+ * Multi-threaded class that tracks the status of active tasks being 
processed. A single instance of this class is
+ * shared between all StreamThreads.
+ */
+public class TaskExecutionMetadata {
+    private Logger log;
+
+    private final boolean hasNamedTopologies;
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToMetadata = new ConcurrentHashMap<>();
+
+    public TaskExecutionMetadata(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+        allTopologyNames.forEach(name -> topologyNameToMetadata.put(name, new 
NamedTopologyMetadata(name)));
+    }
+
+    public void setLog(final LogContext logContext) {
+        log = logContext.logger(getClass());
+    }
+
+    public boolean canProcessTopology(final String topologyName) {
+        if (!hasNamedTopologies) {

Review comment:
       Should we check under if that the topologyName should always be 
`UNNAMED_TOPOLOGY`, or `null`?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutor.java
##########
@@ -63,8 +68,21 @@ public TaskExecutor(final Tasks tasks, final ProcessingMode 
processingMode, fina
     int process(final int maxNumRecords, final Time time) {
         int totalProcessed = 0;
 
-        for (final Task task : tasks.activeTasks()) {
-            totalProcessed += processTask(task, maxNumRecords, time);
+        for (final Map.Entry<String, Set<StreamTask>> topologyEntry : 
tasks.activeTasksByTopology().entrySet()) {

Review comment:
       I think it's a bit overkill to first organize all tasks into 
`activeTasksByTopology` also may have unexpected scheduling bias compared to 
what we did today as more or less random-roundrobin. What about just checking 
for each task, if `canProcess(taskName) && 
canProcessTopology(task.topologyName())`?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadata.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.processor.internals;
+
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.streams.errors.UnknownTopologyException;
+import org.apache.kafka.streams.processor.TaskId;
+
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import org.slf4j.Logger;
+
+import static 
org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
+
+/**
+ * Multi-threaded class that tracks the status of active tasks being 
processed. A single instance of this class is
+ * shared between all StreamThreads.
+ */
+public class TaskExecutionMetadata {

Review comment:
       Thanks for bringing this up, I think we can come back and clean this up 
after we've gained confidence and is ready to extend beyond named topology 
later.




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


Reply via email to