wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r959685431


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2242,60 +2251,57 @@ private[spark] class DAGScheduler(
     val numMergers = stage.shuffleDep.getMergerLocs.length
     val results = (0 until numMergers).map(_ => 
SettableFuture.create[Boolean]())
     externalShuffleClient.foreach { shuffleClient =>
-      if (!registerMergeResults) {
-        results.foreach(_.set(true))
-        // Finalize in separate thread as shuffle merge is a no-op in this case
-        shuffleMergeFinalizeScheduler.schedule(new Runnable {
-          override def run(): Unit = {
-            stage.shuffleDep.getMergerLocs.foreach {
-              case shuffleServiceLoc =>
-                // Sends async request to shuffle service to finalize shuffle 
merge on that host.
-                // Since merge statuses will not be registered in this case,
-                // we pass a no-op listener.
-                shuffleClient.finalizeShuffleMerge(shuffleServiceLoc.host,
-                  shuffleServiceLoc.port, shuffleId, shuffleMergeId,
-                  new MergeFinalizerListener {
-                    override def onShuffleMergeSuccess(statuses: 
MergeStatuses): Unit = {
-                    }
+      val scheduledFutures = stage.shuffleDep.getMergerLocs.zipWithIndex.map {
+        case (shuffleServiceLoc, index) =>
+          // Sends async request to shuffle service to finalize shuffle merge 
on that host
+          // TODO: SPARK-35536: Cancel finalizeShuffleMerge if the stage is 
cancelled
+          // TODO: during shuffleMergeFinalizeWaitSec
+          shuffleSendFinalizeRpcExecutor.submit(new Runnable() {
+            override def run(): Unit = {
+              shuffleClient.finalizeShuffleMerge(shuffleServiceLoc.host,
+                shuffleServiceLoc.port, shuffleId, shuffleMergeId,
+                new MergeFinalizerListener {
+                  override def onShuffleMergeSuccess(statuses: MergeStatuses): 
Unit = {
+                    assert(shuffleId == statuses.shuffleId)
+                    eventProcessLoop.post(RegisterMergeStatuses(stage, 
MergeStatus.
+                      convertMergeStatusesToMergeStatusArr(statuses, 
shuffleServiceLoc)))
+                    results(index).set(true)
+                  }
 
-                    override def onShuffleMergeFailure(e: Throwable): Unit = {
+                  override def onShuffleMergeFailure(e: Throwable): Unit = {
+                    logWarning(s"Exception encountered when trying to finalize 
shuffle " +
+                      s"merge on ${shuffleServiceLoc.host} for shuffle 
$shuffleId", e)
+                    // Do not fail the future as this would cause dag 
scheduler to prematurely
+                    // give up on waiting for merge results from the remaining 
shuffle services
+                    // if one fails
+                    if (e.isInstanceOf[IOException]) {
+                      logInfo(s"Failed to connect external shuffle service " +
+                        s"${shuffleServiceLoc.hostPort}")
+                      
blockManagerMaster.removeShufflePushMergerLocation(shuffleServiceLoc.host)
+                      results(index).set(false)
                     }
-                  })
+                  }
+                })
             }
-          }
-        }, 0, TimeUnit.SECONDS)
-      } else {
-        stage.shuffleDep.getMergerLocs.zipWithIndex.foreach {
-          case (shuffleServiceLoc, index) =>
-            // Sends async request to shuffle service to finalize shuffle 
merge on that host
-            // TODO: SPARK-35536: Cancel finalizeShuffleMerge if the stage is 
cancelled
-            // TODO: during shuffleMergeFinalizeWaitSec
-            shuffleClient.finalizeShuffleMerge(shuffleServiceLoc.host,
-              shuffleServiceLoc.port, shuffleId, shuffleMergeId,
-              new MergeFinalizerListener {
-                override def onShuffleMergeSuccess(statuses: MergeStatuses): 
Unit = {
-                  assert(shuffleId == statuses.shuffleId)
-                  eventProcessLoop.post(RegisterMergeStatuses(stage, 
MergeStatus.
-                    convertMergeStatusesToMergeStatusArr(statuses, 
shuffleServiceLoc)))
-                  results(index).set(true)
-                }
-
-                override def onShuffleMergeFailure(e: Throwable): Unit = {
-                  logWarning(s"Exception encountered when trying to finalize 
shuffle " +
-                    s"merge on ${shuffleServiceLoc.host} for shuffle 
$shuffleId", e)
-                  // Do not fail the future as this would cause dag scheduler 
to prematurely
-                  // give up on waiting for merge results from the remaining 
shuffle services
-                  // if one fails
-                  results(index).set(false)
-                }
-              })
-        }
+          })
       }
-      // DAGScheduler only waits for a limited amount of time for the merge 
results.
-      // It will attempt to submit the next stage(s) irrespective of whether 
merge results
-      // from all shuffle services are received or not.
       try {
-        Futures.allAsList(results: _*).get(shuffleMergeResultsTimeoutSec, 
TimeUnit.SECONDS)
+        val timedOutTask: Runnable = () => {
+          // Waits for a limited amount of time for the merge results.
+          Futures.allAsList(results: _*).get(shuffleMergeResultsTimeoutSec, 
TimeUnit.SECONDS)
+          // Cancel sending rpc tasks due to connection slow
+          scheduledFutures.map(future => {
+            if (!future.isDone) {
+              future.cancel(true)
+            }

Review Comment:
   We will wait for all the tasks that send the finalize RPC to complete within 
shuffleMergeResultsTimeoutSec, or we will forcefully terminate them.
   If `registerMergeResults = false`, we will do that in a new thread to not 
block the main workflow?
   



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