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


##########
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)
+                        }
+                      }
+                    })
                 }
+              })
+          }
+        } else {
+          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 = {
-                  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)
+                      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)
+                        }
+                      }
+                    })
                 }
               })
+          }
         }
-      }
-      // 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.
+      var timedOut = false
       try {
         Futures.allAsList(results: _*).get(shuffleMergeResultsTimeoutSec, 
TimeUnit.SECONDS)
       } catch {
         case _: TimeoutException =>
+          timedOut = true
           logInfo(s"Timed out on waiting for merge results from all " +
             s"$numMergers mergers for shuffle $shuffleId")
       } finally {
+        if (timedOut || !registerMergeResults) {
+          cancelFinalizeShuffleMergeFutures(scheduledFutures,

Review Comment:
   Hmmm. Even if the send finalize tasks of future stages get blocked, with the 
other changes here, DAGScheduler will not stall and will schedule next stages. 
Just that the finalization of the future stages will just timeout. And since we 
are talking about small shuffle job, it is fine that push/merge didn't succeed. 
We are adding this code to address an extreme case but that increases the 
chances of shuffle service not cleaning up metadata/closing shuffle files which 
it does during finalization.
   And ideally to address these issues the `connectionCreationTimeout` should 
be lowered. This was one of the reasons to introduce that configuration.



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