wcarlson5 commented on code in PR #14180: URL: https://github.com/apache/kafka/pull/14180#discussion_r1312206725
########## streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskManagerTest.java: ########## @@ -94,6 +103,114 @@ public void shouldAssignTaskThatCanBeProcessed() { assertNull(taskManager.assignNextTask(taskExecutor)); } + private class AwaitingRunnable implements Runnable { + private final CountDownLatch awaitDone = new CountDownLatch(1); + private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + @Override + public void run() { + while (!shutdownRequested.get()) { + try { + taskManager.awaitProcessableTasks(); + } catch (final InterruptedException ignored) { + } + awaitDone.countDown(); + } + } + + public void shutdown() { + shutdownRequested.set(true); + taskManager.signalProcessableTasks(); + } + } + + @Test + public void shouldBlockOnAwait() throws InterruptedException { + final AwaitingRunnable awaitingRunnable = new AwaitingRunnable(); + final Thread awaitingThread = new Thread(awaitingRunnable); + awaitingThread.start(); + + assertFalse(awaitingRunnable.awaitDone.await(100, TimeUnit.MILLISECONDS)); Review Comment: Do you want to use the `VERIFICATION_TIMEOUT` here? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskManager.java: ########## @@ -118,6 +120,41 @@ public StreamTask assignNextTask(final TaskExecutor executor) { }); } + @Override + public void awaitProcessableTasks() throws InterruptedException { + final boolean interrupted = returnWithTasksLocked(() -> { + for (final Task task : tasks.activeTasks()) { + if (!assignedTasks.containsKey(task.id()) && + !lockedTasks.contains(task.id()) && + canProgress((StreamTask) task, time.milliseconds()) + ) { + log.info("Await unblocked: returning early from await since a processable task {} was found", task.id()); Review Comment: I think info might be too high of a log level. This should be called frequently right? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java: ########## @@ -121,4 +119,14 @@ public interface TaskManager { */ Map<TaskId, StreamsException> drainUncaughtExceptions(); + /** + * Signals that at least one task has become processable, e.g. because it was resumed or new records may be available. Review Comment: I assume this method with be used in later commits? -- 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