vamossagar12 commented on code in PR #11433:
URL: https://github.com/apache/kafka/pull/11433#discussion_r1158249466


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##########
@@ -342,7 +342,18 @@ public void handleAssignment(final Map<TaskId, 
Set<TopicPartition>> activeTasks,
 
         maybeThrowTaskExceptions(taskCloseExceptions);
 
-        createNewTasks(activeTasksToCreate, standbyTasksToCreate);
+        final Collection<Task> newActiveTasks = 
createNewTasks(activeTasksToCreate, standbyTasksToCreate);
+        // If there are any transactions in flight and there are newly created 
active tasks, commit the tasks
+        // to avoid potential long restoration times.
+        if (processingMode == EXACTLY_ONCE_V2 && 
threadProducer().transactionInFlight() && !newActiveTasks.isEmpty()) {
+            log.info("New active tasks were added and there is an inflight 
transaction. Attempting to commit tasks.");
+            final int numCommitted = 
commitTasksAndMaybeUpdateCommittableOffsets(newActiveTasks, new HashMap<>());
+            if (numCommitted == -1) {

Review Comment:
   Thanks @guozhangwang for clarifying. I moved the `rebalance=false` 
assignment from `handleRebalanceComplete` into `handleAssignment` right before 
`createNewTasks`. It appears to me that it has a side effect. This test in 
`StreamThreadTest#shouldNotEnforceRebalanceWhenCurrentlyRebalancing` is a good 
indicator where in it's expecting a clean shutdown and a state transition to 
DEAD state but it's never happening.
   
   Because the rebalanceInProgress is never falsified, the `runLoop` stays in 
an infinite loop in `StreamThread` since this condition is never met:
   
   ```
   if (!taskManager.rebalanceInProgress() && nextProbingRebalanceMs.get() < 
time.milliseconds()) {
                       log.info("Triggering the followup rebalance scheduled 
for {}.", Utils.toLogDateTimeFormat(nextProbingRebalanceMs.get()));
                       mainConsumer.enforceRebalance("triggered followup 
rebalance scheduled for " + nextProbingRebalanceMs.get());
                       nextProbingRebalanceMs.set(Long.MAX_VALUE);
                   }
   ```
   In the successful case(i.e with the rebalanceInProgress=false; retained in 
`handleRebalanceComplete` I see this: 
   
   ```
   [2023-04-05 14:39:20,664] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] 
Informed to shut down 
(org.apache.kafka.streams.processor.internals.StreamThread:1168)
   [2023-04-05 14:39:22,442] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] State 
transition from STARTING to PENDING_SHUTDOWN 
(org.apache.kafka.streams.processor.internals.StreamThread:239)
   [2023-04-05 14:40:31,036] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] 
Triggering the followup rebalance scheduled for 2023-04-05 14:39:14,762 +05:30. 
(org.apache.kafka.streams.processor.internals.StreamThread:621)
   [2023-04-05 14:40:31,039] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] 
Shutting down clean 
(org.apache.kafka.streams.processor.internals.StreamThread:1182)
   [2023-04-05 14:40:31,060] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] State 
transition from PENDING_SHUTDOWN to DEAD 
(org.apache.kafka.streams.processor.internals.StreamThread:239)
   [2023-04-05 14:40:31,060] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] 
Shutdown complete 
(org.apache.kafka.streams.processor.internals.StreamThread:1217)
   ```
   
   while in the other case, the follow up rebalance is never scheduled which 
means the current one never ends and the StreamThread can never shutdown looks 
like:
   
   ```
   [2023-04-05 14:43:11,201] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] 
Starting (org.apache.kafka.streams.processor.internals.StreamThread:572)
   [2023-04-05 14:43:11,205] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] State 
transition from CREATED to STARTING 
(org.apache.kafka.streams.processor.internals.StreamThread:239)
   [2023-04-05 14:43:11,221] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] 
Informed to shut down 
(org.apache.kafka.streams.processor.internals.StreamThread:1168)
   [2023-04-05 14:43:11,221] INFO stream-thread 
[stream-thread-test-87bf53a8-54f2-485f-a4b6-acdbec0a8b3d-StreamThread-1] State 
transition from STARTING to PENDING_SHUTDOWN 
(org.apache.kafka.streams.processor.internals.StreamThread:239)
   ```



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