lucasbru commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591121181


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
         tasks.addPendingTaskToCloseClean(taskId);
     }
 
+    private void addToTasksToClose(final Map<TaskId, 
CompletableFuture<StateUpdater.RemovedTaskResult>> futures,
+                                   final Set<Task> 
tasksToCloseCleanFromStateUpdater,
+                                   final Set<Task> 
tasksToCloseDirtyFromStateUpdater) {
+        iterateAndActOnFuture(futures, removedTaskResult -> {
+            final Task task = removedTaskResult.task();
+            final Optional<RuntimeException> exception = 
removedTaskResult.exception();
+            if (exception.isPresent()) {
+                tasksToCloseDirtyFromStateUpdater.add(task);
+            } else {
+                tasksToCloseCleanFromStateUpdater.add(task);
+            }
+        });
+    }
+
+    private void iterateAndActOnFuture(final Map<TaskId, 
CompletableFuture<StateUpdater.RemovedTaskResult>> futures,
+                                       final 
java.util.function.Consumer<StateUpdater.RemovedTaskResult> action) {
+        for (final Map.Entry<TaskId, 
CompletableFuture<StateUpdater.RemovedTaskResult>> entry : futures.entrySet()) {
+            final TaskId taskId = entry.getKey();
+            final CompletableFuture<StateUpdater.RemovedTaskResult> future = 
entry.getValue();
+            try {
+                final StateUpdater.RemovedTaskResult removedTaskResult = 
waitForFuture(taskId, future);
+                action.accept(removedTaskResult);
+            } catch (final ExecutionException executionException) {
+                log.warn("An exception happened when removing task {} from the 
state updater. The exception will be handled later: ",
+                    taskId, executionException);
+            } catch (final InterruptedException ignored) { }

Review Comment:
   Can we just ignore this? I see other classes in the package rethrowing it



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -71,6 +73,10 @@
 import static 
org.apache.kafka.streams.processor.internals.StateManagerUtil.parseTaskDirectoryName;
 
 public class TaskManager {
+
+    private final static String BUG_ERROR_MESSAGE = "This indicates a bug. " +

Review Comment:
   I see that's already defined in a couple of places. Did you check if there 
is a good utility class where this could be defined?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
         tasks.addPendingTaskToCloseClean(taskId);
     }
 
+    private void addToTasksToClose(final Map<TaskId, 
CompletableFuture<StateUpdater.RemovedTaskResult>> futures,
+                                   final Set<Task> 
tasksToCloseCleanFromStateUpdater,
+                                   final Set<Task> 
tasksToCloseDirtyFromStateUpdater) {
+        iterateAndActOnFuture(futures, removedTaskResult -> {
+            final Task task = removedTaskResult.task();
+            final Optional<RuntimeException> exception = 
removedTaskResult.exception();
+            if (exception.isPresent()) {
+                tasksToCloseDirtyFromStateUpdater.add(task);

Review Comment:
   Is the exception already logged somehwere else? We are just dropping it here.



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##########
@@ -1421,15 +1422,20 @@ public void 
shouldRemoveAllActiveTasksFromStateUpdaterOnPartitionLost() {
             .withInputPartitions(taskId02Partitions).build();
         final TasksRegistry tasks = mock(TasksRegistry.class);
         final TaskManager taskManager = setupForRevocationAndLost(mkSet(task1, 
task2, task3), tasks);
+        final CompletableFuture<StateUpdater.RemovedTaskResult> future1 = new 
CompletableFuture<>();

Review Comment:
   Do we need a test that covers the part where we get an exception?



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

Reply via email to