guozhangwang commented on code in PR #12044:
URL: https://github.com/apache/kafka/pull/12044#discussion_r850797609


##########
clients/src/test/java/org/apache/kafka/common/utils/ExponentialBackoffTest.java:
##########
@@ -20,30 +20,42 @@
 import org.junit.jupiter.api.Test;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class ExponentialBackoffTest {
     @Test
     public void testExponentialBackoff() {
-        long scaleFactor = 100;
+        long scaleFactor = 3000;
         int ratio = 2;
-        long backoffMax = 2000;
-        double jitter = 0.2;
+        long backoffMax = 60 * 1000;
+        double jitter = 0.02;
         ExponentialBackoff exponentialBackoff = new ExponentialBackoff(
                 scaleFactor, ratio, backoffMax, jitter
         );
 
-        for (int i = 0; i <= 100; i++) {
-            for (int attempts = 0; attempts <= 10; attempts++) {
-                if (attempts <= 4) {
-                    assertEquals(scaleFactor * Math.pow(ratio, attempts),
-                            exponentialBackoff.backoff(attempts),
-                            scaleFactor * Math.pow(ratio, attempts) * jitter);
-                } else {
-                    assertTrue(exponentialBackoff.backoff(attempts) <= 
backoffMax * (1 + jitter));
-                }
-            }
-        }
+
+        int numAttempts = 0;
+        System.out.println(String.format("backoff with num attempts = %s is 
%s", numAttempts, exponentialBackoff.backoff(numAttempts)));

Review Comment:
   Is this println intentional? Also we seems do not have any assertions 
anymore.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskScheduler.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.ExponentialBackoff;
+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 
and decides if/when they can
+ * be executed.
+ *
+ * Note: A single instance of this class is shared between all StreamThreads, 
so it must be thread-safe
+ */
+public class TaskScheduler {
+    // Reaches the maximum backoff interval in about 5 attempts, at which point
+    private static final long INITIAL_BACKOFF_MS = 3 * 1000L;  // wait 3s 
after the first task failure
+    private static final int RETRY_BACKOFF_EXP_BASE = 2;
+    private static final long MAXIMUM_BACKOFF_MS = 60 * 1000L; // back off up 
to a maximum of 1 minute between retries
+    private static final double RETRY_BACKOFF_JITTER = 0.02;
+
+    // Once an individual task has been failing consistently for the past 30s 
or more, we "extend" the backoff to all
+    // other tasks in that topology, to avoid letting different subtopologies 
become too far out of sync and potentially
+    // missing for example the output of a join where only one side is blocked 
upstream due to task errors
+    private static final long EXTEND_TASK_BACKOFF_TO_FULL_TOPOLOGY_MS = 30 * 
1000L;
+
+    enum TaskStatus {
+        RUNNING,  // the task and its topology are healthy and able to be 
processed
+        BACKOFF,  // the task and/or its topology are unhealthy, task has 
remaining backoff time to wait before a retry
+        RETRIABLE // the task and/or its topology are still considered 
unhealthy but are ready to be retried
+    }
+
+    private final ExponentialBackoff taskBackoff = new ExponentialBackoff(
+        INITIAL_BACKOFF_MS,
+        RETRY_BACKOFF_EXP_BASE,
+        MAXIMUM_BACKOFF_MS,
+        RETRY_BACKOFF_JITTER
+    );
+
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+    private final boolean hasNamedTopologies;
+
+    public TaskScheduler(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public TaskStatus getTaskStatus(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 TaskStatus.RUNNING;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            if (metadata == null) {
+                return TaskStatus.RUNNING;
+            } else {
+                return metadata.getTaskStatus(task, now);
+            }
+        }
+    }
+
+    public void registerTaskError(final Task task, final long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            // Only need to register this error if the task is currently 
healthy
+            if (!topologyNameToErrorMetadata.containsKey(topologyName)) {
+                final NamedTopologyMetadata namedTopologyMetadata = new 
NamedTopologyMetadata(topologyName);
+                namedTopologyMetadata.registerTaskError(task, now);
+                topologyNameToErrorMetadata.put(topologyName, 
namedTopologyMetadata);
+            }
+        }
+    }
+
+    public void registerTaskSuccess(final Task task) {
+        if (hasNamedTopologies && task.id().topologyName() != null) {
+            final NamedTopologyMetadata topologyMetadata = 
topologyNameToErrorMetadata.get(task.id().topologyName());
+            if (topologyMetadata != null) {
+                topologyMetadata.registerRetrySuccess(task);
+            }
+        }
+    }
+
+    private class NamedTopologyMetadata {
+        private final Logger log;
+        private final Map<TaskId, ErrorMetadata> tasksToErrorTime = new 
ConcurrentHashMap<>();
+
+        public NamedTopologyMetadata(final String topologyName) {
+            final LogContext logContext = new 
LogContext(String.format("topology-name [%s] ", topologyName));
+            this.log = logContext.logger(NamedTopologyMetadata.class);
+        }
+
+        public TaskStatus getTaskStatus(final Task task, final long now) {
+            final ErrorMetadata errorMetadata = 
tasksToErrorTime.get(task.id());
+            if (errorMetadata != null) {
+                ++errorMetadata.numAttempts;
+
+                final long remainingBackoffMs = now - 
errorMetadata.firstErrorTimeMs + taskBackoff.backoff(errorMetadata.numAttempts);

Review Comment:
   The logic here seems incorrect to me: the remaining backoff should be
   
   ```
   taskBackoff.backoff(errorMetadata.numAttempts) - (now - 
errorMetadata.firstErrorTimeM)
   
   = taskBackoff.backoff(errorMetadata.numAttempts) - now + 
errorMetadata.firstErrorTimeM
   ```
   
   right?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskScheduler.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.ExponentialBackoff;
+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 
and decides if/when they can
+ * be executed.
+ *
+ * Note: A single instance of this class is shared between all StreamThreads, 
so it must be thread-safe
+ */
+public class TaskScheduler {
+    // Reaches the maximum backoff interval in about 5 attempts, at which point
+    private static final long INITIAL_BACKOFF_MS = 3 * 1000L;  // wait 3s 
after the first task failure
+    private static final int RETRY_BACKOFF_EXP_BASE = 2;
+    private static final long MAXIMUM_BACKOFF_MS = 60 * 1000L; // back off up 
to a maximum of 1 minute between retries
+    private static final double RETRY_BACKOFF_JITTER = 0.02;
+
+    // Once an individual task has been failing consistently for the past 30s 
or more, we "extend" the backoff to all
+    // other tasks in that topology, to avoid letting different subtopologies 
become too far out of sync and potentially
+    // missing for example the output of a join where only one side is blocked 
upstream due to task errors
+    private static final long EXTEND_TASK_BACKOFF_TO_FULL_TOPOLOGY_MS = 30 * 
1000L;
+
+    enum TaskStatus {
+        RUNNING,  // the task and its topology are healthy and able to be 
processed
+        BACKOFF,  // the task and/or its topology are unhealthy, task has 
remaining backoff time to wait before a retry
+        RETRIABLE // the task and/or its topology are still considered 
unhealthy but are ready to be retried
+    }
+
+    private final ExponentialBackoff taskBackoff = new ExponentialBackoff(
+        INITIAL_BACKOFF_MS,
+        RETRY_BACKOFF_EXP_BASE,
+        MAXIMUM_BACKOFF_MS,
+        RETRY_BACKOFF_JITTER
+    );
+
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+    private final boolean hasNamedTopologies;
+
+    public TaskScheduler(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public TaskStatus getTaskStatus(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 TaskStatus.RUNNING;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            if (metadata == null) {
+                return TaskStatus.RUNNING;
+            } else {
+                return metadata.getTaskStatus(task, now);
+            }
+        }
+    }
+
+    public void registerTaskError(final Task task, final long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            // Only need to register this error if the task is currently 
healthy
+            if (!topologyNameToErrorMetadata.containsKey(topologyName)) {
+                final NamedTopologyMetadata namedTopologyMetadata = new 
NamedTopologyMetadata(topologyName);
+                namedTopologyMetadata.registerTaskError(task, now);
+                topologyNameToErrorMetadata.put(topologyName, 
namedTopologyMetadata);
+            }
+        }
+    }
+
+    public void registerTaskSuccess(final Task task) {
+        if (hasNamedTopologies && task.id().topologyName() != null) {
+            final NamedTopologyMetadata topologyMetadata = 
topologyNameToErrorMetadata.get(task.id().topologyName());
+            if (topologyMetadata != null) {
+                topologyMetadata.registerRetrySuccess(task);
+            }
+        }
+    }
+
+    private class NamedTopologyMetadata {
+        private final Logger log;
+        private final Map<TaskId, ErrorMetadata> tasksToErrorTime = new 
ConcurrentHashMap<>();
+
+        public NamedTopologyMetadata(final String topologyName) {
+            final LogContext logContext = new 
LogContext(String.format("topology-name [%s] ", topologyName));
+            this.log = logContext.logger(NamedTopologyMetadata.class);
+        }
+
+        public TaskStatus getTaskStatus(final Task task, final long now) {
+            final ErrorMetadata errorMetadata = 
tasksToErrorTime.get(task.id());
+            if (errorMetadata != null) {
+                ++errorMetadata.numAttempts;
+
+                final long remainingBackoffMs = now - 
errorMetadata.firstErrorTimeMs + taskBackoff.backoff(errorMetadata.numAttempts);
+
+                if (remainingBackoffMs > 0) {

Review Comment:
   If my previous comment is reasonable, this should be `remainingBackoffMs <= 
0` right?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java:
##########
@@ -166,8 +166,8 @@ public void unregisterThread(final String threadName) {
         maybeNotifyTopologyVersionListeners();
     }
 
-    public TaskExecutionMetadata taskExecutionMetadata() {
-        return taskExecutionMetadata;
+    public TaskScheduler taskExecutionMetadata() {

Review Comment:
   nit: rename the function too?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskScheduler.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.ExponentialBackoff;
+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 
and decides if/when they can
+ * be executed.
+ *
+ * Note: A single instance of this class is shared between all StreamThreads, 
so it must be thread-safe
+ */
+public class TaskScheduler {
+    // Reaches the maximum backoff interval in about 5 attempts, at which point
+    private static final long INITIAL_BACKOFF_MS = 3 * 1000L;  // wait 3s 
after the first task failure
+    private static final int RETRY_BACKOFF_EXP_BASE = 2;
+    private static final long MAXIMUM_BACKOFF_MS = 60 * 1000L; // back off up 
to a maximum of 1 minute between retries
+    private static final double RETRY_BACKOFF_JITTER = 0.02;
+
+    // Once an individual task has been failing consistently for the past 30s 
or more, we "extend" the backoff to all
+    // other tasks in that topology, to avoid letting different subtopologies 
become too far out of sync and potentially
+    // missing for example the output of a join where only one side is blocked 
upstream due to task errors
+    private static final long EXTEND_TASK_BACKOFF_TO_FULL_TOPOLOGY_MS = 30 * 
1000L;
+
+    enum TaskStatus {
+        RUNNING,  // the task and its topology are healthy and able to be 
processed
+        BACKOFF,  // the task and/or its topology are unhealthy, task has 
remaining backoff time to wait before a retry
+        RETRIABLE // the task and/or its topology are still considered 
unhealthy but are ready to be retried
+    }
+
+    private final ExponentialBackoff taskBackoff = new ExponentialBackoff(
+        INITIAL_BACKOFF_MS,
+        RETRY_BACKOFF_EXP_BASE,
+        MAXIMUM_BACKOFF_MS,
+        RETRY_BACKOFF_JITTER
+    );
+
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+    private final boolean hasNamedTopologies;
+
+    public TaskScheduler(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public TaskStatus getTaskStatus(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 TaskStatus.RUNNING;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            if (metadata == null) {
+                return TaskStatus.RUNNING;
+            } else {
+                return metadata.getTaskStatus(task, now);
+            }
+        }
+    }
+
+    public void registerTaskError(final Task task, final long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            // Only need to register this error if the task is currently 
healthy
+            if (!topologyNameToErrorMetadata.containsKey(topologyName)) {
+                final NamedTopologyMetadata namedTopologyMetadata = new 
NamedTopologyMetadata(topologyName);
+                namedTopologyMetadata.registerTaskError(task, now);
+                topologyNameToErrorMetadata.put(topologyName, 
namedTopologyMetadata);
+            }
+        }
+    }
+
+    public void registerTaskSuccess(final Task task) {
+        if (hasNamedTopologies && task.id().topologyName() != null) {
+            final NamedTopologyMetadata topologyMetadata = 
topologyNameToErrorMetadata.get(task.id().topologyName());
+            if (topologyMetadata != null) {
+                topologyMetadata.registerRetrySuccess(task);
+            }
+        }
+    }
+
+    private class NamedTopologyMetadata {
+        private final Logger log;
+        private final Map<TaskId, ErrorMetadata> tasksToErrorTime = new 
ConcurrentHashMap<>();
+
+        public NamedTopologyMetadata(final String topologyName) {
+            final LogContext logContext = new 
LogContext(String.format("topology-name [%s] ", topologyName));
+            this.log = logContext.logger(NamedTopologyMetadata.class);
+        }
+
+        public TaskStatus getTaskStatus(final Task task, final long now) {
+            final ErrorMetadata errorMetadata = 
tasksToErrorTime.get(task.id());
+            if (errorMetadata != null) {
+                ++errorMetadata.numAttempts;
+
+                final long remainingBackoffMs = now - 
errorMetadata.firstErrorTimeMs + taskBackoff.backoff(errorMetadata.numAttempts);
+
+                if (remainingBackoffMs > 0) {
+                    log.info("Task {} is ready to re-attempt processing, retry 
attempt count is {}",
+                             task.id(), errorMetadata.numAttempts);
+                    return TaskStatus.RETRIABLE;
+                } else {
+                    log.debug("Skip processing for task {} with remaining 
backoff time = {}ms",
+                              task.id(), remainingBackoffMs);
+                    return TaskStatus.BACKOFF;
+                }
+            } else {
+                // TODO: implement full backoff for unhealthy topologies with 
high-frequency task errors
+
+                return TaskStatus.RUNNING;
+            }
+        }
+
+        public synchronized void registerTaskError(final Task task, final long 
now) {

Review Comment:
   Why these two functions are synchronized but not `getTaskStatus` above? I 
feel the `synchronized` here and below are not necessary.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskScheduler.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.ExponentialBackoff;
+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 
and decides if/when they can
+ * be executed.
+ *
+ * Note: A single instance of this class is shared between all StreamThreads, 
so it must be thread-safe
+ */
+public class TaskScheduler {
+    // Reaches the maximum backoff interval in about 5 attempts, at which point
+    private static final long INITIAL_BACKOFF_MS = 3 * 1000L;  // wait 3s 
after the first task failure
+    private static final int RETRY_BACKOFF_EXP_BASE = 2;
+    private static final long MAXIMUM_BACKOFF_MS = 60 * 1000L; // back off up 
to a maximum of 1 minute between retries
+    private static final double RETRY_BACKOFF_JITTER = 0.02;
+
+    // Once an individual task has been failing consistently for the past 30s 
or more, we "extend" the backoff to all
+    // other tasks in that topology, to avoid letting different subtopologies 
become too far out of sync and potentially
+    // missing for example the output of a join where only one side is blocked 
upstream due to task errors
+    private static final long EXTEND_TASK_BACKOFF_TO_FULL_TOPOLOGY_MS = 30 * 
1000L;
+
+    enum TaskStatus {
+        RUNNING,  // the task and its topology are healthy and able to be 
processed
+        BACKOFF,  // the task and/or its topology are unhealthy, task has 
remaining backoff time to wait before a retry
+        RETRIABLE // the task and/or its topology are still considered 
unhealthy but are ready to be retried
+    }
+
+    private final ExponentialBackoff taskBackoff = new ExponentialBackoff(
+        INITIAL_BACKOFF_MS,
+        RETRY_BACKOFF_EXP_BASE,
+        MAXIMUM_BACKOFF_MS,
+        RETRY_BACKOFF_JITTER
+    );
+
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+    private final boolean hasNamedTopologies;
+
+    public TaskScheduler(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public TaskStatus getTaskStatus(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 TaskStatus.RUNNING;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            if (metadata == null) {
+                return TaskStatus.RUNNING;
+            } else {
+                return metadata.getTaskStatus(task, now);
+            }
+        }
+    }
+
+    public void registerTaskError(final Task task, final long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            // Only need to register this error if the task is currently 
healthy
+            if (!topologyNameToErrorMetadata.containsKey(topologyName)) {
+                final NamedTopologyMetadata namedTopologyMetadata = new 
NamedTopologyMetadata(topologyName);
+                namedTopologyMetadata.registerTaskError(task, now);
+                topologyNameToErrorMetadata.put(topologyName, 
namedTopologyMetadata);
+            }
+        }
+    }
+
+    public void registerTaskSuccess(final Task task) {
+        if (hasNamedTopologies && task.id().topologyName() != null) {
+            final NamedTopologyMetadata topologyMetadata = 
topologyNameToErrorMetadata.get(task.id().topologyName());
+            if (topologyMetadata != null) {
+                topologyMetadata.registerRetrySuccess(task);
+            }
+        }
+    }
+
+    private class NamedTopologyMetadata {
+        private final Logger log;
+        private final Map<TaskId, ErrorMetadata> tasksToErrorTime = new 
ConcurrentHashMap<>();
+
+        public NamedTopologyMetadata(final String topologyName) {
+            final LogContext logContext = new 
LogContext(String.format("topology-name [%s] ", topologyName));
+            this.log = logContext.logger(NamedTopologyMetadata.class);
+        }
+
+        public TaskStatus getTaskStatus(final Task task, final long now) {
+            final ErrorMetadata errorMetadata = 
tasksToErrorTime.get(task.id());
+            if (errorMetadata != null) {
+                ++errorMetadata.numAttempts;

Review Comment:
   I thought we would only increment `numAttempts` when we fail the task 
(repeatedly), but not whenever we call `getTaskStatus`. Otherwise the backoff 
would keep increasing as we call `getTaskStatus` repeatedly right?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskScheduler.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.ExponentialBackoff;
+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 
and decides if/when they can
+ * be executed.
+ *
+ * Note: A single instance of this class is shared between all StreamThreads, 
so it must be thread-safe
+ */
+public class TaskScheduler {
+    // Reaches the maximum backoff interval in about 5 attempts, at which point
+    private static final long INITIAL_BACKOFF_MS = 3 * 1000L;  // wait 3s 
after the first task failure
+    private static final int RETRY_BACKOFF_EXP_BASE = 2;
+    private static final long MAXIMUM_BACKOFF_MS = 60 * 1000L; // back off up 
to a maximum of 1 minute between retries
+    private static final double RETRY_BACKOFF_JITTER = 0.02;
+
+    // Once an individual task has been failing consistently for the past 30s 
or more, we "extend" the backoff to all
+    // other tasks in that topology, to avoid letting different subtopologies 
become too far out of sync and potentially
+    // missing for example the output of a join where only one side is blocked 
upstream due to task errors
+    private static final long EXTEND_TASK_BACKOFF_TO_FULL_TOPOLOGY_MS = 30 * 
1000L;
+
+    enum TaskStatus {
+        RUNNING,  // the task and its topology are healthy and able to be 
processed
+        BACKOFF,  // the task and/or its topology are unhealthy, task has 
remaining backoff time to wait before a retry
+        RETRIABLE // the task and/or its topology are still considered 
unhealthy but are ready to be retried
+    }
+
+    private final ExponentialBackoff taskBackoff = new ExponentialBackoff(
+        INITIAL_BACKOFF_MS,
+        RETRY_BACKOFF_EXP_BASE,
+        MAXIMUM_BACKOFF_MS,
+        RETRY_BACKOFF_JITTER
+    );
+
+    // map of topologies experiencing errors/currently under backoff
+    private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+    private final boolean hasNamedTopologies;
+
+    public TaskScheduler(final Set<String> allTopologyNames) {
+        this.hasNamedTopologies = !(allTopologyNames.size() == 1 && 
allTopologyNames.contains(UNNAMED_TOPOLOGY));
+    }
+
+    public TaskStatus getTaskStatus(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 TaskStatus.RUNNING;
+        } else {
+            final NamedTopologyMetadata metadata = 
topologyNameToErrorMetadata.get(topologyName);
+            if (metadata == null) {
+                return TaskStatus.RUNNING;
+            } else {
+                return metadata.getTaskStatus(task, now);
+            }
+        }
+    }
+
+    public void registerTaskError(final Task task, final long now) {
+        if (hasNamedTopologies) {
+            final String topologyName = task.id().topologyName();
+            // Only need to register this error if the task is currently 
healthy
+            if (!topologyNameToErrorMetadata.containsKey(topologyName)) {
+                final NamedTopologyMetadata namedTopologyMetadata = new 
NamedTopologyMetadata(topologyName);
+                namedTopologyMetadata.registerTaskError(task, now);
+                topologyNameToErrorMetadata.put(topologyName, 
namedTopologyMetadata);
+            }
+        }
+    }
+
+    public void registerTaskSuccess(final Task task) {
+        if (hasNamedTopologies && task.id().topologyName() != null) {
+            final NamedTopologyMetadata topologyMetadata = 
topologyNameToErrorMetadata.get(task.id().topologyName());
+            if (topologyMetadata != null) {
+                topologyMetadata.registerRetrySuccess(task);
+            }
+        }
+    }
+
+    private class NamedTopologyMetadata {
+        private final Logger log;
+        private final Map<TaskId, ErrorMetadata> tasksToErrorTime = new 
ConcurrentHashMap<>();
+
+        public NamedTopologyMetadata(final String topologyName) {
+            final LogContext logContext = new 
LogContext(String.format("topology-name [%s] ", topologyName));
+            this.log = logContext.logger(NamedTopologyMetadata.class);
+        }
+
+        public TaskStatus getTaskStatus(final Task task, final long now) {
+            final ErrorMetadata errorMetadata = 
tasksToErrorTime.get(task.id());
+            if (errorMetadata != null) {
+                ++errorMetadata.numAttempts;
+
+                final long remainingBackoffMs = now - 
errorMetadata.firstErrorTimeMs + taskBackoff.backoff(errorMetadata.numAttempts);
+
+                if (remainingBackoffMs > 0) {
+                    log.info("Task {} is ready to re-attempt processing, retry 
attempt count is {}",
+                             task.id(), errorMetadata.numAttempts);
+                    return TaskStatus.RETRIABLE;
+                } else {
+                    log.debug("Skip processing for task {} with remaining 
backoff time = {}ms",
+                              task.id(), remainingBackoffMs);
+                    return TaskStatus.BACKOFF;
+                }
+            } else {
+                // TODO: implement full backoff for unhealthy topologies with 
high-frequency task errors

Review Comment:
   I'm not sure I follow this TODO here: why it's related to `if (errorMetadata 
== null)` condition?



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