You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/07/26 14:48:10 UTC

[GitHub] [kafka] cadonna opened a new pull request, #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

cadonna opened a new pull request, #12442:
URL: https://github.com/apache/kafka/pull/12442

   Bookkeeps tasks to be recycled, closed, and updated during handling of
   the assignment. The bookkeeping is needed for integrating the
   state updater.
   
   These change is hidden behind internal config STATE_UPDATER_ENABLED.
   If the config is false Streams should not use the state updater
   and behave as usual.
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] guozhangwang merged pull request #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

Posted by GitBox <gi...@apache.org>.
guozhangwang merged PR #12442:
URL: https://github.com/apache/kafka/pull/12442


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cadonna commented on a diff in pull request #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

Posted by GitBox <gi...@apache.org>.
cadonna commented on code in PR #12442:
URL: https://github.com/apache/kafka/pull/12442#discussion_r931901447


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -380,6 +346,104 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks,
         tasks.createTasks(activeTasksToCreate, standbyTasksToCreate);
     }
 
+    private void classifyTasksWithoutStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                                  final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                                  final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                                  final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, activeTasksToCreate.get(taskId));
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyRunningTasks(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                      final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                      final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                      final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyTasksWithStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                               final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                               final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                               final Set<Task> tasksToCloseClean) {
+        classifyRunningTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean);
+        for (final Task task : stateUpdater.getTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);
+                    }
+                } else {
+                    stateUpdater.remove(taskId);
+                    tasks.addPendingStandbyTaskToRecycle(taskId);
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);

Review Comment:
   I replicated the existing behavior here. What would happen if we remove the standby task to recycle it and the input partitions changed? I do not think we check the input partitions during recycling.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cadonna commented on a diff in pull request #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

Posted by GitBox <gi...@apache.org>.
cadonna commented on code in PR #12442:
URL: https://github.com/apache/kafka/pull/12442#discussion_r931907686


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -380,6 +346,104 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks,
         tasks.createTasks(activeTasksToCreate, standbyTasksToCreate);
     }
 
+    private void classifyTasksWithoutStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                                  final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                                  final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                                  final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, activeTasksToCreate.get(taskId));
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyRunningTasks(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                      final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                      final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                      final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyTasksWithStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                               final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                               final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                               final Set<Task> tasksToCloseClean) {
+        classifyRunningTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean);
+        for (final Task task : stateUpdater.getTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);

Review Comment:
   I think such a comment will be out-dated quite quickly since when we finish this work we will remove the old code path.
   One idea that came to my mind now: Maybe we should provide an unmodifiable version of active and standby tasks so that we can ensure that tasks that are returned from `stateUpdater.getTasks()` cannot be modified. In such a way also resuming such a task would throw an exception. I think that would be better than a comment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on code in PR #12442:
URL: https://github.com/apache/kafka/pull/12442#discussion_r930193079


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -396,11 +460,8 @@ private Map<TaskId, Set<TopicPartition>> pendingTasksToCreate(final Map<TaskId,
 
     private void closeAndRecycleTasks(final Map<Task, Set<TopicPartition>> tasksToRecycle,
                                       final Set<Task> tasksToCloseClean,
-                                      final Set<Task> tasksToCloseDirty,

Review Comment:
   Nice catch. Thanks for the cleanup!



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java:
##########
@@ -64,6 +64,11 @@ class Tasks {
     private final Map<TaskId, Set<TopicPartition>> pendingActiveTasks = new HashMap<>();
     private final Map<TaskId, Set<TopicPartition>> pendingStandbyTasks = new HashMap<>();
 
+    private final Set<TaskId> pendingActiveTasksToRecycle = new HashSet<>();

Review Comment:
   Maybe we can then rename the ones in 64/65 above as `pending..TasksToCreate` then?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -380,6 +346,104 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks,
         tasks.createTasks(activeTasksToCreate, standbyTasksToCreate);
     }
 
+    private void classifyTasksWithoutStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                                  final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                                  final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                                  final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, activeTasksToCreate.get(taskId));
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyRunningTasks(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                      final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                      final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                      final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyTasksWithStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                               final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                               final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                               final Set<Task> tasksToCloseClean) {
+        classifyRunningTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean);
+        for (final Task task : stateUpdater.getTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);

Review Comment:
   Maybe we could add a comment here, to emphasize that with the new model, we would not pause restoring tasks during a rebalance, so they would not need to be resumed as well? This is a behavioral change compared to the old code, hence worth mentioning.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -380,6 +346,104 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks,
         tasks.createTasks(activeTasksToCreate, standbyTasksToCreate);
     }
 
