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


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2242,70 +2252,110 @@ 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 =
+        if (!registerMergeResults) {
+          results.foreach(_.set(true))
+          // Finalize in separate thread as shuffle merge is a no-op in this 
case
+          stage.shuffleDep.getMergerLocs.map {
+            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.
+              shuffleSendFinalizeRpcExecutor.submit(new Runnable() {
+                override def run(): Unit = {
+                  shuffleClient.finalizeShuffleMerge(shuffleServiceLoc.host,
+                    shuffleServiceLoc.port, shuffleId, shuffleMergeId,
+                    new MergeFinalizerListener {
+                      override def onShuffleMergeSuccess(statuses: 
MergeStatuses): Unit = {
+                      }
 
-                    override def onShuffleMergeFailure(e: Throwable): Unit = {
-                    }
-                  })
-            }
-          }
-        }, 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 = 
{
+                        if (e.isInstanceOf[IOException]) {
+                          logInfo(s"Failed to connect external shuffle service 
" +
+                            s"${shuffleServiceLoc.hostPort}")
+                          
blockManagerMaster.removeShufflePushMergerLocation(shuffleServiceLoc.host)

Review Comment:
   Because we don't want those nodes that failed to connect to be selected as 
merge locations again. 
   The merge locations come from two parts, the active block managers and the 
shuffle services that current app registered an executor in the past. We will 
remove node that failed to connect from the second part.
   So if the merge ESS is still active, it will still can be selected as merge 
location as it is in the first part. Or it will not be selected as merge 
location if it is down.



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