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


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2266,58 @@ private[spark] class DAGScheduler(
                     }
 
                     override def onShuffleMergeFailure(e: Throwable): Unit = {
+                      if (e.isInstanceOf[IOException]) {
+                        logInfo(s"Failed to connect external shuffle service 
on " +
+                          s"${shuffleServiceLoc.host} and add it to blacklist")
+                        
blockManagerMaster.removeShufflePushMergerLocation(shuffleServiceLoc.host)
+                        finalizeBlackNodes.put(shuffleServiceLoc.host, 
shuffleServiceLoc.host)
+                      }
                     }
                   })
+
+              case _ =>
             }
           }
         }, 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)
-                }
+        shuffleMergeFinalizeScheduler.schedule(new Runnable {
+          override def run(): Unit = {
+            stage.shuffleDep.getMergerLocs.zipWithIndex.foreach {
+              case (shuffleServiceLoc, index)
+                if finalizeBlackNodes.getIfPresent(shuffleServiceLoc.host) == 
null =>
+                // 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)
-                }
-              })
-        }
+                    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 
on " +
+                          s"${shuffleServiceLoc.host} and add it to blacklist")
+                        
blockManagerMaster.removeShufflePushMergerLocation(shuffleServiceLoc.host)
+                        finalizeBlackNodes.put(shuffleServiceLoc.host, 
shuffleServiceLoc.host)
+                      }
+                      results(index).set(false)
+                    }
+                  })
+
+              case (_, index) => results(index).set(true)
+            }
+          }
+        }, 0, TimeUnit.SECONDS)

Review Comment:
   If `shuffleMergeFinalizeScheduler` has just 1 thread, `finalizeShuffleMerge` 
method and send RPCs to merger locations will run one by one. so we won't get 
any merged results, and stages will behave as if push based shuffle was 
disabled. 
   
   A small UT code
   ```java
   test("test schedule executor with only one thread") {
       import java.util.concurrent.TimeoutException
       import com.google.common.util.concurrent.Futures
       import com.google.common.util.concurrent.SettableFuture
       import org.apache.spark.util.ThreadUtils
   
       logInfo("UT start")
       val pool =
         ThreadUtils.newDaemonThreadPoolScheduledExecutor("test_thread_pool", 1)
   
       val workerNum = 2
       pool.schedule(new Runnable() {
         override def run(): Unit = {
           logInfo("run finalizeShuffleMerge method")
           val results = (0 until workerNum).map(_ => 
SettableFuture.create[Boolean]())
           pool.schedule(new Runnable() {
             override def run(): Unit = {
               logInfo("run finalizeShuffleMerge method")
               (0 until workerNum).map(index => {
                 logInfo(s"begin send finalize RPC to ESS $index")
                 Thread.sleep(2000)
                 logInfo(s"end send finalize RPC to ESS $index")
                 try{
                   results(index).set(true)
                 } catch {
                   case ex: Throwable =>
                     logError(s"Fail to set result status")
                 }
               })
             }
           }, 0, TimeUnit.SECONDS)
   
           try {
             Futures.allAsList(results: _*).get(5, TimeUnit.SECONDS)
           } catch {
             case _: TimeoutException =>
               logError(s"Timed out exception from main thread")
           }
         }
       }, 0, TimeUnit.SECONDS)
   
       Thread.sleep(30000L)
       logInfo("UT end")
     }
   ```
   
   ```
   22/08/24 15:52:39.584 ScalaTest-run-running-DAGSchedulerSuite INFO 
DAGSchedulerSuite: UT start
   22/08/24 15:52:39.595 test_thread_pool-0 INFO DAGSchedulerSuite: run 
finalizeShuffleMerge method
   22/08/24 15:52:44.614 test_thread_pool-0 ERROR DAGSchedulerSuite: Timed out 
exception from main thread
   22/08/24 15:52:44.614 test_thread_pool-0 INFO DAGSchedulerSuite: run 
finalizeShuffleMerge method
   22/08/24 15:52:44.615 test_thread_pool-0 INFO DAGSchedulerSuite: begin send 
finalize RPC to ESS 0
   22/08/24 15:52:46.621 test_thread_pool-0 INFO DAGSchedulerSuite: end send 
finalize RPC to ESS 0
   22/08/24 15:52:46.623 test_thread_pool-0 INFO DAGSchedulerSuite: begin send 
finalize RPC to ESS 1
   22/08/24 15:52:48.625 test_thread_pool-0 INFO DAGSchedulerSuite: end send 
finalize RPC to ESS 1
   22/08/24 15:53:09.597 ScalaTest-run-running-DAGSchedulerSuite INFO 
DAGSchedulerSuite: UT end
   ```



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