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


##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##########
@@ -4816,18 +4787,6 @@ public void 
shouldUseStartupTasksFromStateDirectoryAsStandbyWithStateUpdater() {
         assertEquals(Collections.singletonMap(taskId00, startupTask), 
taskManager.standbyTaskMap());
     }
 
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void shouldStartStateUpdaterOnInit(final boolean 
stateUpdaterEnabled) {
-        final TaskManager taskManager = 
setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, stateUpdaterEnabled);
-        taskManager.init();
-        if (stateUpdaterEnabled) {
-            verify(stateUpdater).start();

Review Comment:
   Why do we remove the sate updaer variant of this test?



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##########
@@ -2281,38 +2276,43 @@ public void suspend() {
 
     @Test
     public void shouldCommitNonCorruptedTasksOnTaskCorruptedException() {
-        final ProcessorStateManager stateManager = 
mock(ProcessorStateManager.class);
-
-        final StateMachineTask corruptedTask = new StateMachineTask(taskId00, 
taskId00Partitions, true, stateManager);
-        final StateMachineTask nonCorruptedTask = new 
StateMachineTask(taskId01, taskId01Partitions, true, stateManager);
+        final StreamTask corruptedTask = statefulTask(taskId00, 
taskId00ChangelogPartitions)
+            .withInputPartitions(taskId00Partitions)
+            .inState(State.RUNNING)
+            .build();
 
-        final Map<TaskId, Set<TopicPartition>> firstAssignment = new 
HashMap<>(taskId00Assignment);
-        firstAssignment.putAll(taskId01Assignment);
+        final StreamTask nonCorruptedTask = statefulTask(taskId01, 
taskId01ChangelogPartitions)
+            .withInputPartitions(taskId01Partitions)
+            .inState(State.RUNNING)
+            .build();
 
-        // `handleAssignment`
-        when(activeTaskCreator.createTasks(any(), eq(firstAssignment)))
-            .thenReturn(asList(corruptedTask, nonCorruptedTask));
+        final TasksRegistry tasks = mock(TasksRegistry.class);
+        when(tasks.task(taskId00)).thenReturn(corruptedTask);
+        when(tasks.allTasksPerId()).thenReturn(mkMap(
+            mkEntry(taskId00, corruptedTask),
+            mkEntry(taskId01, nonCorruptedTask)
+        ));
+        when(tasks.activeTaskIds()).thenReturn(Set.of(taskId00, taskId01));
 
-        when(consumer.assignment())
-            .thenReturn(assignment)
-            .thenReturn(taskId00Partitions);
+        when(nonCorruptedTask.commitNeeded()).thenReturn(true);
+        when(nonCorruptedTask.prepareCommit(true)).thenReturn(emptyMap());
+        when(corruptedTask.prepareCommit(false)).thenReturn(emptyMap());
+        doNothing().when(corruptedTask).postCommit(anyBoolean());
 
-        taskManager.handleAssignment(firstAssignment, emptyMap());
-        assertThat(taskManager.tryToCompleteRestoration(time.milliseconds(), 
tp -> assertThat(tp, is(empty()))), is(true));
+        when(consumer.assignment()).thenReturn(taskId00Partitions);
 
-        assertThat(nonCorruptedTask.state(), is(Task.State.RUNNING));
-        nonCorruptedTask.setCommitNeeded();
+        final TaskManager taskManager = 
setUpTaskManagerWithStateUpdater(ProcessingMode.AT_LEAST_ONCE, tasks);
 
-        corruptedTask.setChangelogOffsets(singletonMap(t1p0, 0L));

Review Comment:
   Why did you remove this?



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