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


##########
core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala:
##########
@@ -643,10 +643,11 @@ private[spark] class ExecutorAllocationManager(
     // Should be 0 when no stages are active.
     private val stageAttemptToNumRunningTask = new 
mutable.HashMap[StageAttempt, Int]
     private val stageAttemptToTaskIndices = new mutable.HashMap[StageAttempt, 
mutable.HashSet[Int]]
-    // Number of speculative tasks pending/running in each stageAttempt
-    private val stageAttemptToNumSpeculativeTasks = new 
mutable.HashMap[StageAttempt, Int]
-    // The speculative tasks started in each stageAttempt
+    // Number of speculative tasks running in each stageAttempt
     private val stageAttemptToSpeculativeTaskIndices =
+      new mutable.HashMap[StageAttempt, mutable.HashSet[Int]]()

Review Comment:
   Note: Ideally, we simply need an `Int` for this - since we have split 
`stageAttemptToUnsubmittedSpeculativeTasks`.
   For now, let us keep it as a HashSet though - I have slight concerns around 
dropped events which might make the map's go inconsistent.



##########
core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala:
##########
@@ -722,9 +723,8 @@ private[spark] class ExecutorAllocationManager(
         // because the attempt may still have running tasks,
         // even after another attempt for the stage is submitted.
         stageAttemptToNumTasks -= stageAttempt
-        stageAttemptToNumSpeculativeTasks -= stageAttempt
+        stageAttemptToUnsubmittedSpeculativeTasks -= stageAttempt

Review Comment:
   Remove from both maps.



##########
core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala:
##########
@@ -896,9 +897,8 @@ private[spark] class ExecutorAllocationManager(
     }
 
     private def getPendingSpeculativeTaskSum(attempt: StageAttempt): Int = {
-      val numTotalTasks = stageAttemptToNumSpeculativeTasks.getOrElse(attempt, 
0)
-      val numRunning = 
stageAttemptToSpeculativeTaskIndices.get(attempt).map(_.size).getOrElse(0)
-      numTotalTasks - numRunning
+      stageAttemptToUnsubmittedSpeculativeTasks
+        .getOrElse(attempt, mutable.HashSet.empty[Int]).size

Review Comment:
   nit:
   ```suggestion
         
stageAttemptToUnsubmittedSpeculativeTasks.get(attempt).map(_.size).getOrElse(0)
   ```



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -383,8 +383,8 @@ private[spark] class DAGScheduler(
   /**
    * Called by the TaskSetManager when it decides a speculative task is needed.
    */
-  def speculativeTaskSubmitted(task: Task[_]): Unit = {
-    eventProcessLoop.post(SpeculativeTaskSubmitted(task))
+  def speculativeTaskSubmitted(task: Task[_], taskIndex: Int = -1): Unit = {
+    eventProcessLoop.post(SpeculativeTaskSubmitted(task, taskIndex))

Review Comment:
   Why not simply use `task.partitionId` instead for `taskIndex` ?



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to