mridulm commented on code in PR #49270:
URL: https://github.com/apache/spark/pull/49270#discussion_r1897189837


##########
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala:
##########
@@ -1212,6 +1212,16 @@ private[spark] class TaskSchedulerImpl(
 
   override def applicationAttemptId(): Option[String] = 
backend.applicationAttemptId()
 
+  override def hasRunningTasks(stageId: Int): Boolean = synchronized {
+    var hasRunningTasks = false
+    taskSetsByStageIdAndAttempt.get(stageId).foreach { attempts =>
+      attempts.foreach { case (_, tsm) =>
+        hasRunningTasks = hasRunningTasks || tsm.runningTasksSet.nonEmpty
+      }
+    }
+    hasRunningTasks

Review Comment:
   ```suggestion
       taskSetsByStageIdAndAttempt.get(stageId).exists{ attempts =>
         attempts.exists(_._2.runningTasksSet.nonEmpty)
       }
   ```



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2187,7 +2187,8 @@ private[spark] class DAGScheduler(
                   log"we will roll back and rerun below stages which include 
itself and all its " +
                   log"indeterminate child stages: ${MDC(STAGES, 
rollingBackStages)}")
               }
-
+              failedStage.markResubmitInFetchFailed()
+              mapStage.markResubmitInFetchFailed()

Review Comment:
   nit: `markResubmitInFetchFailed` -> `setResubmitByFetchFailure`.
   
   Also, please reset this to `false` - when stage transitions to running - so 
that scheduler state is consistent.
   `submitMissingTasks` preferably, or in `submitStage`



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to