vvcephei commented on a change in pull request #8856: URL: https://github.com/apache/kafka/pull/8856#discussion_r439532197
########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java ########## @@ -112,10 +112,19 @@ public void completeRestoration() { @Override public void suspend() { log.trace("No-op suspend with state {}", state()); - if (state() == State.RUNNING) { - transitionTo(State.SUSPENDED); - } else if (state() == State.RESTORING) { - throw new IllegalStateException("Illegal state " + state() + " while suspending standby task " + id); + switch (state()) { + case CREATED: + case RUNNING: + case SUSPENDED: + transitionTo(State.SUSPENDED); + break; + + case RESTORING: Review comment: I get that standbys should never really be in RESTORING state, but it still doesn't seem like it's philosophically any more illegal to suspend from RESTORING than it is from RUNNING. I'd vote to legalize RESTORING here. It does seem like a useful sanity check for CLOSED to be illegal, though. ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java ########## @@ -66,11 +66,11 @@ * </pre> */ enum State { - CREATED(1, 4), // 0 - RESTORING(2, 3, 4), // 1 - RUNNING(3), // 2 - SUSPENDED(1, 4), // 3 - CLOSED(0); // 4, we allow CLOSED to transit to CREATED to handle corrupted tasks + CREATED(1, 3, 4), // 0 + RESTORING(2, 3, 4), // 1 + RUNNING(3), // 2 + SUSPENDED(1, 3, 4), // 3 + CLOSED(0); // 4, we allow CLOSED to transit to CREATED to handle corrupted tasks Review comment: It looks like your changes in the tasks have prohibited any state from transitioning to CLOSED except SUSPENDED. Should we update the state machine to reflect this? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java ########## @@ -219,13 +214,19 @@ public void handleAssignment(final Map<TaskId, Set<TopicPartition>> activeTasks, tasksToRecycle.add(task); } else { try { - task.suspend(); - final Map<TopicPartition, OffsetAndMetadata> committableOffsets = task.prepareCommit(); - - tasksToClose.add(task); - if (!committableOffsets.isEmpty()) { - consumedOffsetsAndMetadataPerTask.put(task.id(), committableOffsets); + task.suspend(); // Should be a no-op for all active tasks, unless we hit an exception during handleRevocation + if (task.commitNeeded()) { + if (task.isActive()) { + log.error("Active task {} was revoked and should have already been committed", task.id()); + throw new IllegalStateException("Revoked active task was not committed during handleRevocation"); + } else { + task.prepareCommit(); + task.postCommit(); + } } + completeTaskCloseClean(task); + cleanUpTaskProducer(task, taskCloseExceptions); + taskIter.remove(); Review comment: I think we previously followed the "loop over iterator and remove during iteration" pattern, and we got away from it because it was too confusing. Do we really need to re-introduce it now? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org