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


##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##########
@@ -1805,11 +1783,13 @@ public void 
shouldComputeOffsetSumForRunningStatefulTask() {
         final StreamTask runningStatefulTask = statefulTask(taskId00, 
taskId00ChangelogPartitions)
             .inState(State.RUNNING).build();
         final long changelogOffsetOfRunningTask = Task.LATEST_OFFSET;
-        when(runningStatefulTask.changelogOffsets())
-            .thenReturn(mkMap(mkEntry(t1p0changelog, 
changelogOffsetOfRunningTask)));
+        final Map<TopicPartition, Long> changelogOffsets = mkMap(
+            mkEntry(t1p0changelog, changelogOffsetOfRunningTask)
+        );
+        
when(runningStatefulTask.changelogOffsets()).thenReturn(changelogOffsets);
         final TasksRegistry tasks = mock(TasksRegistry.class);
-        final TaskManager taskManager = 
setUpTaskManagerWithoutStateUpdater(ProcessingMode.AT_LEAST_ONCE, tasks, false);
-        when(tasks.allTasksPerId()).thenReturn(mkMap(mkEntry(taskId00, 
runningStatefulTask)));
+        final TaskManager taskManager = 
setUpTaskManagerWithStateUpdater(ProcessingMode.AT_LEAST_ONCE, tasks);
+        when(stateUpdater.tasks()).thenReturn(Set.of(runningStatefulTask));

Review Comment:
   `RUNNING` tasks should not me owned by the state updater. I think mocking 
`tasks.allTasksPerId` makes more sense here



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##########
@@ -2124,47 +2096,37 @@ public void closeClean() {
 
     @Test
     public void shouldCloseActiveTasksWhenHandlingLostTasks() {
-        final StateMachineTask task00 = new StateMachineTask(taskId00, 
taskId00Partitions, true, stateManager);
-        final StateMachineTask task01 = new StateMachineTask(taskId01, 
taskId01Partitions, false, stateManager);
-
-        // `handleAssignment`
-        when(consumer.assignment()).thenReturn(assignment);
-        when(activeTaskCreator.createTasks(any(), 
eq(taskId00Assignment))).thenReturn(singletonList(task00));
-        
when(standbyTaskCreator.createTasks(taskId01Assignment)).thenReturn(singletonList(task01));
-
-        final ArrayList<TaskDirectory> taskFolders = new ArrayList<>(2);
-        taskFolders.add(new 
TaskDirectory(testFolder.resolve(taskId00.toString()).toFile(), null));
-        taskFolders.add(new 
TaskDirectory(testFolder.resolve(taskId01.toString()).toFile(), null));
-
-        when(stateDirectory.listNonEmptyTaskDirectories())
-            .thenReturn(taskFolders)
-            .thenReturn(new ArrayList<>());
+        final StreamTask task00 = statefulTask(taskId00, 
taskId00ChangelogPartitions)
+            .withInputPartitions(taskId00Partitions)
+            .inState(State.RUNNING)
+            .build();
 
-        expectLockObtainedFor(taskId00, taskId01);
-        expectDirectoryNotEmpty(taskId00, taskId01);
+        final StandbyTask task01 = standbyTask(taskId01, 
taskId01ChangelogPartitions)
+            .inState(State.RUNNING)
+            .build();
 
-        taskManager.handleRebalanceStart(emptySet());
-        assertThat(taskManager.lockedTaskDirectories(), 
Matchers.is(Set.of(taskId00, taskId01)));
+        final TasksRegistry tasks = mock(TasksRegistry.class);
+        when(tasks.allTasks()).thenReturn(Set.of(task00, task01));
+        when(tasks.allTaskIds()).thenReturn(Set.of(taskId00, taskId01));
 
-        taskManager.handleAssignment(taskId00Assignment, taskId01Assignment);
-        assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), 
null), is(true));
-        assertThat(task00.state(), is(Task.State.RUNNING));
-        assertThat(task01.state(), is(Task.State.RUNNING));
+        final TaskManager taskManager = 
setUpTaskManagerWithStateUpdater(ProcessingMode.AT_LEAST_ONCE, tasks);
 
-        // `handleLostAll`
+        // this should close only active tasks as zombies
         taskManager.handleLostAll();
-        assertThat(task00.commitPrepared, is(true));
-        assertThat(task00.state(), is(Task.State.CLOSED));
-        assertThat(task01.state(), is(Task.State.RUNNING));
-        assertThat(taskManager.activeTaskMap(), Matchers.anEmptyMap());
-        assertThat(taskManager.standbyTaskMap(), is(singletonMap(taskId01, 
task01)));
 
-        // The locked task map will not be cleared.

Review Comment:
   do we remove something here? It seems we tested `lockedTaskDirectories` 
before...



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##########
@@ -1843,11 +1835,11 @@ public void 
shouldComputeOffsetSumForRestoringActiveTaskWithStateUpdater() throw
         when(stateUpdater.tasks()).thenReturn(Set.of(restoringStatefulTask));
         taskManager.handleRebalanceStart(singleton("topic"));
 
-        assertThat(taskManager.taskOffsetSums(), is(mkMap(mkEntry(taskId00, 
changelogOffset))));
+        assertThat(taskManager.taskOffsetSums(), is(expectedOffsetSums));

Review Comment:
   What's the difference to the "nonRunning" test right above?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to