+    private void classifyTasksWithoutStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                                  final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                                  final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                                  final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, activeTasksToCreate.get(taskId));
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyRunningTasks(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                      final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                      final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                      final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyTasksWithStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                               final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                               final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                               final Set<Task> tasksToCloseClean) {
+        classifyRunningTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean);
+        for (final Task task : stateUpdater.getTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);
+                    }
+                } else {
+                    stateUpdater.remove(taskId);
+                    tasks.addPendingStandbyTaskToRecycle(taskId);
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);

Review Comment:
   For standby tasks, the tasks are never going to go out of the state updater until it was removed, so input partitions are never used. Hence I think we do not need to add to this set?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] guozhangwang commented on a diff in pull request #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on code in PR #12442:
URL: https://github.com/apache/kafka/pull/12442#discussion_r932639066


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -380,6 +346,104 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks,
         tasks.createTasks(activeTasksToCreate, standbyTasksToCreate);
     }
 
+    private void classifyTasksWithoutStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                                  final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                                  final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                                  final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, activeTasksToCreate.get(taskId));
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = standbyTasksToCreate.get(taskId);
+                    task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    task.resume();
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyRunningTasks(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                      final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                      final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                      final Set<Task> tasksToCloseClean) {
+        for (final Task task : tasks.allTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (tasks.updateActiveTaskInputPartitions(task, topicPartitions)) {
+                        task.updateInputPartitions(topicPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+                    }
+                    task.resume();
+                } else {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                }
+                activeTasksToCreate.remove(taskId);
+            } else if (standbyTasksToCreate.containsKey(taskId)) {
+                if (!task.isActive()) {
+                    throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+                } else {
+                    tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
+                }
+                standbyTasksToCreate.remove(taskId);
+            } else {
+                tasksToCloseClean.add(task);
+            }
+        }
+    }
+
+    private void classifyTasksWithStateUpdater(final Map<TaskId, Set<TopicPartition>> activeTasksToCreate,
+                                               final Map<TaskId, Set<TopicPartition>> standbyTasksToCreate,
+                                               final Map<Task, Set<TopicPartition>> tasksToRecycle,
+                                               final Set<Task> tasksToCloseClean) {
+        classifyRunningTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean);
+        for (final Task task : stateUpdater.getTasks()) {
+            final TaskId taskId = task.id();
+            if (activeTasksToCreate.containsKey(taskId)) {
+                if (task.isActive()) {
+                    final Set<TopicPartition> topicPartitions = activeTasksToCreate.get(taskId);
+                    if (!task.inputPartitions().equals(topicPartitions)) {
+                        tasks.addPendingTaskThatNeedsInputPartitionsUpdate(taskId);

Review Comment:
   Sounds good!



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cadonna commented on a diff in pull request #12442: KAFKA-10199: Bookkeep tasks during assignment for use with state updater

Posted by GitBox <gi...@apache.org>.
cadonna commented on code in PR #12442:
URL: https://github.com/apache/kafka/pull/12442#discussion_r931903062


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java:
##########
@@ -64,6 +64,11 @@ class Tasks {
     private final Map<TaskId, Set<TopicPartition>> pendingActiveTasks = new HashMap<>();
     private final Map<TaskId, Set<TopicPartition>> pendingStandbyTasks = new HashMap<>();
 
+    private final Set<TaskId> pendingActiveTasksToRecycle = new HashSet<>();

Review Comment:
   Since you did that in #12439 I will just rebase this PR.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org