lucasbru commented on code in PR #15852: URL: https://github.com/apache/kafka/pull/15852#discussion_r1589251704
########## streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java: ########## @@ -191,14 +193,20 @@ private void performActionsOnTasks() { tasksAndActionsLock.lock(); try { for (final TaskAndAction taskAndAction : getTasksAndActions()) { - final Action action = taskAndAction.getAction(); + final Action action = taskAndAction.action(); switch (action) { case ADD: - addTask(taskAndAction.getTask()); + addTask(taskAndAction.task()); break; case REMOVE: - removeTask(taskAndAction.getTaskId()); + if (taskAndAction.futureForRemove() == null) { Review Comment: Do we need the variant without the future? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java: ########## @@ -496,6 +504,103 @@ private void addTask(final Task task) { } } + private void removeTask(final TaskId taskId, final CompletableFuture<RemovedTaskResult> future) { + try { + if (updatingTasks.containsKey(taskId)) { + removeUpdatingTask(taskId, future); Review Comment: Could all four `remove...Task` methods just return a boolean if successful? ########## streams/src/main/java/org/apache/kafka/streams/processor/internals/StateUpdater.java: ########## @@ -103,6 +149,17 @@ public String toString() { */ void remove(final TaskId taskId); + /** + * Removes a task (active or standby) from the state updater. + * + * This method does not block until the removed task is removed from the state updater. But it returns a future on + * which processing can be blocked. The task to remove is removed from the updating tasks, paused tasks, + * restored tasks, or failed tasks. + * + * @param taskId ID of the task to remove + */ + CompletableFuture<RemovedTaskResult> removeWithFuture(final TaskId taskId); Review Comment: removeAsync? -- 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