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



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadata.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.processor.TaskId;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+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 final boolean hasNamedTopologies;
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+
+    public TaskExecutionMetadata(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public boolean canProcessTask(final Task task, final long now) {
+        final String topologyName = task.id().topologyName();
+        if (!hasNamedTopologies) {
+            // TODO implement error handling/backoff for non-named topologies 
(needs KIP)
+            return true;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            return metadata == null || (metadata.canProcess() && 
metadata.canProcessTask(task, now));
+        }
+    }
+
+    public void registerTaskError(final Task task, final Throwable t, final 
long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            topologyNameToErrorMetadata.computeIfAbsent(topologyName, n -> new 
NamedTopologyMetadata(topologyName))

Review comment:
       Changed it so that we only track topologies in the map here if they have 
an active backoff/task in error, rather than registering and unregistering 
named topologies and trying to keep this in sync between the TopologyMetadata 
and the individual StreamThreads' view (which was starting to look pretty ugly)
   
   Instead we just pop the topology's metadata into the map when one of its 
tasks hits a new error, and clear it if/when all tasks are healthy again




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