mjsax commented on code in PR #20693:
URL: https://github.com/apache/kafka/pull/20693#discussion_r2496681941


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutionMetadata.java:
##########
@@ -37,20 +37,27 @@
 public class TaskExecutionMetadata {
     // TODO: implement exponential backoff, for now we just wait 5s
     private static final long CONSTANT_BACKOFF_MS = 5_000L;
+    private static final long NOT_READY_LOG_INTERVAL_MS = 120_000L; // Log 
interval of not being ready (2 minutes)
 
     private final boolean hasNamedTopologies;
     private final Set<String> pausedTopologies;
     private final ProcessingMode processingMode;
     private final Collection<Task> successfullyProcessed = new HashSet<>();
     // map of topologies experiencing errors/currently under backoff
     private final ConcurrentHashMap<String, NamedTopologyMetadata> 
topologyNameToErrorMetadata = new ConcurrentHashMap<>();
+    // map of task to last not ready logging time (not present means ready, >= 
0 means not ready and tracks last log time)
+    private final ConcurrentHashMap<TaskId, Long> taskToLastNotReadyLogTime = 
new ConcurrentHashMap<>();
+    // map of task to time when idling started
+    private final ConcurrentHashMap<TaskId, Long> taskToIdlingStartTime = new 
ConcurrentHashMap<>();

Review Comment:
   I don't think we need this idle-timer here -- the difference is, that a user 
can call `KafkaStreams#pause()`, what we track here.
   
   The idle-time inside a task, has a different purpose: it's for the case that 
there is more than one input topic partition for a task, and the task cannot 
make progress because one partitions has no data. For this case, we don't want 
to block the task forever, and unblock it to process the data we have after 
"max idle time" passed.
   
   But if a task is `paused()`, we are not "idel" waiting for new input data.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to