mirror of https://github.com/apache/kafka.git
KAFKA-16077: Streams with state updater fails to close task upon fencing (#15117)
* KAFKA-16077: Streams fails to close task after restoration when input partitions are updated There is a race condition in the state updater that can cause the following: 1. We have an active task in the state updater 2. We get fenced. We recreate the producer, transactions now uninitialized. We ask the state updater to give back the task, add a pending action to close the task clean once it’s handed back 3. We get a new assignment with updated input partitions. The task is still owned by the state updater, so we ask the state updater again to hand it back and add a pending action to update its input partition 4. The task is handed back by the state updater. We update its input partitions but forget to close it clean (pending action was overwritten) 5. Now the task is in an initialized state, but the underlying producer does not have transactions initialized This can cause an IllegalStateException: `Invalid transition attempted from state UNINITIALIZED to state IN_TRANSACTION` when running in EOSv2. To fix this, we introduce a new pending action CloseReviveAndUpdateInputPartitions that is added when we handle a new assignment with updated input partitions, but we still need to close the task before reopening it. We should not remove the task twice, otherwise, we'll end up in this situation 1. We have an active task in the state updater 2. We get fenced. We recreate the producer, transactions now uninitialized. We ask the state updater to give back the task, add a pending action to close the task clean once it’s handed back 3. The state updater moves the task from the updating tasks to the removed tasks 4. We get a new assignment with updated input partitions. The task is still owned by the state updater, so we ask the state updater again to hand it back (adding a task+remove into the task and action queue) and add a pending action to close, revive and update input partitions 5. The task is handed back by the state updater. We close revive and update input partitions, and add the task back to the state updater 6. The state updater executes the "task+remove" action that is still in its task + action queue, and hands the task immediately back to the main thread 7. The main thread discoveres a removed task that was not restored and has no pending action attached to it. IllegalStateException Reviewers: Bruno Cadonna <cadonna@apache.org>
This commit is contained in:
parent
599e22b842
commit
c0b6493455
|
@ -25,6 +25,7 @@ public class PendingUpdateAction {
|
||||||
|
|
||||||
enum Action {
|
enum Action {
|
||||||
UPDATE_INPUT_PARTITIONS,
|
UPDATE_INPUT_PARTITIONS,
|
||||||
|
CLOSE_REVIVE_AND_UPDATE_INPUT_PARTITIONS,
|
||||||
RECYCLE,
|
RECYCLE,
|
||||||
SUSPEND,
|
SUSPEND,
|
||||||
ADD_BACK,
|
ADD_BACK,
|
||||||
|
@ -48,6 +49,11 @@ public class PendingUpdateAction {
|
||||||
return new PendingUpdateAction(Action.UPDATE_INPUT_PARTITIONS, inputPartitions);
|
return new PendingUpdateAction(Action.UPDATE_INPUT_PARTITIONS, inputPartitions);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static PendingUpdateAction createCloseReviveAndUpdateInputPartition(final Set<TopicPartition> inputPartitions) {
|
||||||
|
Objects.requireNonNull(inputPartitions, "Set of input partitions to update is null!");
|
||||||
|
return new PendingUpdateAction(Action.CLOSE_REVIVE_AND_UPDATE_INPUT_PARTITIONS, inputPartitions);
|
||||||
|
}
|
||||||
|
|
||||||
public static PendingUpdateAction createRecycleTask(final Set<TopicPartition> inputPartitions) {
|
public static PendingUpdateAction createRecycleTask(final Set<TopicPartition> inputPartitions) {
|
||||||
Objects.requireNonNull(inputPartitions, "Set of input partitions to update is null!");
|
Objects.requireNonNull(inputPartitions, "Set of input partitions to update is null!");
|
||||||
return new PendingUpdateAction(Action.RECYCLE, inputPartitions);
|
return new PendingUpdateAction(Action.RECYCLE, inputPartitions);
|
||||||
|
@ -66,7 +72,7 @@ public class PendingUpdateAction {
|
||||||
}
|
}
|
||||||
|
|
||||||
public Set<TopicPartition> getInputPartitions() {
|
public Set<TopicPartition> getInputPartitions() {
|
||||||
if (action != Action.UPDATE_INPUT_PARTITIONS && action != Action.RECYCLE) {
|
if (action != Action.UPDATE_INPUT_PARTITIONS && action != Action.CLOSE_REVIVE_AND_UPDATE_INPUT_PARTITIONS && action != Action.RECYCLE) {
|
||||||
throw new IllegalStateException("Action type " + action + " does not have a set of input partitions!");
|
throw new IllegalStateException("Action type " + action + " does not have a set of input partitions!");
|
||||||
}
|
}
|
||||||
return inputPartitions;
|
return inputPartitions;
|
||||||
|
|
|
@ -543,8 +543,12 @@ public class TaskManager {
|
||||||
if (activeTasksToCreate.containsKey(taskId)) {
|
if (activeTasksToCreate.containsKey(taskId)) {
|
||||||
final Set<TopicPartition> inputPartitions = activeTasksToCreate.get(taskId);
|
final Set<TopicPartition> inputPartitions = activeTasksToCreate.get(taskId);
|
||||||
if (task.isActive() && !task.inputPartitions().equals(inputPartitions)) {
|
if (task.isActive() && !task.inputPartitions().equals(inputPartitions)) {
|
||||||
stateUpdater.remove(taskId);
|
if (tasks.removePendingTaskToCloseClean(taskId)) {
|
||||||
tasks.addPendingTaskToUpdateInputPartitions(taskId, inputPartitions);
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(taskId, inputPartitions);
|
||||||
|
} else {
|
||||||
|
tasks.addPendingTaskToUpdateInputPartitions(taskId, inputPartitions);
|
||||||
|
stateUpdater.remove(taskId);
|
||||||
|
}
|
||||||
} else if (task.isActive()) {
|
} else if (task.isActive()) {
|
||||||
tasks.removePendingActiveTaskToSuspend(taskId);
|
tasks.removePendingActiveTaskToSuspend(taskId);
|
||||||
if (tasks.removePendingTaskToCloseClean(taskId)) {
|
if (tasks.removePendingTaskToCloseClean(taskId)) {
|
||||||
|
@ -819,15 +823,17 @@ public class TaskManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void closeTaskClean(final Task task,
|
/** Returns true if the task closed clean */
|
||||||
final Set<Task> tasksToCloseDirty,
|
private boolean closeTaskClean(final Task task,
|
||||||
final Map<TaskId, RuntimeException> taskExceptions) {
|
final Set<Task> tasksToCloseDirty,
|
||||||
|
final Map<TaskId, RuntimeException> taskExceptions) {
|
||||||
try {
|
try {
|
||||||
task.suspend();
|
task.suspend();
|
||||||
task.closeClean();
|
task.closeClean();
|
||||||
if (task.isActive()) {
|
if (task.isActive()) {
|
||||||
activeTaskCreator.closeAndRemoveTaskProducerIfNeeded(task.id());
|
activeTaskCreator.closeAndRemoveTaskProducerIfNeeded(task.id());
|
||||||
}
|
}
|
||||||
|
return true;
|
||||||
} catch (final RuntimeException e) {
|
} catch (final RuntimeException e) {
|
||||||
final String uncleanMessage = String.format("Failed to close task %s cleanly. " +
|
final String uncleanMessage = String.format("Failed to close task %s cleanly. " +
|
||||||
"Attempting to close remaining tasks before re-throwing:", task.id());
|
"Attempting to close remaining tasks before re-throwing:", task.id());
|
||||||
|
@ -838,6 +844,7 @@ public class TaskManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
taskExceptions.putIfAbsent(task.id(), e);
|
taskExceptions.putIfAbsent(task.id(), e);
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -916,6 +923,12 @@ public class TaskManager {
|
||||||
stateUpdater.add(task);
|
stateUpdater.add(task);
|
||||||
} else if (tasks.removePendingTaskToCloseClean(task.id())) {
|
} else if (tasks.removePendingTaskToCloseClean(task.id())) {
|
||||||
closeTaskClean(task, tasksToCloseDirty, taskExceptions);
|
closeTaskClean(task, tasksToCloseDirty, taskExceptions);
|
||||||
|
} else if ((inputPartitions = tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(task.id())) != null) {
|
||||||
|
if (closeTaskClean(task, tasksToCloseDirty, taskExceptions)) {
|
||||||
|
task.revive();
|
||||||
|
task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
|
||||||
|
addTaskToStateUpdater(task);
|
||||||
|
}
|
||||||
} else if ((inputPartitions = tasks.removePendingTaskToUpdateInputPartitions(task.id())) != null) {
|
} else if ((inputPartitions = tasks.removePendingTaskToUpdateInputPartitions(task.id())) != null) {
|
||||||
task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
|
task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
|
||||||
stateUpdater.add(task);
|
stateUpdater.add(task);
|
||||||
|
@ -950,6 +963,12 @@ public class TaskManager {
|
||||||
closeTaskClean(task, tasksToCloseDirty, taskExceptions);
|
closeTaskClean(task, tasksToCloseDirty, taskExceptions);
|
||||||
} else if (tasks.removePendingTaskToAddBack(task.id())) {
|
} else if (tasks.removePendingTaskToAddBack(task.id())) {
|
||||||
stateUpdater.add(task);
|
stateUpdater.add(task);
|
||||||
|
} else if ((inputPartitions = tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(task.id())) != null) {
|
||||||
|
if (closeTaskClean(task, tasksToCloseDirty, taskExceptions)) {
|
||||||
|
task.revive();
|
||||||
|
task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
|
||||||
|
addTaskToStateUpdater(task);
|
||||||
|
}
|
||||||
} else if ((inputPartitions = tasks.removePendingTaskToUpdateInputPartitions(task.id())) != null) {
|
} else if ((inputPartitions = tasks.removePendingTaskToUpdateInputPartitions(task.id())) != null) {
|
||||||
task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
|
task.updateInputPartitions(inputPartitions, topologyMetadata.nodeToSourceTopics(task.id()));
|
||||||
transitRestoredTaskToRunning(task, now, offsetResetter);
|
transitRestoredTaskToRunning(task, now, offsetResetter);
|
||||||
|
@ -1306,7 +1325,7 @@ public class TaskManager {
|
||||||
// before suspending and closing the topology
|
// before suspending and closing the topology
|
||||||
task.prepareCommit();
|
task.prepareCommit();
|
||||||
} catch (final RuntimeException swallow) {
|
} catch (final RuntimeException swallow) {
|
||||||
log.error("Error flushing caches of dirty task {} ", task.id(), swallow);
|
log.error("Error flushing caches of dirty task {}", task.id(), swallow);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -120,6 +120,19 @@ class Tasks implements TasksRegistry {
|
||||||
return pendingUpdateActions.values().stream().anyMatch(action -> action.getAction() == Action.RECYCLE);
|
return pendingUpdateActions.values().stream().anyMatch(action -> action.getAction() == Action.RECYCLE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<TopicPartition> removePendingTaskToCloseReviveAndUpdateInputPartitions(final TaskId taskId) {
|
||||||
|
if (containsTaskIdWithAction(taskId, Action.CLOSE_REVIVE_AND_UPDATE_INPUT_PARTITIONS)) {
|
||||||
|
return pendingUpdateActions.remove(taskId).getInputPartitions();
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void addPendingTaskToCloseReviveAndUpdateInputPartitions(final TaskId taskId, final Set<TopicPartition> inputPartitions) {
|
||||||
|
pendingUpdateActions.put(taskId, PendingUpdateAction.createCloseReviveAndUpdateInputPartition(inputPartitions));
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<TopicPartition> removePendingTaskToUpdateInputPartitions(final TaskId taskId) {
|
public Set<TopicPartition> removePendingTaskToUpdateInputPartitions(final TaskId taskId) {
|
||||||
if (containsTaskIdWithAction(taskId, Action.UPDATE_INPUT_PARTITIONS)) {
|
if (containsTaskIdWithAction(taskId, Action.UPDATE_INPUT_PARTITIONS)) {
|
||||||
|
|
|
@ -41,6 +41,10 @@ public interface TasksRegistry {
|
||||||
|
|
||||||
void addPendingTaskToRecycle(final TaskId taskId, final Set<TopicPartition> inputPartitions);
|
void addPendingTaskToRecycle(final TaskId taskId, final Set<TopicPartition> inputPartitions);
|
||||||
|
|
||||||
|
Set<TopicPartition> removePendingTaskToCloseReviveAndUpdateInputPartitions(final TaskId taskId);
|
||||||
|
|
||||||
|
void addPendingTaskToCloseReviveAndUpdateInputPartitions(final TaskId taskId, final Set<TopicPartition> inputPartitions);
|
||||||
|
|
||||||
Set<TopicPartition> removePendingTaskToUpdateInputPartitions(final TaskId taskId);
|
Set<TopicPartition> removePendingTaskToUpdateInputPartitions(final TaskId taskId);
|
||||||
|
|
||||||
void addPendingTaskToUpdateInputPartitions(final TaskId taskId, final Set<TopicPartition> inputPartitions);
|
void addPendingTaskToUpdateInputPartitions(final TaskId taskId, final Set<TopicPartition> inputPartitions);
|
||||||
|
|
|
@ -107,6 +107,7 @@ import static org.apache.kafka.common.utils.Utils.union;
|
||||||
import static org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
|
import static org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
|
||||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.standbyTask;
|
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.standbyTask;
|
||||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statefulTask;
|
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statefulTask;
|
||||||
|
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statelessTask;
|
||||||
import static org.easymock.EasyMock.anyObject;
|
import static org.easymock.EasyMock.anyObject;
|
||||||
import static org.easymock.EasyMock.eq;
|
import static org.easymock.EasyMock.eq;
|
||||||
import static org.easymock.EasyMock.expect;
|
import static org.easymock.EasyMock.expect;
|
||||||
|
@ -529,6 +530,29 @@ public class TaskManagerTest {
|
||||||
Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
|
Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldCloseReviveAndUpdateInputPartitionOfActiveTaskInStateUpdater() {
|
||||||
|
final StreamTask activeTaskToUpdateInputPartitions = statefulTask(taskId03, taskId03ChangelogPartitions)
|
||||||
|
.inState(State.RESTORING)
|
||||||
|
.withInputPartitions(taskId03Partitions).build();
|
||||||
|
final Set<TopicPartition> newInputPartitions = taskId02Partitions;
|
||||||
|
final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
|
||||||
|
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
|
when(stateUpdater.getTasks()).thenReturn(mkSet(activeTaskToUpdateInputPartitions));
|
||||||
|
when(tasks.removePendingTaskToCloseClean(activeTaskToUpdateInputPartitions.id())).thenReturn(true);
|
||||||
|
|
||||||
|
taskManager.handleAssignment(
|
||||||
|
mkMap(mkEntry(activeTaskToUpdateInputPartitions.id(), newInputPartitions)),
|
||||||
|
Collections.emptyMap()
|
||||||
|
);
|
||||||
|
|
||||||
|
Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
|
||||||
|
Mockito.verify(stateUpdater, never()).remove(activeTaskToUpdateInputPartitions.id());
|
||||||
|
Mockito.verify(tasks).removePendingTaskToCloseClean(activeTaskToUpdateInputPartitions.id());
|
||||||
|
Mockito.verify(tasks).addPendingTaskToCloseReviveAndUpdateInputPartitions(activeTaskToUpdateInputPartitions.id(), newInputPartitions);
|
||||||
|
Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldKeepReassignedActiveTaskInStateUpdater() {
|
public void shouldKeepReassignedActiveTaskInStateUpdater() {
|
||||||
final StreamTask reassignedActiveTask = statefulTask(taskId03, taskId03ChangelogPartitions)
|
final StreamTask reassignedActiveTask = statefulTask(taskId03, taskId03ChangelogPartitions)
|
||||||
|
@ -629,6 +653,29 @@ public class TaskManagerTest {
|
||||||
Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
|
Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldNeverCloseReviveAndUpdateInputPartitionsOfStandbyTaskInStateUpdater() {
|
||||||
|
final StandbyTask standbyTaskToUpdateInputPartitions = standbyTask(taskId02, taskId02ChangelogPartitions)
|
||||||
|
.inState(State.RUNNING)
|
||||||
|
.withInputPartitions(taskId02Partitions).build();
|
||||||
|
final Set<TopicPartition> newInputPartitions = taskId03Partitions;
|
||||||
|
final TasksRegistry tasks = Mockito.mock(TasksRegistry.class);
|
||||||
|
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
|
when(stateUpdater.getTasks()).thenReturn(mkSet(standbyTaskToUpdateInputPartitions));
|
||||||
|
|
||||||
|
taskManager.handleAssignment(
|
||||||
|
Collections.emptyMap(),
|
||||||
|
mkMap(mkEntry(standbyTaskToUpdateInputPartitions.id(), newInputPartitions))
|
||||||
|
);
|
||||||
|
|
||||||
|
Mockito.verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap());
|
||||||
|
Mockito.verify(stateUpdater, never()).remove(standbyTaskToUpdateInputPartitions.id());
|
||||||
|
Mockito.verify(tasks, never()).removePendingTaskToCloseClean(standbyTaskToUpdateInputPartitions.id());
|
||||||
|
Mockito.verify(tasks, never())
|
||||||
|
.addPendingTaskToCloseReviveAndUpdateInputPartitions(standbyTaskToUpdateInputPartitions.id(), newInputPartitions);
|
||||||
|
Mockito.verify(standbyTaskCreator).createTasks(Collections.emptyMap());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldKeepReassignedStandbyTaskInStateUpdater() {
|
public void shouldKeepReassignedStandbyTaskInStateUpdater() {
|
||||||
final StandbyTask reassignedStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
|
final StandbyTask reassignedStandbyTask = standbyTask(taskId02, taskId02ChangelogPartitions)
|
||||||
|
@ -1101,30 +1148,53 @@ public class TaskManagerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldUpdateInputPartitionsOfTasksRemovedFromStateUpdater() {
|
public void shouldUpdateInputPartitionsOfTasksRemovedFromStateUpdater() {
|
||||||
final StreamTask task00 = statefulTask(taskId00, taskId00ChangelogPartitions)
|
final StreamTask activeTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||||
.withInputPartitions(taskId00Partitions)
|
.withInputPartitions(taskId00Partitions)
|
||||||
.inState(State.RESTORING).build();
|
.inState(State.RESTORING).build();
|
||||||
final StandbyTask task01 = standbyTask(taskId01, taskId01ChangelogPartitions)
|
final StandbyTask standbyTask = standbyTask(taskId01, taskId01ChangelogPartitions)
|
||||||
.withInputPartitions(taskId01Partitions)
|
.withInputPartitions(taskId01Partitions)
|
||||||
.inState(State.RUNNING).build();
|
.inState(State.RUNNING).build();
|
||||||
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
||||||
when(stateUpdater.drainRemovedTasks()).thenReturn(mkSet(task00, task01));
|
when(stateUpdater.drainRemovedTasks()).thenReturn(mkSet(activeTask, standbyTask));
|
||||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
when(tasks.removePendingTaskToRecycle(any())).thenReturn(null);
|
when(tasks.removePendingTaskToRecycle(any())).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(task00.id())).thenReturn(taskId02Partitions);
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(any())).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(task01.id())).thenReturn(taskId03Partitions);
|
when(tasks.removePendingTaskToUpdateInputPartitions(activeTask.id())).thenReturn(taskId02Partitions);
|
||||||
|
when(tasks.removePendingTaskToUpdateInputPartitions(standbyTask.id())).thenReturn(taskId03Partitions);
|
||||||
taskManager = setUpTaskManager(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
taskManager = setUpTaskManager(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
|
|
||||||
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
|
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
|
||||||
|
|
||||||
Mockito.verify(task00).updateInputPartitions(Mockito.eq(taskId02Partitions), anyMap());
|
Mockito.verify(activeTask).updateInputPartitions(Mockito.eq(taskId02Partitions), anyMap());
|
||||||
Mockito.verify(task00, never()).closeDirty();
|
Mockito.verify(activeTask, never()).closeDirty();
|
||||||
Mockito.verify(task00, never()).closeClean();
|
Mockito.verify(activeTask, never()).closeClean();
|
||||||
Mockito.verify(stateUpdater).add(task00);
|
Mockito.verify(stateUpdater).add(activeTask);
|
||||||
Mockito.verify(task01).updateInputPartitions(Mockito.eq(taskId03Partitions), anyMap());
|
Mockito.verify(standbyTask).updateInputPartitions(Mockito.eq(taskId03Partitions), anyMap());
|
||||||
Mockito.verify(task01, never()).closeDirty();
|
Mockito.verify(standbyTask, never()).closeDirty();
|
||||||
Mockito.verify(task01, never()).closeClean();
|
Mockito.verify(standbyTask, never()).closeClean();
|
||||||
Mockito.verify(stateUpdater).add(task01);
|
Mockito.verify(stateUpdater).add(standbyTask);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldCloseReviveAndUpdateInputPartitionsOfTasksRemovedFromStateUpdater() {
|
||||||
|
final StreamTask activeTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||||
|
.withInputPartitions(taskId00Partitions)
|
||||||
|
.inState(State.RESTORING).build();
|
||||||
|
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
||||||
|
when(stateUpdater.drainRemovedTasks()).thenReturn(mkSet(activeTask));
|
||||||
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
|
when(tasks.removePendingTaskToRecycle(any())).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(activeTask.id())).thenReturn(taskId02Partitions);
|
||||||
|
taskManager = setUpTaskManager(StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
|
|
||||||
|
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
|
||||||
|
|
||||||
|
Mockito.verify(activeTask).closeClean();
|
||||||
|
Mockito.verify(activeTask).revive();
|
||||||
|
Mockito.verify(activeTask).updateInputPartitions(Mockito.eq(taskId02Partitions), anyMap());
|
||||||
|
Mockito.verify(activeTask).initializeIfNeeded();
|
||||||
|
Mockito.verify(activeTask, never()).closeDirty();
|
||||||
|
Mockito.verify(stateUpdater).add(activeTask);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1135,6 +1205,7 @@ public class TaskManagerTest {
|
||||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
when(tasks.removePendingActiveTaskToSuspend(statefulTask.id())).thenReturn(true);
|
when(tasks.removePendingActiveTaskToSuspend(statefulTask.id())).thenReturn(true);
|
||||||
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
||||||
when(stateUpdater.drainRemovedTasks()).thenReturn(mkSet(statefulTask));
|
when(stateUpdater.drainRemovedTasks()).thenReturn(mkSet(statefulTask));
|
||||||
|
@ -1164,9 +1235,12 @@ public class TaskManagerTest {
|
||||||
final StreamTask taskToUpdateInputPartitions = statefulTask(taskId03, taskId03ChangelogPartitions)
|
final StreamTask taskToUpdateInputPartitions = statefulTask(taskId03, taskId03ChangelogPartitions)
|
||||||
.inState(State.RESTORING)
|
.inState(State.RESTORING)
|
||||||
.withInputPartitions(taskId03Partitions).build();
|
.withInputPartitions(taskId03Partitions).build();
|
||||||
|
final StreamTask taskToCloseReviveAndUpdateInputPartitions = statefulTask(taskId04, taskId04ChangelogPartitions)
|
||||||
|
.inState(State.RESTORING)
|
||||||
|
.withInputPartitions(taskId04Partitions).build();
|
||||||
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
when(stateUpdater.hasRemovedTasks()).thenReturn(true);
|
||||||
when(stateUpdater.drainRemovedTasks())
|
when(stateUpdater.drainRemovedTasks())
|
||||||
.thenReturn(mkSet(taskToRecycle0, taskToRecycle1, taskToClose, taskToUpdateInputPartitions));
|
.thenReturn(mkSet(taskToRecycle0, taskToRecycle1, taskToClose, taskToUpdateInputPartitions, taskToCloseReviveAndUpdateInputPartitions));
|
||||||
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
||||||
when(activeTaskCreator.createActiveTaskFromStandby(taskToRecycle1, taskId01Partitions, consumer))
|
when(activeTaskCreator.createActiveTaskFromStandby(taskToRecycle1, taskId01Partitions, consumer))
|
||||||
.thenReturn(convertedTask1);
|
.thenReturn(convertedTask1);
|
||||||
|
@ -1184,6 +1258,10 @@ public class TaskManagerTest {
|
||||||
argThat(taskId -> !taskId.equals(taskToRecycle0.id()) && !taskId.equals(taskToRecycle1.id())))
|
argThat(taskId -> !taskId.equals(taskToRecycle0.id()) && !taskId.equals(taskToRecycle1.id())))
|
||||||
).thenReturn(null);
|
).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(taskToUpdateInputPartitions.id())).thenReturn(taskId04Partitions);
|
when(tasks.removePendingTaskToUpdateInputPartitions(taskToUpdateInputPartitions.id())).thenReturn(taskId04Partitions);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(taskToCloseReviveAndUpdateInputPartitions.id())).thenReturn(taskId05Partitions);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(
|
||||||
|
argThat(taskId -> !taskId.equals(taskToCloseReviveAndUpdateInputPartitions.id()))
|
||||||
|
)).thenReturn(null);
|
||||||
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
taskManager.setMainConsumer(consumer);
|
taskManager.setMainConsumer(consumer);
|
||||||
replay(consumer);
|
replay(consumer);
|
||||||
|
@ -1191,7 +1269,7 @@ public class TaskManagerTest {
|
||||||
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter -> { });
|
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter -> { });
|
||||||
|
|
||||||
verify(consumer);
|
verify(consumer);
|
||||||
Mockito.verify(activeTaskCreator, times(2)).closeAndRemoveTaskProducerIfNeeded(any());
|
Mockito.verify(activeTaskCreator, times(3)).closeAndRemoveTaskProducerIfNeeded(any());
|
||||||
Mockito.verify(convertedTask0).initializeIfNeeded();
|
Mockito.verify(convertedTask0).initializeIfNeeded();
|
||||||
Mockito.verify(convertedTask1).initializeIfNeeded();
|
Mockito.verify(convertedTask1).initializeIfNeeded();
|
||||||
Mockito.verify(stateUpdater).add(convertedTask0);
|
Mockito.verify(stateUpdater).add(convertedTask0);
|
||||||
|
@ -1199,6 +1277,11 @@ public class TaskManagerTest {
|
||||||
Mockito.verify(taskToClose).closeClean();
|
Mockito.verify(taskToClose).closeClean();
|
||||||
Mockito.verify(taskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId04Partitions), anyMap());
|
Mockito.verify(taskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId04Partitions), anyMap());
|
||||||
Mockito.verify(stateUpdater).add(taskToUpdateInputPartitions);
|
Mockito.verify(stateUpdater).add(taskToUpdateInputPartitions);
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).closeClean();
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).revive();
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId05Partitions), anyMap());
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).initializeIfNeeded();
|
||||||
|
Mockito.verify(stateUpdater).add(taskToCloseReviveAndUpdateInputPartitions);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1368,6 +1451,7 @@ public class TaskManagerTest {
|
||||||
final TasksRegistry tasks) {
|
final TasksRegistry tasks) {
|
||||||
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
||||||
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
||||||
|
|
||||||
|
@ -1545,6 +1629,7 @@ public class TaskManagerTest {
|
||||||
.withInputPartitions(taskId00Partitions).build();
|
.withInputPartitions(taskId00Partitions).build();
|
||||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(taskId01Partitions);
|
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(taskId01Partitions);
|
||||||
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
||||||
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
||||||
|
@ -1561,6 +1646,27 @@ public class TaskManagerTest {
|
||||||
Mockito.verify(tasks).addTask(statefulTask);
|
Mockito.verify(tasks).addTask(statefulTask);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldCloseReviveAndUpdateInputPartitionsOfRestoredTask() {
|
||||||
|
final StreamTask statefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||||
|
.inState(State.RESTORING)
|
||||||
|
.withInputPartitions(taskId00Partitions).build();
|
||||||
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
|
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(statefulTask.id())).thenReturn(taskId01Partitions);
|
||||||
|
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
||||||
|
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
||||||
|
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
|
|
||||||
|
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
|
||||||
|
|
||||||
|
Mockito.verify(statefulTask).updateInputPartitions(Mockito.eq(taskId01Partitions), anyMap());
|
||||||
|
Mockito.verify(statefulTask).closeClean();
|
||||||
|
Mockito.verify(statefulTask).revive();
|
||||||
|
Mockito.verify(statefulTask).initializeIfNeeded();
|
||||||
|
Mockito.verify(stateUpdater).add(statefulTask);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldSuspendRestoredTaskIfRevoked() {
|
public void shouldSuspendRestoredTaskIfRevoked() {
|
||||||
final StreamTask statefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
final StreamTask statefulTask = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||||
|
@ -1569,6 +1675,7 @@ public class TaskManagerTest {
|
||||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToRecycle(statefulTask.id())).thenReturn(null);
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
when(tasks.removePendingTaskToUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(statefulTask.id())).thenReturn(null);
|
||||||
when(tasks.removePendingActiveTaskToSuspend(statefulTask.id())).thenReturn(true);
|
when(tasks.removePendingActiveTaskToSuspend(statefulTask.id())).thenReturn(true);
|
||||||
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(statefulTask));
|
||||||
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
||||||
|
@ -1602,6 +1709,9 @@ public class TaskManagerTest {
|
||||||
final StreamTask taskToUpdateInputPartitions = statefulTask(taskId04, taskId04ChangelogPartitions)
|
final StreamTask taskToUpdateInputPartitions = statefulTask(taskId04, taskId04ChangelogPartitions)
|
||||||
.inState(State.RESTORING)
|
.inState(State.RESTORING)
|
||||||
.withInputPartitions(taskId04Partitions).build();
|
.withInputPartitions(taskId04Partitions).build();
|
||||||
|
final StreamTask taskToCloseReviveAndUpdateInputPartitions = statelessTask(taskId05)
|
||||||
|
.inState(State.RESTORING)
|
||||||
|
.withInputPartitions(taskId05Partitions).build();
|
||||||
final TasksRegistry tasks = mock(TasksRegistry.class);
|
final TasksRegistry tasks = mock(TasksRegistry.class);
|
||||||
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true);
|
||||||
when(standbyTaskCreator.createStandbyTaskFromActive(taskToRecycle, taskToRecycle.inputPartitions()))
|
when(standbyTaskCreator.createStandbyTaskFromActive(taskToRecycle, taskToRecycle.inputPartitions()))
|
||||||
|
@ -1622,13 +1732,18 @@ public class TaskManagerTest {
|
||||||
when(tasks.removePendingTaskToUpdateInputPartitions(
|
when(tasks.removePendingTaskToUpdateInputPartitions(
|
||||||
argThat(taskId -> !taskId.equals(taskToUpdateInputPartitions.id())))
|
argThat(taskId -> !taskId.equals(taskToUpdateInputPartitions.id())))
|
||||||
).thenReturn(null);
|
).thenReturn(null);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(taskToCloseReviveAndUpdateInputPartitions.id())).thenReturn(taskId04Partitions);
|
||||||
|
when(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(
|
||||||
|
argThat(taskId -> !taskId.equals(taskToCloseReviveAndUpdateInputPartitions.id())))
|
||||||
|
).thenReturn(null);
|
||||||
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
when(stateUpdater.restoresActiveTasks()).thenReturn(true);
|
||||||
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(
|
when(stateUpdater.drainRestoredActiveTasks(any(Duration.class))).thenReturn(mkSet(
|
||||||
taskToTransitToRunning,
|
taskToTransitToRunning,
|
||||||
taskToRecycle,
|
taskToRecycle,
|
||||||
taskToCloseClean,
|
taskToCloseClean,
|
||||||
taskToAddBack,
|
taskToAddBack,
|
||||||
taskToUpdateInputPartitions
|
taskToUpdateInputPartitions,
|
||||||
|
taskToCloseReviveAndUpdateInputPartitions
|
||||||
));
|
));
|
||||||
|
|
||||||
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
|
taskManager.checkStateUpdater(time.milliseconds(), noOpResetter);
|
||||||
|
@ -1639,6 +1754,10 @@ public class TaskManagerTest {
|
||||||
Mockito.verify(taskToCloseClean).closeClean();
|
Mockito.verify(taskToCloseClean).closeClean();
|
||||||
Mockito.verify(stateUpdater).add(taskToAddBack);
|
Mockito.verify(stateUpdater).add(taskToAddBack);
|
||||||
Mockito.verify(taskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId05Partitions), anyMap());
|
Mockito.verify(taskToUpdateInputPartitions).updateInputPartitions(Mockito.eq(taskId05Partitions), anyMap());
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).closeClean();
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).revive();
|
||||||
|
Mockito.verify(taskToCloseReviveAndUpdateInputPartitions).initializeIfNeeded();
|
||||||
|
Mockito.verify(stateUpdater).add(taskToCloseReviveAndUpdateInputPartitions);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -1726,11 +1845,11 @@ public class TaskManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void shouldRethrowTaskCorruptedExceptionFromInitialization() {
|
public void shouldRethrowTaskCorruptedExceptionFromInitialization() {
|
||||||
final StreamTask statefulTask0 = statefulTask(taskId00, taskId00ChangelogPartitions)
|
final StreamTask statefulTask0 = statefulTask(taskId00, taskId00ChangelogPartitions)
|
||||||
.inState(State.CREATED)
|
.inState(State.CREATED)
|
||||||
.withInputPartitions(taskId00Partitions).build();
|
.withInputPartitions(taskId00Partitions).build();
|
||||||
final StreamTask statefulTask1 = statefulTask(taskId01, taskId01ChangelogPartitions)
|
final StreamTask statefulTask1 = statefulTask(taskId01, taskId01ChangelogPartitions)
|
||||||
.inState(State.CREATED)
|
.inState(State.CREATED)
|
||||||
.withInputPartitions(taskId01Partitions).build();
|
.withInputPartitions(taskId01Partitions).build();
|
||||||
final StreamTask statefulTask2 = statefulTask(taskId02, taskId02ChangelogPartitions)
|
final StreamTask statefulTask2 = statefulTask(taskId02, taskId02ChangelogPartitions)
|
||||||
.inState(State.CREATED)
|
.inState(State.CREATED)
|
||||||
.withInputPartitions(taskId02Partitions).build();
|
.withInputPartitions(taskId02Partitions).build();
|
||||||
|
@ -1741,8 +1860,8 @@ public class TaskManagerTest {
|
||||||
doThrow(new TaskCorruptedException(Collections.singleton(statefulTask1.id))).when(statefulTask1).initializeIfNeeded();
|
doThrow(new TaskCorruptedException(Collections.singleton(statefulTask1.id))).when(statefulTask1).initializeIfNeeded();
|
||||||
|
|
||||||
final TaskCorruptedException thrown = assertThrows(
|
final TaskCorruptedException thrown = assertThrows(
|
||||||
TaskCorruptedException.class,
|
TaskCorruptedException.class,
|
||||||
() -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
|
() -> taskManager.checkStateUpdater(time.milliseconds(), noOpResetter)
|
||||||
);
|
);
|
||||||
|
|
||||||
Mockito.verify(tasks).addTask(statefulTask0);
|
Mockito.verify(tasks).addTask(statefulTask0);
|
||||||
|
|
|
@ -154,6 +154,7 @@ public class TasksTest {
|
||||||
tasks.addPendingTaskToCloseClean(TASK_0_1);
|
tasks.addPendingTaskToCloseClean(TASK_0_1);
|
||||||
tasks.addPendingTaskToAddBack(TASK_0_2);
|
tasks.addPendingTaskToAddBack(TASK_0_2);
|
||||||
tasks.addPendingTaskToUpdateInputPartitions(TASK_1_1, mkSet(TOPIC_PARTITION_B_0));
|
tasks.addPendingTaskToUpdateInputPartitions(TASK_1_1, mkSet(TOPIC_PARTITION_B_0));
|
||||||
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(TASK_1_1, mkSet(TOPIC_PARTITION_B_0));
|
||||||
tasks.addPendingActiveTaskToSuspend(TASK_1_2);
|
tasks.addPendingActiveTaskToSuspend(TASK_1_2);
|
||||||
assertTrue(tasks.hasPendingTasksToRecycle());
|
assertTrue(tasks.hasPendingTasksToRecycle());
|
||||||
|
|
||||||
|
@ -179,6 +180,7 @@ public class TasksTest {
|
||||||
tasks.addPendingTaskToCloseClean(TASK_0_1);
|
tasks.addPendingTaskToCloseClean(TASK_0_1);
|
||||||
tasks.addPendingTaskToAddBack(TASK_0_2);
|
tasks.addPendingTaskToAddBack(TASK_0_2);
|
||||||
tasks.addPendingTaskToUpdateInputPartitions(TASK_1_1, mkSet(TOPIC_PARTITION_B_0));
|
tasks.addPendingTaskToUpdateInputPartitions(TASK_1_1, mkSet(TOPIC_PARTITION_B_0));
|
||||||
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(TASK_1_1, mkSet(TOPIC_PARTITION_B_0));
|
||||||
tasks.addPendingActiveTaskToSuspend(TASK_1_2);
|
tasks.addPendingActiveTaskToSuspend(TASK_1_2);
|
||||||
assertTrue(tasks.hasPendingTasksToInit());
|
assertTrue(tasks.hasPendingTasksToInit());
|
||||||
|
|
||||||
|
@ -198,6 +200,18 @@ public class TasksTest {
|
||||||
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void shouldAddAndRemovePendingTaskToCloseReviveAndUpdateInputPartitions() {
|
||||||
|
final Set<TopicPartition> expectedInputPartitions = mkSet(TOPIC_PARTITION_A_0);
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
|
|
||||||
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0, expectedInputPartitions);
|
||||||
|
final Set<TopicPartition> actualInputPartitions = tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0);
|
||||||
|
|
||||||
|
assertEquals(expectedInputPartitions, actualInputPartitions);
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void shouldAddAndRemovePendingTaskToCloseClean() {
|
public void shouldAddAndRemovePendingTaskToCloseClean() {
|
||||||
assertFalse(tasks.removePendingTaskToCloseClean(TASK_0_0));
|
assertFalse(tasks.removePendingTaskToCloseClean(TASK_0_0));
|
||||||
|
@ -237,6 +251,19 @@ public class TasksTest {
|
||||||
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
||||||
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
assertNotNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
assertNotNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void onlyRemovePendingTaskToCloseReviveAndUpdateInputPartitionsShouldRemoveTaskFromPendingUpdateActions() {
|
||||||
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0, mkSet(TOPIC_PARTITION_A_0));
|
||||||
|
|
||||||
|
assertFalse(tasks.removePendingTaskToAddBack(TASK_0_0));
|
||||||
|
assertFalse(tasks.removePendingTaskToCloseClean(TASK_0_0));
|
||||||
|
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
|
assertNotNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -248,6 +275,7 @@ public class TasksTest {
|
||||||
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
||||||
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
assertNotNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
assertNotNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -259,6 +287,7 @@ public class TasksTest {
|
||||||
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
assertTrue(tasks.removePendingTaskToCloseClean(TASK_0_0));
|
assertTrue(tasks.removePendingTaskToCloseClean(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -270,6 +299,7 @@ public class TasksTest {
|
||||||
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
assertTrue(tasks.removePendingTaskToAddBack(TASK_0_0));
|
assertTrue(tasks.removePendingTaskToAddBack(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -281,6 +311,7 @@ public class TasksTest {
|
||||||
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
assertNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
assertNull(tasks.removePendingTaskToUpdateInputPartitions(TASK_0_0));
|
||||||
assertTrue(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
assertTrue(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -306,8 +337,13 @@ public class TasksTest {
|
||||||
assertTrue(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
assertTrue(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
||||||
|
|
||||||
tasks.addPendingActiveTaskToSuspend(TASK_0_0);
|
tasks.addPendingActiveTaskToSuspend(TASK_0_0);
|
||||||
tasks.addPendingTaskToRecycle(TASK_0_0, mkSet(TOPIC_PARTITION_A_0));
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0, mkSet(TOPIC_PARTITION_A_0));
|
||||||
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
assertFalse(tasks.removePendingActiveTaskToSuspend(TASK_0_0));
|
||||||
|
assertNotNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
|
|
||||||
|
tasks.addPendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0, mkSet(TOPIC_PARTITION_A_0));
|
||||||
|
tasks.addPendingTaskToRecycle(TASK_0_0, mkSet(TOPIC_PARTITION_A_0));
|
||||||
|
assertNull(tasks.removePendingTaskToCloseReviveAndUpdateInputPartitions(TASK_0_0));
|
||||||
assertNotNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
assertNotNull(tasks.removePendingTaskToRecycle(TASK_0_0));
|
||||||
}
|
}
|
||||||
}
|
}
|
Loading…
Reference in New Issue