cadonna commented on code in PR #13957:
URL: https://github.com/apache/kafka/pull/13957#discussion_r1253371567


##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskManagerTest.java:
##########
@@ -166,6 +168,44 @@ public void shouldNotAssignAnyLockedTask() {
         assertNull(taskManager.assignNextTask(taskExecutor));
     }
 
+    @Test
+    public void shouldNotSetUncaughtExceptionsForUnassignedTasks() {
+        taskManager.add(Collections.singleton(task));
+
+        assertThrows(IllegalArgumentException.class, () -> 
taskManager.setUncaughtException(exception, task.id()));

Review Comment:
   Could you please also verify the error message? Otherwise, we actually do 
not know which `IllegalArgumentException` was thrown.



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskManagerTest.java:
##########
@@ -166,6 +168,44 @@ public void shouldNotAssignAnyLockedTask() {
         assertNull(taskManager.assignNextTask(taskExecutor));
     }
 
+    @Test
+    public void shouldNotSetUncaughtExceptionsForUnassignedTasks() {
+        taskManager.add(Collections.singleton(task));
+
+        assertThrows(IllegalArgumentException.class, () -> 
taskManager.setUncaughtException(exception, task.id()));
+    }
+
+    @Test
+    public void shouldNotSetUncaughtExceptionsTwice() {
+        taskManager.add(Collections.singleton(task));
+        when(tasks.activeTasks()).thenReturn(Collections.singleton(task));
+        taskManager.assignNextTask(taskExecutor);
+        taskManager.setUncaughtException(exception, task.id());
+
+        assertThrows(IllegalArgumentException.class, () -> 
taskManager.setUncaughtException(exception, task.id()));

Review Comment:
   Could you please also verify the error message? Otherwise, we actually do 
not know which `IllegalArgumentException` was thrown.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/TaskManager.java:
##########
@@ -98,4 +100,25 @@ public interface TaskManager {
      * @return set of all managed active tasks
      */
     Set<ReadOnlyTask> getTasks();
+
+    /**
+     * Called whenever an existing task has thrown an uncaught exception.
+     *
+     * Setting an uncaught exception for a task prevents it from being 
reassigned until the
+     * corresponding exception has been handled in the polling thread.

Review Comment:
   That part is not yet implemented, right?



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskManagerTest.java:
##########
@@ -166,6 +168,44 @@ public void shouldNotAssignAnyLockedTask() {
         assertNull(taskManager.assignNextTask(taskExecutor));
     }
 
+    @Test
+    public void shouldNotSetUncaughtExceptionsForUnassignedTasks() {
+        taskManager.add(Collections.singleton(task));
+
+        assertThrows(IllegalArgumentException.class, () -> 
taskManager.setUncaughtException(exception, task.id()));
+    }
+
+    @Test
+    public void shouldNotSetUncaughtExceptionsTwice() {
+        taskManager.add(Collections.singleton(task));
+        when(tasks.activeTasks()).thenReturn(Collections.singleton(task));
+        taskManager.assignNextTask(taskExecutor);
+        taskManager.setUncaughtException(exception, task.id());
+
+        assertThrows(IllegalArgumentException.class, () -> 
taskManager.setUncaughtException(exception, task.id()));
+    }
+
+    @Test
+    public void shouldReturnExceptionsOnDrainExceptions() {
+        taskManager.add(Collections.singleton(task));
+        when(tasks.activeTasks()).thenReturn(Collections.singleton(task));
+        taskManager.assignNextTask(taskExecutor);
+        taskManager.setUncaughtException(exception, task.id());
+
+        assertEquals(taskManager.drainUncaughtExceptions(), 
Collections.singletonMap(task.id(), exception));
+    }
+
+    @Test
+    public void shouldClearExceptionsOnDrainExceptions() {
+        taskManager.add(Collections.singleton(task));
+        when(tasks.activeTasks()).thenReturn(Collections.singleton(task));
+        taskManager.assignNextTask(taskExecutor);
+        taskManager.setUncaughtException(exception, task.id());
+        taskManager.drainUncaughtExceptions();
+
+        assertEquals(taskManager.drainUncaughtExceptions(), 
Collections.emptyMap());
+    }

Review Comment:
   I think, it would be OK to merge these two tests. Just add the 
`assertEquals()` of the second test to the end of the first test.



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskExecutorTest.java:
##########
@@ -104,4 +105,19 @@ public void shouldUnassignTaskWhenRequired() throws 
Exception {
         assertTrue(future.isDone(), "Unassign is not completed");
         assertEquals(task, future.get(), "Unexpected task was unassigned");
     }
+
+    @Test
+    public void shouldSetUncaughtStreamsException() {
+        final StreamsException exception = mock(StreamsException.class);
+        when(task.process(anyLong())).thenThrow(exception);
+
+        taskExecutor.start();
+
+        verify(taskManager, 
timeout(VERIFICATION_TIMEOUT)).assignNextTask(taskExecutor);
+

Review Comment:
   nit:
   ```suggestion
   ```



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskManager.java:
##########
@@ -225,6 +227,37 @@ public Set<ReadOnlyTask> getTasks() {
         return returnWithTasksLocked(() -> 
tasks.activeTasks().stream().map(ReadOnlyTask::new).collect(Collectors.toSet()));
     }
 
+    @Override
+    public void setUncaughtException(final StreamsException exception, final 
TaskId taskId) {
+        executeWithTasksLocked(() -> {
+
+            if (!assignedTasks.containsKey(taskId)) {
+                throw new IllegalArgumentException("An uncaught exception can 
only be set as long as the task is still assigned");
+            }
+
+            if (uncaughtExceptions.containsKey(taskId)) {
+                throw new IllegalArgumentException("The uncaught exception 
must be cleared before restarting processing");
+            }
+
+            uncaughtExceptions.put(taskId, exception);
+        });
+
+        log.info("Set an uncaught exception for task {}", taskId);

Review Comment:
   Should we log some more information about the exception, like the type and 
the error message? Just to be able to better reason about the error case.



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