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/09/09 00:21:52 UTC

[GitHub] [kafka] lihaosky commented on a diff in pull request #12600: KAFKA-10199: Remove and suspend tasks in the state updater on revocation

lihaosky commented on code in PR #12600:
URL: https://github.com/apache/kafka/pull/12600#discussion_r966520188


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -960,7 +1011,7 @@ private void removeRevokedTasksFromStateUpdater(final Set<TopicPartition> remain
             for (final Task restoringTask : stateUpdater.getTasks()) {
                 if (restoringTask.isActive()) {
                     if (remainingRevokedPartitions.containsAll(restoringTask.inputPartitions())) {
-                        tasks.addPendingTaskToCloseClean(restoringTask.id());
+                        tasks.addPendingActiveTaskToSuspend(restoringTask.id());

Review Comment:
   noob question: Why do we suspend instead of close?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -421,73 +421,120 @@ private void classifyTasksWithoutStateUpdater(final Map<TaskId, Set<TopicPartiti
         }
     }
 
-    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) {
+    private void classifyRunningAndSuspendedTasks(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()) {
+            if (!task.isActive()) {
+                throw new IllegalStateException("Standby tasks should only be managed by the state updater");
+            }
             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");
-                }
+                handleReAssignedActiveTask(task, activeTasksToCreate.get(taskId));
                 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));
-                }
+                tasksToRecycle.put(task, standbyTasksToCreate.get(taskId));
                 standbyTasksToCreate.remove(taskId);
             } else {
                 tasksToCloseClean.add(task);
             }
         }
     }
 
+    private void handleReAssignedActiveTask(final Task task,
+                                            final Set<TopicPartition> inputPartitions) {
+        if (tasks.updateActiveTaskInputPartitions(task, inputPartitions)) {
+            task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
+        }
+        task.resume();
+        if (task.state() == State.RESTORING) {
+            handleReAssignedRevokedActiveTask(task);
+        }
+    }
+
+    private void handleReAssignedRevokedActiveTask(final Task task) {
+        tasks.removeTask(task);
+        stateUpdater.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);
+        classifyRunningAndSuspendedTasks(activeTasksToCreate, standbyTasksToCreate, tasksToRecycle, tasksToCloseClean);

Review Comment:
   noob question: are taskIds in `activeTasksToCreate` and `standbyTasksToCreate` always mutually exclusive? I guess standby is always disabled if there's only 1 host/node? 



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -960,7 +1011,7 @@ private void removeRevokedTasksFromStateUpdater(final Set<TopicPartition> remain
             for (final Task restoringTask : stateUpdater.getTasks()) {
                 if (restoringTask.isActive()) {
                     if (remainingRevokedPartitions.containsAll(restoringTask.inputPartitions())) {
-                        tasks.addPendingTaskToCloseClean(restoringTask.id());
+                        tasks.addPendingActiveTaskToSuspend(restoringTask.id());
                         stateUpdater.remove(restoringTask.id());
                         remainingRevokedPartitions.removeAll(restoringTask.inputPartitions());

Review Comment:
   Is it possible that task A and B sharing inputPartitions? If so, removeAll won't revoke both A and B?



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