You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/08/16 05:52:28 UTC

[GitHub] [spark] wankunde opened a new pull request, #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

wankunde opened a new pull request, #37533:
URL: https://github.com/apache/spark/pull/37533

   
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   
   Bug fix for finalize shuffle stage slow. When we enable push based shuffle and some merger external shuffle service nodes were lost, driver try to connect those merger shuffle services and send finalize RPC one by one, each connection creation will timeout after SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY (120s by default). This will be very slow.
   
   We will send these RPCs in shuffleMergeFinalizeScheduler  thread pool and handle the connection creation exception
   
   ### Why are the changes needed?
   
   Send finalize RPCs will block the main thread  due to creating connection to some unreachable nodes.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No
   
   ### How was this patch tested?
   
   Add UT
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1225359818

   > So the issue is that the wait period timer doesn't take into account the time for connection creation which is a bug. However, in this PR you are adding another major change of excluding merger nodes based on this. I don't think we should combine that with the fix for this bug because if we decide to do that then we need to pay more careful consideration of how it interacts with exclusion of executor nodes, etc.
   
   Hi, @otterc  thanks for your review. I have remove the logical for adding merger nodes into blacklist if an IOException was thrown. Now each stage needs to wait for `min(SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT * lostMergerNodesSize, PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT)`, maybe we can optimize this later.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1226729326

   Hi, @otterc @mridulm , I updated the code, could you help to review the new code?


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r968109053


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4443,37 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+    initPushBasedShuffleConfs(conf)
+
+    val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
+    val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
+    blockStoreClientField.setAccessible(true)
+    blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+    val sentHosts = ArrayBuffer[String]()
+    doAnswer { (invoke: InvocationOnMock) =>
+      val host = invoke.getArgument[String](0)
+      sentHosts += host
+      // Block FinalizeShuffleMerge rpc for 2 seconds
+      if (invoke.getArgument[String](0) == "hostA") {
+        Thread.sleep(2000)
+      }
+    }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
+
+    val shuffleMapRdd = new MyRDD(sc, 1, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
+
+    Seq(true, false).foreach { registerMergeResults =>
+      sentHosts.clear()
+      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
+      verify(blockStoreClient, times(2))
+        .finalizeShuffleMerge(any(), any(), any(), any(), any())
+      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
+      reset(blockStoreClient)
+    }
+  }

Review Comment:
   This test should be essentially checking for merge finalize getting sent to `hostB` even though `hostA` is blocked
   
   Something like:
   
   ```
   initPushBasedShuffleConfs(conf)
   val timeoutSecs = 1
   conf.set("spark.shuffle.push.results.timeout", s"${timeoutSecs}s")
   
   ...
   
   val sendRequestsLatch = new CountDownLatch(shuffleDep.getMergerLocs.size)
   // does not necessary need to be latch, can be any other lock as well
   val canSendRequestLatch = new CountDownLatch(1)
   
   var hostAInterrupted = false
   doAnswer { (invoke: InvocationOnMock) =>
         sendRequestsLatch.countDown()
   
         val host = invoke.getArgument[String](0)
         // Block FinalizeShuffleMerge rpc until allowed by main thread
         try {
           if (invoke.getArgument[String](0) == "hostA") {
             // the actual value does not matter .... just high enough not to be affected by jitter
             canSendRequestLatch.await(timeoutSecs * 5, TimeUnit.SECONDS)
           }
           sentHosts += host
         } catch {
           case iEx: InterruptedException => hostAInterrupted = true
         }
       }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
   
   ...
       scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
       assert(sentHosts.toSeq === Seq("hostB"))
       assert(hostAInterrupted)
   
     ...
    
   ```
   
   Thoughts ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956887936


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   ESS does not connect back to the executors or driver - connections/requests are  initiated by executors and driver to ESS.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r961896092


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,16 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_BASED_SHUFFLE_SEND_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .doc("Number of threads used by driver to send finalize shuffle RPC to the merger" +

Review Comment:
   Nit-> Number of threads used by the driver to send finalize shuffle RPC to mergers.



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,16 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_BASED_SHUFFLE_SEND_FINALIZE_RPC_THREADS =

Review Comment:
   Nit-> can this be renamed to `PUSH_SHUFFLE_FINALIZE_RPC_THREADS`



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.

Review Comment:
   Please change the comment to explain what this scheduler is used for.



##########
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:
   I don't think we should do this. Do we get any benefits from cancelling any pending send RPC tasks? I understand that when the timeout is elapsed, then the driver will not use the results from the shuffle service for that shuffle but at least the shuffle service can process the finalize request and close all the files for that shuffle. 
   At the least, I think we shouldn't be clubbing this behavior change with the bug fix.
   I know there was a discussion about this and sorry for commenting late on this.
   cc @mridulm 



##########
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:
   Why are we doing this? Network issues can be transient. We don't have retries for sending finalize RPC which is fine because push/merge operation is best effort. However, why do we decide remove the host from shuffle push merger location. I don't recommend clubbing these changes in the bug fix.



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // Send finalize RPC tasks to merger ESS, one thread per RPC and will be cancelled after
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. Please close the opened files in the merger ESS

Review Comment:
   `Please close the opened files in the merger ESS...` why is that added as a comment here?



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -273,6 +274,9 @@ private[spark] class DAGScheduler(
   private val shuffleMergeFinalizeNumThreads =
     sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_THREADS)
 
+  private val shuffleSendFinalizeRpcThreads =

Review Comment:
   Nit-> rename to `shuffleFinalizeRpcThreads`



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

Review Comment:
   Same here. PTAL at the above comment



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955284322


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // The merge finalization task (per stage) will submit a asynchronous thread to send finalize
+  // RPC to the merger location and then get MergeStatus from the merger. This thread won't stop
+  // after PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
+  private val shuffleSendFinalizeRPCContext =
+    ExecutionContext.fromExecutor(ThreadUtils.newDaemonFixedThreadPool(
+      shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc"))

Review Comment:
   Ok, I missed the comments. Why did you chose 6 as the default for this config?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r968109053


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4443,37 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+    initPushBasedShuffleConfs(conf)
+
+    val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
+    val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
+    blockStoreClientField.setAccessible(true)
+    blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+    val sentHosts = ArrayBuffer[String]()
+    doAnswer { (invoke: InvocationOnMock) =>
+      val host = invoke.getArgument[String](0)
+      sentHosts += host
+      // Block FinalizeShuffleMerge rpc for 2 seconds
+      if (invoke.getArgument[String](0) == "hostA") {
+        Thread.sleep(2000)
+      }
+    }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
+
+    val shuffleMapRdd = new MyRDD(sc, 1, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
+
+    Seq(true, false).foreach { registerMergeResults =>
+      sentHosts.clear()
+      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
+      verify(blockStoreClient, times(2))
+        .finalizeShuffleMerge(any(), any(), any(), any(), any())
+      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
+      reset(blockStoreClient)
+    }
+  }

Review Comment:
   This test should be essentially checking for merge finalize getting sent to `hostB` even though `hostA` is blocked
   
   Something like:
   
   ```
   initPushBasedShuffleConfs(conf)
   val timeoutSecs = 1
   conf.set("spark.shuffle.push.results.timeout", s"${timeoutSecs}s")
   
   ...
   
   val sendRequestsLatch = new CountDownLatch(shuffleDep.getMergerLocs.size)
   // does not necessary need to be latch, can be any other lock as well
   val canSendRequestLatch = new CountDownLatch(1)
   
   var hostAInterrupted = false
   doAnswer { (invoke: InvocationOnMock) =>
         sendRequestsLatch.countDown()
   
         val host = invoke.getArgument[String](0)
         // Block FinalizeShuffleMerge rpc until allowed by main thread
         try {
           if (invoke.getArgument[String](0) == "hostA") {
             // the actual value does not matter .... just high enough not to be affected by jitter
             canSendRequestLatch.await(timeoutSecs * 5, TimeUnit.SECONDS)
           }
           sentHosts += host
         } catch {
           case iEx: InterruptedException => hostAInterrupted = true
         }
       }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
   
   ...
       scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
       assert(!sentHosts.contains("hostA"))
       assert(hostAInterrupted)
   
     ...
    
   ```
   
   Thoughts ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954486898


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // The merge finalization task (per stage) will submit a asynchronous thread to send finalize
+  // RPC to the merger location and then get MergeStatus from the merger. This thread won't stop
+  // after PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
+  private val shuffleSendFinalizeRPCContext =
+    ExecutionContext.fromExecutor(ThreadUtils.newDaemonFixedThreadPool(
+      shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc"))

Review Comment:
   Do we really need more than 1 thread for this? 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1224250892

   Hi, @otterc could you help me to review this PR? Or should I provide more information on this issue ?


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r975069710


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,18 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_SHUFFLE_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .internal()
+      .doc("Number of threads used by the driver to send finalize shuffle RPC to mergers" +
+        " location and then get MergeStatus. The thread won't stop" +
+        " PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. The merger ESS may open too many files" +

Review Comment:
   I'm sorry, update the doc to `The thread will run for up to PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973642094


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

Review Comment:
   Resolving, as it is addressed



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955615397


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   When either all mergers report success or there is a timeout (whichever is earlier), we fire `ShuffleMergeFinalized` - at which point, the shuffle is marked finalized - and all subsequent merge statuses from ESS are ignored.
   
   So sending `FinalizeShuffleMerge` to mergers does not impact the application once `ShuffleMergeFinalized` is done.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   On option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry a few times to the send the message and mitigate the issue (thoughts on adding a cancel-merge instead of finalize-merge for that shuffleid-shuffleattemptid ?). This will have no performance/functional impact on driver, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r962110234


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.

Review Comment:
   Updated



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r963122608


##########
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:
   When the connection fails to get established, it will not be always because ESS is down. It could be transient network failure. We don't retry connecting to it here so I don't think based on the connection failure we should remove the merger.
   
   > Because we don't want those nodes that failed to connect to be selected as merge locations again.
   
   This may happen any ways. Since the ESS is down, there will be fetch failures against it and the current code in DAGScheduler will handle removing the unreachable host from merger locations.
   
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r963274409


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +285,18 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // When push-based shuffle is enabled, spark driver will submit a finalize task which will send
+  // a finalize rpc to each merger ESS after the shuffle map stage is complete. The merge
+  // finalization task takes up to PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.

Review Comment:
   ```suggestion
     // finalization takes up to PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   ```



##########
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:
   @otterc, this applies when we are unable to send the finalization request to ESS - and so we start having more and more threads blocked on `shuffleClient.finalizeShuffleMerge` over time - preventing all other merge finalizations from getting submitted - as @wankunde illustrated above.
   In almost all cases, `shuffleClient.finalizeShuffleMerge` should finish very quickly - and so `cancelFinalizeShuffleMergeFutures` is effectively a noop.



##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4440,39 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 

Review Comment:
   Can we add a test specifically for the behavior we have introduced ? Namely, we send finalize shuffle even if sending msg to "first" merger blocks indefinitely (and so does not block a) other sends, b) merge finalization completes within the timeout).
   



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +285,18 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // When push-based shuffle is enabled, spark driver will submit a finalize task which will send
+  // a finalize rpc to each merger ESS after the shuffle map stage is complete. The merge
+  // finalization task takes up to PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // Send finalize RPC tasks to merger ESS, one thread per RPC and will be cancelled after
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.

Review Comment:
   ```suggestion
     // Send finalize RPC tasks to merger ESS
   ```



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2242,70 +2250,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,
+            if (timedOut) 0L else shuffleMergeResultsTimeoutSec)
+        }
         eventProcessLoop.post(ShuffleMergeFinalized(stage))
       }
     }
   }
 
+  private def cancelFinalizeShuffleMergeFutures(
+      futures: Seq[JFutrue[_]],
+      delayInSecs: Long): Unit = {
+    def cancelFutures(): Unit = {
+      futures.map(future => {
+        if (!future.isDone) {
+          future.cancel(true)
+        }
+      })

Review Comment:
   ```suggestion
         futures.foreach(_.cancel(true))
   ```
   
   



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,17 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_SHUFFLE_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .doc("Number of threads used by the driver to send finalize shuffle RPC to mergers" +
+        " location and then get MergeStatus. The thread won't stop" +
+        " PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. The merger ESS may open too many files" +
+        " if the finalize rpc is not received.")

Review Comment:
   ```suggestion
       ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
         .doc("Number of threads used by the driver to send finalize shuffle RPC to mergers." +
           " External shuffle servers initiate merge finalization on receiving this request.")
   ```
   
   Also, mark it as `internal`



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,17 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_SHUFFLE_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .doc("Number of threads used by the driver to send finalize shuffle RPC to mergers" +
+        " location and then get MergeStatus. The thread won't stop" +
+        " PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. The merger ESS may open too many files" +
+        " if the finalize rpc is not received.")
+      .version("3.3.0")

Review Comment:
   ```suggestion
         .version("3.4.0")
   ```



##########
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:
   Agree with @otterc, let us revert this change.
   We can analyze the need for it as a follow up, if required.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973869591


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4443,36 +4443,115 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
-  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely " +
+    "with registerMergeResults is true") {
     initPushBasedShuffleConfs(conf)
 
+    sc.conf.set("spark.shuffle.push.results.timeout", "1s")
+    val myScheduler = new MyDAGScheduler(
+      sc,
+      taskScheduler,
+      sc.listenerBus,
+      mapOutputTracker,
+      blockManagerMaster,
+      sc.env,
+      shuffleMergeFinalize = false)
+
+    val mergerLocs = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))
+    val timeoutSecs = 1
+    val sendRequestsLatch = new CountDownLatch(mergerLocs.size)
+    val completeLatch = new CountDownLatch(mergerLocs.size)
+    val canSendRequestLatch = new CountDownLatch(1)
+
     val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
     val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
     blockStoreClientField.setAccessible(true)
     blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+
     val sentHosts = ArrayBuffer[String]()
+    var hostAInterrupted = false
     doAnswer { (invoke: InvocationOnMock) =>
       val host = invoke.getArgument[String](0)
-      sentHosts += host
-      // Block FinalizeShuffleMerge rpc for 2 seconds
-      if (invoke.getArgument[String](0) == "hostA") {
-        Thread.sleep(2000)
+      sendRequestsLatch.countDown()
+      try {
+        if (host == "hostA") {
+          canSendRequestLatch.await(timeoutSecs * 2, TimeUnit.SECONDS)
+        }
+        sentHosts += host
+      } catch {
+        case _: InterruptedException => hostAInterrupted = true
+      } finally {
+        completeLatch.countDown()
       }
     }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
 
     val shuffleMapRdd = new MyRDD(sc, 1, Nil)
     val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
-    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
-    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
-
-    Seq(true, false).foreach { registerMergeResults =>
-      sentHosts.clear()
-      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
-      verify(blockStoreClient, times(2))
-        .finalizeShuffleMerge(any(), any(), any(), any(), any())
-      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
-      reset(blockStoreClient)
-    }
+    shuffleDep.setMergerLocs(mergerLocs)
+    val shuffleStage = myScheduler.createShuffleMapStage(shuffleDep, 0)
+
+    myScheduler.finalizeShuffleMerge(shuffleStage, true)
+    sendRequestsLatch.await()
+    verify(blockStoreClient, times(2))
+      .finalizeShuffleMerge(any(), any(), any(), any(), any())
+    assert(sentHosts === Seq("hostB"))
+    completeLatch.await()
+    assert(hostAInterrupted)
+  }
+
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely " +
+    "with registerMergeResults is false") {

Review Comment:
   Thanks @mridulm I have merged UTs



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954454349


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Thoughts on pushing finalization send into the threadpool instead ?
   
   ```
           val scheduledFutures = stage.shuffleDep.getMergerLocs.zipWithIndex.map {
             case (shuffleServiceLoc, index) =>
               shuffleMergeFinalizeScheduler.schedule(new Runnable() {
                 override def run(): Unit = {
                       // existing code within the "case"
                  
                 }
               }, 0, TimeUnit.SECONDS));
          }.toList
   ```
   
   And if there is a `TimeoutException`, we cancel all the `scheduledFutures` (`scheduledFutures.map(_.cancel(true))`)
   
   We should also bump up the default number of threads in this threadpool.
   
   This will make sure that we wait utmost for `shuffleMergeResultsTimeoutSec` seconds for finalization to complete. In almost all cases, the `shuffleClient.finalizeShuffleMerge` will be really quick - and so the overhead is fairly low - but for the rare cases where it is not, we will only block that specific send (while all other threads will send finalize message), and we will always complete within the timeout (and release threads)
   
   Thoughts ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954566063


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // The merge finalization task (per stage) will submit a asynchronous thread to send finalize
+  // RPC to the merger location and then get MergeStatus from the merger. This thread won't stop
+  // after PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
+  private val shuffleSendFinalizeRPCContext =
+    ExecutionContext.fromExecutor(ThreadUtils.newDaemonFixedThreadPool(
+      shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc"))

Review Comment:
   According to @mridulm comments, we will submit a task for each finalize RPC to one merger node. If one task tries to create a connection to a merger node, the other tasks won't be blocked, so we can get most of the merged statuses.
   If there are `shuffleSendFinalizeRPCThreads` tasks trying to create a connection to a merger node, the corresponding stages will be blocked for `SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT`
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955615397


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   When either all mergers report success or there is a timeout (whichever is earlier), we fire `ShuffleMergeFinalized` - at which point, the shuffle is marked finalized - and all subsequent merge statuses from ESS are ignored.
   
   At this point, sending `FinalizeShuffleMerge` to mergers does not impact the application once `ShuffleMergeFinalized` is done.
   So to answer your question - once there is a timeout, it does not help to send `FinalizeShuffleMerge`
   
   The current code in master is assuming that sending `FinalizeShuffleMerge` is really cheap (which would typically be the case if there are no n/w issues connecting to mergers !) - and so does it as a prefix, before starting to wait for mergers to complete.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r960777861


##########
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)
+                  }

Review Comment:
   Thanks for your explanation,  I have updated the PR.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973488467


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4443,36 +4443,115 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
-  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely " +
+    "with registerMergeResults is true") {
     initPushBasedShuffleConfs(conf)
 
+    sc.conf.set("spark.shuffle.push.results.timeout", "1s")
+    val myScheduler = new MyDAGScheduler(
+      sc,
+      taskScheduler,
+      sc.listenerBus,
+      mapOutputTracker,
+      blockManagerMaster,
+      sc.env,
+      shuffleMergeFinalize = false)
+
+    val mergerLocs = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))
+    val timeoutSecs = 1
+    val sendRequestsLatch = new CountDownLatch(mergerLocs.size)
+    val completeLatch = new CountDownLatch(mergerLocs.size)
+    val canSendRequestLatch = new CountDownLatch(1)
+
     val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
     val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
     blockStoreClientField.setAccessible(true)
     blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+
     val sentHosts = ArrayBuffer[String]()
+    var hostAInterrupted = false
     doAnswer { (invoke: InvocationOnMock) =>
       val host = invoke.getArgument[String](0)
-      sentHosts += host
-      // Block FinalizeShuffleMerge rpc for 2 seconds
-      if (invoke.getArgument[String](0) == "hostA") {
-        Thread.sleep(2000)
+      sendRequestsLatch.countDown()
+      try {
+        if (host == "hostA") {
+          canSendRequestLatch.await(timeoutSecs * 2, TimeUnit.SECONDS)
+        }
+        sentHosts += host
+      } catch {
+        case _: InterruptedException => hostAInterrupted = true
+      } finally {
+        completeLatch.countDown()
       }
     }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
 
     val shuffleMapRdd = new MyRDD(sc, 1, Nil)
     val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
-    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
-    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
-
-    Seq(true, false).foreach { registerMergeResults =>
-      sentHosts.clear()
-      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
-      verify(blockStoreClient, times(2))
-        .finalizeShuffleMerge(any(), any(), any(), any(), any())
-      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
-      reset(blockStoreClient)
-    }
+    shuffleDep.setMergerLocs(mergerLocs)
+    val shuffleStage = myScheduler.createShuffleMapStage(shuffleDep, 0)
+
+    myScheduler.finalizeShuffleMerge(shuffleStage, true)
+    sendRequestsLatch.await()
+    verify(blockStoreClient, times(2))
+      .finalizeShuffleMerge(any(), any(), any(), any(), any())
+    assert(sentHosts === Seq("hostB"))
+    completeLatch.await()
+    assert(hostAInterrupted)
+  }
+
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely " +
+    "with registerMergeResults is false") {

Review Comment:
   Can we merge this test with the previous one ?
   Essentially, something like:
   
   ```
   
   Seq(true, false) { registerMergeResults => 
     test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely " +
         s"with registerMergeResults is $registerMergeResults") {
      ....
   
      myScheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
   
     ...
   }
   ```
   
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973642323


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2311,6 +2311,15 @@ package object config {
       .intConf
       .createWithDefault(3)

Review Comment:
   Has been updated, resolving



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] AmplabJenkins commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1216218540

   Can one of the admins verify this patch?


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1219786069

   Also your solution is adding shuffle service nodes to an excluded list which isn't what the description says. Could you please explain with examples/logs of what problem are you facing.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r952800397


##########
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:
   What happens when `shuffleMergeFinalizeScheduler` has just 1 thread?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1229847247

   Hi, @mridulm @otterc 
   New processing flow:
   * Send all the finalize RPC tasks and wait for merged status in a new thread pool.
   * Wait for `shuffleMergeResultsTimeoutSec` in main thread if registerMergeResults = true, and then cancel all sending rpc tasks
   * Wait for `shuffleMergeResultsTimeoutSec` asynchronously if registerMergeResults = false, and then cancel all sending rpc tasks
   
   Does this works?
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   On option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry a few times to the send the message and mitigate the issue (thoughts on adding a cancel instead of merge ?). This will have no performance/functional impact on driver, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956839303


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Another idea, could we stop sending finalize RPCs after `shuffleMergeResultsTimeoutSec`. And start a thread in ESS that periodically checks and closes the opened files which the shuffle has already finalized?
   Do that in another PR?
   What do you think? @mridulm @otterc 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954011984


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Considering just 1 thread in the `shuffleMergeFinalizerScheduler`,  that thread will submit another task  (task to send RPCs) to pool here but then it will wait on line 2312 for the results. The thread will never get a chance to actually execute the other task (task which sends the RPCs).



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1249916683

   +CC @otterc, @Ngone51 PTAL


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r964091915


##########
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:
   You are right that the stage itself at driver wont stall after this PR @otterc - but if we dont cancel on timeout, a reduced number of mergers start receiving the finalization message in time, and we start seeing decreased merge ratio (or until the network situation improves).
   
   Cancelling the task mitigates this issue.
   The drawback being, if we had waited for some `X` seconds more, it is possible that the message could have made it through to ESS ... and so now we have a potential resource leak at shuffle service which is mitigated only when application exits.
   Unfortunately, we dont have an upper bound of how long we can wait for restoring n/w operations (note - this is no ESS being down, that is immediately identified by a conn refusal - but rather n/w issue causing conn establishment or message send to block for an extended period of time)



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1218157844

   Hi, @otterc Could you help to review this PR ? thanks


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955615397


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   When either all mergers report success or there is a timeout (whichever is earlier), we fire `ShuffleMergeFinalized` - at which point, the shuffle is marked finalized - and all subsequent merge statuses from ESS are ignored.
   
   At this point, sending `FinalizeShuffleMerge` to mergers does not impact the application once `ShuffleMergeFinalized` is done.
   So to answer your question - once there is a timeout, it does not help to send `FinalizeShuffleMerge`
   
   The current code is assuming that sending `FinalizeShuffleMerge` is really cheap (which would typically be the case if there are no n/w issues connecting to merger !) - and so does it as a prefix, before starting to wait for mergers to complete.



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   When either all mergers report success or there is a timeout (whichever is earlier), we fire `ShuffleMergeFinalized` - at which point, the shuffle is marked finalized - and all subsequent merge statuses from ESS are ignored.
   
   At this point, sending `FinalizeShuffleMerge` to mergers does not impact the application once `ShuffleMergeFinalized` is done.
   So to answer your question - once there is a timeout, it does not help to send `FinalizeShuffleMerge`
   
   The current code in master is assuming that sending `FinalizeShuffleMerge` is really cheap (which would typically be the case if there are no n/w issues connecting to merger !) - and so does it as a prefix, before starting to wait for mergers to complete.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   On option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry a few times to the send the message and mitigate the issue (thoughts on adding a cancel instead of merge for that shuffleid-shuffleattemptid ?). This will have no performance/functional impact on driver, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   One option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry a "few" times to send the message and mitigate the issue (thoughts on adding a cancel-merge instead of finalize-merge for that shuffleid-shuffleattemptid ?). This will have no performance/functional impact on driver, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954011984


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Considering just 1 thread in the `shuffleMergeFinalizerScheduler`,  that thread will submit another task  (task to send RPCs) to pool here but then it will wait on line 2312 for the results.  It will timeout because the other task is pending. This is a problem. I don't think we can use the same threadpool to run finalizeMethod and send the RPCs.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954454349


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Thoughts on pushing finalization send into the threadpool instead ?
   
   ```
           val scheduledFutures = stage.shuffleDep.getMergerLocs.zipWithIndex.map {
             case (shuffleServiceLoc, index) =>
               shuffleMergeFinalizeScheduler.schedule(new Runnable() {
                 override def run(): Unit = {
                       // existing code within the "case"
                  
                 }
               }, 0, TimeUnit.SECONDS));
          }.toList
   ```
   
   And if there is a `TimeoutException`, we cancel all the `scheduledFutures` (`scheduledFutures.map(_.cancel(true))`)
   
   We should also bump up the default number of threads in this threadpool.
   
   This will make sure that we wait utmost for `shuffleMergeResultsTimeoutSec` seconds for finalization to complete. In almost all cases, the `shuffleClient.finalizeShuffleMerge` will be really quick - and so the overhead is fairly low - but for the rare cases where it is not, we will only block that specific sending thread (while all other threads will send finalize message to other merger hosts), and we will always complete within the timeout (and release blocked threads)
   
   Thoughts ?
   +CC @venkata91 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954454349


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Thoughts on pushing finalization send into the threadpool instead ?
   
   ```
           val scheduledFutures = stage.shuffleDep.getMergerLocs.zipWithIndex.map {
             case (shuffleServiceLoc, index) =>
               shuffleMergeFinalizeScheduler.schedule(new Runnable() {
                 override def run(): Unit = {
                       // existing code within the "case"
                  
                 }
               }, 0, TimeUnit.SECONDS));
          }.toList
   ```
   
   And if there is a `TimeoutException`, we cancel all the `scheduledFutures` (`scheduledFutures.map(_.cancel(true))`)
   
   We should also bump up the default number of threads in this threadpool.
   
   This will make sure that we wait utmost for `shuffleMergeResultsTimeoutSec` seconds for finalization to complete. In almost all cases, the `shuffleClient.finalizeShuffleMerge` will be really quick - and so the overhead is fairly low - but for the rare cases where it is not, we will only block that specific sending thread (while all other threads will send finalize message to other merger hosts), and we will always complete within the timeout (and release blocked threads)
   
   Thoughts ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r959664178


##########
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)
+                  }

Review Comment:
   Much thanks for your suggestion.
   I have a confusion about the origin code. If there is small shuffle data, registerMergeResults will be false, so we don't wait for the merged statuses, but these merged statuses are still useful if they are available before the reduce tasks fetch them? This should often happen if the cluster is heavy.
   
   Another question, for your suggestion code, if all the finalize RPCs can be completed within `shuffleMergeResultsTimeoutSec`, does the `cancelFinalizeShuffleMergeFutures` task still need to wait for scheduling ?
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] MaxGekk commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r1016948598


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4443,63 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
+  Seq(true, false).foreach { registerMergeResults =>
+    test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely " +

Review Comment:
   The test is flaky. It fails periodically in different PRs. For example,
   https://github.com/MaxGekk/spark/actions/runs/3420903796/jobs/5696336947



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r962110217


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,16 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_BASED_SHUFFLE_SEND_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .doc("Number of threads used by driver to send finalize shuffle RPC to the merger" +

Review Comment:
   Updated



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,16 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_BASED_SHUFFLE_SEND_FINALIZE_RPC_THREADS =

Review Comment:
   Updated



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -273,6 +274,9 @@ private[spark] class DAGScheduler(
   private val shuffleMergeFinalizeNumThreads =
     sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_THREADS)
 
+  private val shuffleSendFinalizeRpcThreads =

Review Comment:
   Updated



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r962112740


##########
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:
   When `CONNECTIONCREATIONTIMEOUT * ((Lost_Node_Number -1) / Send_Finalize_Rpc_Thread_pool_size + 1) > shuffleMergeResultsTimeoutSec`, those futures will block the other send finalize tasks for the other stages. And shuffleMergeResultsTimeoutSec is short for small shuffle job. So I think this is helpful.
   
   WDYT?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r960287381


##########
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:
   Hopefully my comment above has clarified



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r974396707


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,18 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_SHUFFLE_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .internal()
+      .doc("Number of threads used by the driver to send finalize shuffle RPC to mergers" +
+        " location and then get MergeStatus. The thread won't stop" +
+        " PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. The merger ESS may open too many files" +

Review Comment:
   Nit: Could you please rephrase this? What do you mean to say by `The thread won't stop PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT`?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1255711624

   Merged to master.
   Thanks for working on this @wankunde !
   Thanks for the review @otterc :-)


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973642279


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // Send finalize RPC tasks to merger ESS, one thread per RPC and will be cancelled after
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. Please close the opened files in the merger ESS

Review Comment:
   Resolving



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   On option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry the send to mitigate the issue. This will have no performance/functional impact on driver, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r964129846


##########
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:
   Yes, you are right. I would prefer a decreased merge ratio than increasing the chances of resource leak at the ESS. Reason being that push/merge is best effort. When it fails for an app, that single app is impacted. The app will just be slower. With issues on the server, multiple applications get impacted.
   This is just my opinion. I am fine with whatever you folks decide.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r963281413


##########
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:
   @otterc, this applies when we are unable to send the finalization request to ESS - and so we start having more and more threads blocked on `shuffleClient.finalizeShuffleMerge` over time - preventing all other merge finalizations from getting submitted - as @wankunde illustrated above.
   In almost all cases, `shuffleClient.finalizeShuffleMerge` should finish very quickly - and so `cancelFinalizeShuffleMergeFutures` is effectively a noop.
   
   Note that if the request already made it through to ESS, the cancellation in driver would not affect the processing at ESS (though response wont be consumed)



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r966495158


##########
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:
   Discussed with @otterc - let us fix the current issue as discussed above.
   I will let @otterc explore the problem of how to minimize potential for unclosed file handles due to the task cancellation, and due to transient failure to send merge finalize request in a separate jira.
   
   Please fix the other comments in the PR @wankunde - thanks !



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r969117695


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2242,70 +2249,94 @@ 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 = {
+                      }
+                    })
                 }
+              })
+          }
+        } 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

Review Comment:
   I'm sorry,  revert this code.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1242234794

   I'm sorry for the late reply, I have updated the code.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955284322


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // The merge finalization task (per stage) will submit a asynchronous thread to send finalize
+  // RPC to the merger location and then get MergeStatus from the merger. This thread won't stop
+  // after PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
+  private val shuffleSendFinalizeRPCContext =
+    ExecutionContext.fromExecutor(ThreadUtils.newDaemonFixedThreadPool(
+      shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc"))

Review Comment:
   Ok, I missed that comment. Why did you chose 6 as the default for this config?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954467609


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Hi, @mridulm I don't think we should cancel all the `scheduledFutures` if there is a `TimeoutException`.  Merger ESS need this finalize RPC to finish its merge process. So  we should try our best to send all the RPCs even after `shuffleMergeResultsTimeoutSec`. Am I right ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r954454349


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Thoughts on pushing finalization send into the threadpool instead ?
   
   ```
           stage.shuffleDep.getMergerLocs.zipWithIndex.foreach {
             case (shuffleServiceLoc, index) =>
               shuffleMergeFinalizeScheduler.schedule(new Runnable() {
                 override def run(): Unit = {
                       // existing code within the "case"
                  
                 }
               }, 0, TimeUnit.SECONDS));
   ```
   
   We should also bump up the default number of threads in this threadpool.
   
   This will make sure that we wait utmost for `shuffleMergeResultsTimeoutSec` seconds for finalization to complete.
   
   Thoughts ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r962110338


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // Send finalize RPC tasks to merger ESS, one thread per RPC and will be cancelled after
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. Please close the opened files in the merger ESS

Review Comment:
   Remove this comment



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r968089690


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2242,70 +2249,94 @@ 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 = {
+                      }
+                    })
                 }
+              })
+          }
+        } 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
+                      }
+                    })
                 }
               })
+          }
         }
-      }
-      // 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.

Review Comment:
   Why are we removing this comment ?



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2242,70 +2249,94 @@ 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 = {
+                      }
+                    })
                 }
+              })
+          }
+        } 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

Review Comment:
   We are missing `results(index).set(false)`, from prev code, here



##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4443,37 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+    initPushBasedShuffleConfs(conf)
+
+    val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
+    val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
+    blockStoreClientField.setAccessible(true)
+    blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+    val sentHosts = ArrayBuffer[String]()
+    doAnswer { (invoke: InvocationOnMock) =>
+      val host = invoke.getArgument[String](0)
+      sentHosts += host
+      // Block FinalizeShuffleMerge rpc for 2 seconds
+      if (invoke.getArgument[String](0) == "hostA") {
+        Thread.sleep(2000)
+      }
+    }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
+
+    val shuffleMapRdd = new MyRDD(sc, 1, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
+
+    Seq(true, false).foreach { registerMergeResults =>
+      sentHosts.clear()
+      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
+      verify(blockStoreClient, times(2))
+        .finalizeShuffleMerge(any(), any(), any(), any(), any())
+      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
+      reset(blockStoreClient)
+    }
+  }

Review Comment:
   This test should be essentially checking for merge finalize getting sent to `hostB` even though `hostA` is blocked
   
   Something like:
   
   ```
   initPushBasedShuffleConfs(conf)
   val timeoutSecs = 1
   conf.set("spark.shuffle.push.results.timeout", s"${timeoutSecs}s")
   
   ...
   
   val sendRequestsLatch = new CountDownLatch(shuffleDep.getMergerLocs.size)
   // does not necessary need to be latch, can be any other lock as well
   val canSendRequestLatch = new CountDownLatch(1)
   
   var hostAInterrupted = false
   doAnswer { (invoke: InvocationOnMock) =>
         sendRequestsLatch.countDown()
   
         val host = invoke.getArgument[String](0)
         // Block FinalizeShuffleMerge rpc until allowed by main thread
         try {
           if (invoke.getArgument[String](0) == "hostA") {
             // the actual value does not matter .... just high enough not to be affected by jitter
             canSendRequestLatch.await(timeoutSecs * 5, TimeUnit.SECONDS)
           }
           sentHosts += host
         } catch {
           case iEx: InterruptedException => hostAInterrupted = true
         }
       }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
   
   ...
       scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
       assert(!sentHosts.contains("hostA"))
       assert(hostAInterrupted)
   
     ...
    
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r968109053


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4443,37 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+    initPushBasedShuffleConfs(conf)
+
+    val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
+    val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
+    blockStoreClientField.setAccessible(true)
+    blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+    val sentHosts = ArrayBuffer[String]()
+    doAnswer { (invoke: InvocationOnMock) =>
+      val host = invoke.getArgument[String](0)
+      sentHosts += host
+      // Block FinalizeShuffleMerge rpc for 2 seconds
+      if (invoke.getArgument[String](0) == "hostA") {
+        Thread.sleep(2000)
+      }
+    }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
+
+    val shuffleMapRdd = new MyRDD(sc, 1, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
+
+    Seq(true, false).foreach { registerMergeResults =>
+      sentHosts.clear()
+      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
+      verify(blockStoreClient, times(2))
+        .finalizeShuffleMerge(any(), any(), any(), any(), any())
+      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
+      reset(blockStoreClient)
+    }
+  }

Review Comment:
   This test should be essentially checking for merge finalize getting sent to `hostB` even though `hostA` is blocked
   
   Something like:
   
   ```
   initPushBasedShuffleConfs(conf)
   val timeoutSecs = 1
   conf.set("spark.shuffle.push.results.timeout", s"${timeoutSecs}s")
   
   ...
   
   val sendRequestsLatch = new CountDownLatch(shuffleDep.getMergerLocs.size)
   // does not necessary need to be latch, can be any other lock as well
   val canSendRequestLatch = new CountDownLatch(1)
   
   var hostAInterrupted = false
   doAnswer { (invoke: InvocationOnMock) =>
         sendRequestsLatch.countDown()
   
         val host = invoke.getArgument[String](0)
         // Block FinalizeShuffleMerge rpc until allowed by main thread
         try {
           if (invoke.getArgument[String](0) == "hostA") {
             // the actual value does not matter .... just high enough not to be affected by jitter
             canSendRequestLatch.await(timeoutSecs * 5, TimeUnit.SECONDS)
           }
           sentHosts += host
         } catch {
           case iEx: InterruptedException => hostAInterrupted = true
         }
       }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
   
   ...
       scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
       assert(sentHosts === Seq("hostB"))
       assert(hostAInterrupted)
   
     ...
    
   ```
   
   Thoughts ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r969118491


##########
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:
##########
@@ -4440,6 +4443,37 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti
     assert(mapStatuses.count(s => s != null && s.location.executorId == "hostB-exec") === 1)
   }
 
+  test("SPARK-40096: Send finalize events even if shuffle merger blocks indefinitely") {
+    initPushBasedShuffleConfs(conf)
+
+    val blockStoreClient = mock(classOf[ExternalBlockStoreClient])
+    val blockStoreClientField = classOf[BlockManager].getDeclaredField("blockStoreClient")
+    blockStoreClientField.setAccessible(true)
+    blockStoreClientField.set(sc.env.blockManager, blockStoreClient)
+    val sentHosts = ArrayBuffer[String]()
+    doAnswer { (invoke: InvocationOnMock) =>
+      val host = invoke.getArgument[String](0)
+      sentHosts += host
+      // Block FinalizeShuffleMerge rpc for 2 seconds
+      if (invoke.getArgument[String](0) == "hostA") {
+        Thread.sleep(2000)
+      }
+    }.when(blockStoreClient).finalizeShuffleMerge(any(), any(), any(), any(), any())
+
+    val shuffleMapRdd = new MyRDD(sc, 1, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+    shuffleDep.setMergerLocs(Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+    val shuffleStage = scheduler.createShuffleMapStage(shuffleDep, 0)
+
+    Seq(true, false).foreach { registerMergeResults =>
+      sentHosts.clear()
+      scheduler.finalizeShuffleMerge(shuffleStage, registerMergeResults)
+      verify(blockStoreClient, times(2))
+        .finalizeShuffleMerge(any(), any(), any(), any(), any())
+      assert((sentHosts diff Seq("hostA", "hostB")).isEmpty)
+      reset(blockStoreClient)
+    }
+  }

Review Comment:
   Update UT, @mridulm could you help to review it again? Thanks



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r965431205


##########
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:
   @wankunde, let me sync with @otterc offline - there are some nuances in here which I want to thrash out and make sure does not have issues.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
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: reviews-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r964091915


##########
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:
   You are right that the stage itself at driver wont stall after this PR @otterc - but if we dont cancel on timeout, a reduced number of mergers start receiving the finalization message in time, and we start seeing decreased merge ratio (or until the network situation improves).
   
   Cancelling the task mitigates this issue.
   The drawback being, if we had waited for some `X` seconds more, it is possible that the message could have made it through to ESS ... and so now we have a potential resource leak at shuffle service which is mitigated only when application exits.
   Unfortunately, we dont have an upper bound of how long we can wait for restoring n/w operations (note - this is not ESS being down, that is immediately identified by a conn refusal - but rather n/w issue causing conn establishment or message send to block for an extended period of time)



##########
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:
   You are right that the stage itself at driver wont stall after this PR @otterc - but if we dont cancel on timeout, a reduced number of mergers start receiving the finalization message in time, and we start seeing decreased merge ratio (or until the network situation improves).
   
   Cancelling the task mitigates this issue.
   The drawback being, if we had waited for some `X` seconds more, it is possible that the message could have made it through to ESS ... and so now we have a potential resource leak at shuffle service which is mitigated only when application exits.
   Unfortunately, we dont have an upper bound of how long we can wait for restoring n/w operations (note - this is not ESS being down, that is immediately identified by a conn refusal - but rather n/w issue causing conn establishment or message send to block for an extended period of time, etc)



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm closed pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm closed pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow
URL: https://github.com/apache/spark/pull/37533


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1219784129

   @wankunde 
   > Send finalize RPCs will block the main thread due to creating connection to some unreachable nodes.
   
   Which main thread are you referring to here?   Could you please explain which thread is being blocked. AFAICT this is being done by `shuffle-merge-finalizer` threads.
   
   Why can't you reduce the  `SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY` default for the cluster? One of the reasons this configuration was introduced was because creating connection should have a lower timeout that connection idle.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1220198154

   > @wankunde
   > 
   > > Send finalize RPCs will block the main thread due to creating connection to some unreachable nodes.
   > 
   > Which main thread are you referring to here? Could you please explain which thread is being blocked. AFAICT this is already being done by `shuffle-merge-finalizer` threads.
   > 
   > Why can't you reduce the `SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY` default for the cluster? One of the reasons this configuration was introduced was because creating connection should have a lower timeout than connection idle time
   
   Yes, DAGSchuedler will finalize each shuffle map stage in one `shuffle-merge-finalizer` thread,  and lock `clientPool.locks[clientIndex]` when creating connect to the ESS merger node, the other  `shuffle-merge-finalizer` threads (one stage per thread) will wait for `SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY`.
   Although reducing `SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY` helps, the total wait time( SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY * lostMergerNodesSize * stageSize ) will still be long. This PR will run `scheduleShuffleMergeFinalize()` and send `finalizeShuffleMerge` RPCs in two threads, and stop all work after `PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT` regardless of sucess or failure.
   
   Now we will only call `removeShufflePushMergerLocation` when shuffle fetch fails, this PR will also prevent these merger nodes from bing selected as mergeLocations when creating connections fails. Adding those bad merge nodes to finalizeBlackNodes, so subsequent shuffle map stages will not try to connect them.
   
   
   
   ```
   "shuffle-merge-finalizer-4" #1842 daemon prio=5 os_prio=0 tid=0x00007f19440d8000 nid=0x2be822 in Object.wait() [0x00007f19ea7f7000]
      java.lang.Thread.State: TIMED_WAITING (on object monitor)
   	at java.lang.Object.wait(Native Method)
   	at java.lang.Object.wait(Object.java:460)
   	at io.netty.util.concurrent.DefaultPromise.await0(DefaultPromise.java:679)
   	- locked <0x00007f3eb8244598> (a io.netty.bootstrap.AbstractBootstrap$PendingRegistrationPromise)
   	at io.netty.util.concurrent.DefaultPromise.await(DefaultPromise.java:298)
   	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:283)
   	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:218)
   	- locked <0x00007f1d7b0c0ba8> (a java.lang.Object)
   	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:230)
   	at org.apache.spark.network.shuffle.ExternalBlockStoreClient.finalizeShuffleMerge(ExternalBlockStoreClient.java:229)
   	at org.apache.spark.scheduler.DAGScheduler.$anonfun$finalizeShuffleMerge$5(DAGScheduler.scala:2437)
   
   "shuffle-merge-finalizer-3" #1647 daemon prio=5 os_prio=0 tid=0x00007f19440d2800 nid=0x2be52e waiting for monitor entry [0x00007f1688ff2000]
      java.lang.Thread.State: BLOCKED (on object monitor)
   	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:198)
   	- waiting to lock <0x00007f1d7b0c0ba8> (a java.lang.Object)
   	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:230)
   	at org.apache.spark.network.shuffle.ExternalBlockStoreClient.finalizeShuffleMerge(ExternalBlockStoreClient.java:229)
   	at org.apache.spark.scheduler.DAGScheduler$$anon$7.$anonfun$run$2(DAGScheduler.scala:2419)
   ...
   ```


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955615397


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   When either all mergers report success or there is a timeout (whichever is earlier), we fire `ShuffleMergeFinalized` - at which point, the shuffle is marked finalized - and all subsequent merge statuses from ESS are ignored.
   
   At this point, sending `FinalizeShuffleMerge` to mergers does not impact the application once `ShuffleMergeFinalized` is done.
   So to answer your question - once there is a timeout, it does not help to send `FinalizeShuffleMerge`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   One option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry a "few" times to send the message and mitigate the issue (thoughts on adding a cancel-merge instead of finalize-merge for that shuffleid-shuffleattemptid ?). This will have no performance/functional impact on driver/application, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r956329357


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   Agree, and that is why driver makes a best case effort to send the message.
   If we are unable to do so in reasonable time due to network issues (n/w partition, etc), failures, and so on - there could be files which are not yet closed.
   On other hand, keeping this pending state around in driver for extended periods of time will cause its own issues.
   
   Note that in general, there will always be some dropped messages (for ex when failure handler is getting invoked) - so some variant of what you described can occur even if we make this specific case robust - what we have to evaluate is how much of an impact it has on shuffle service itself in context of  whether it will negatively impact NM stability.
   
   In our environment, this has been in prod for a while now - and we have not seen this specific issue. Having said that, for a different application and network characteristics, it could possibly be an issue : more information will help understand it.
   
   
   One option could be to evaluate if we move failed sends and cancelled tasks to a different threadpool and retry a few times to the send the message and mitigate the issue (thoughts on adding a cancel-merge instead of finalize-merge for that shuffleid-shuffleattemptid ?). This will have no performance/functional impact on driver, but can mitigate ESS load in terms of open files.
   Thoughts @wankunde, @otterc ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955564176


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // The merge finalization task (per stage) will submit a asynchronous thread to send finalize
+  // RPC to the merger location and then get MergeStatus from the merger. This thread won't stop
+  // after PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
+  private val shuffleSendFinalizeRPCContext =
+    ExecutionContext.fromExecutor(ThreadUtils.newDaemonFixedThreadPool(
+      shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc"))

Review Comment:
   Because I think the number of send RPC tasks and total elapsed time in `shuffleSendFinalizeRPCContext` should bigger than tasks in `shuffleMergeFinalizeScheduler`.  



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +286,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // The merge finalization task (per stage) will submit a asynchronous thread to send finalize
+  // RPC to the merger location and then get MergeStatus from the merger. This thread won't stop
+  // after PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
+  private val shuffleSendFinalizeRPCContext =
+    ExecutionContext.fromExecutor(ThreadUtils.newDaemonFixedThreadPool(
+      shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc"))

Review Comment:
   Because I think the number of send RPC tasks and total elapsed time in `shuffleSendFinalizeRPCContext` should bigger than tasks in `shuffleMergeFinalizeScheduler`.  



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] wankunde commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
wankunde commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r955858394


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -2259,37 +2259,51 @@ private[spark] class DAGScheduler(
                     }
 
                     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)
+                      }
                     }
                   })
             }
           }
         }, 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 {

Review Comment:
   For a long-running spark application, if no finalize partition RPC is received, ess will not close open data files, meta files, and index files until the application is complete. Too many opening files can be a potential risk. 
   
   Or should we add a parameter to control whether to continue sending finalize RPC to the remaining nodes ?



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r960285819


##########
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)
+                  }

Review Comment:
   > I have a confusion about the origin code. If there is small shuffle data, registerMergeResults will be false, so we don't wait for the merged statuses, but these merged statuses are still useful if they are available before the reduce tasks fetch them? This should often happen if the cluster is heavy.
   
   This is to balance the overhead of waiting for finalization to complete - so that really small stages dont spend most of their time waiting for finalization to complete (so that we minimize the overheads of finalization as the benefits at read wont be sufficient).
   You can tune `spark.shuffle.push.minCompletedPushRatio` and `spark.shuffle.push.minShuffleSizeToWait` as appropriate to your cluster env for modifying the behavior.
   
   > Another question, for your suggestion code, if all the finalize RPCs can be completed within shuffleMergeResultsTimeoutSec, does the cancelFinalizeShuffleMergeFutures task still need to wait for scheduling ?
   
   In the snippet I pasted above, we have two conditions:
   `timedOut || !registerMergeResults`
   
   If `timedOut == true`, not all `results` were obtained - now this would (almost always) mean the merger received finalization message but did not respond in time, or the finalize message never reached it (the issue we observed in this PR).
   
   If `registerMergeResults == false`, we have not had time to send any rpc (we just enqueued the tasks, set all `results` to true - so have not waited in the `Futures.allAsList`) - so we do need to wait for rpc's to complete.
   
   Note that if the task has already completed, cancelling futures is really cheap.
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r958010820


##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +285,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // Send finalize RPC tasks to merger ESS, one thread per RPC and will be cancelled after
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. Please close the opened files in the merger ESS
+  // if finalize RPC is not received due to network issues.
+  private val shuffleSendFinalizeRPCExecutor: ExecutorService =
+  ThreadUtils.newDaemonFixedThreadPool(
+    shuffleSendFinalizeRPCThreads, "send-shuffle-merge-finalize-rpc")

Review Comment:
   Fix indentation



##########
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)
+                  }

Review Comment:
   `onShuffleMergeSuccess` and `onShuffleMergeFailure` are no-op's when `registerMergeResults == false` 
   
   I would suggest to keep the code pretty much as-is, except with the introduction of calling `shuffleClient.finalizeShuffleMerge` in  `shuffleSendFinalizeRPCExecutor`
   
   
   Something like:
   
   ```
   val scheduledFutures = {
     if (!registerMergeResults) {
       results.foreach(_.set(true))
       stage.shuffleDep.getMergerLocs.map {
         case shuffleServiceLoc =>
           shuffleSendFinalizeRPCExecutor.submit(new Runnable() {
             override def run(): Unit = {
                 // earlier code inside case shuffleServiceLoc =>
                 ...
             }
           });
       }
     } else {
       stage.shuffleDep.getMergerLocs.zipWithIndex.map {
         case (shuffleServiceLoc, index) =>
           shuffleSendFinalizeRPCExecutor.submit(new Runnable() {
             override def run(): Unit = {
               // earlier code inside case (shuffleServiceLoc, index) =>
               ...
             }
           });
     }
   }
   
   
   var timedOut = false
   try {
     Futures.allAsList(results: _*).get(shuffleMergeResultsTimeoutSec, TimeUnit.SECONDS)
   } catch {
     timedOut = true
     // earlier code ...
   } finally {
     if (timedOut || !registerMergeResults) {
       cancelFinalizeShuffleMergeFutures(scheduledFutures, 
         if (timedOut) 0 else shuffleMergeResultsTimeoutSec)
     }
     eventProcessLoop.post(ShuffleMergeFinalized(stage))
   }
   
   private def cancelFinalizeShuffleMergeFutures(futures: Seq[Future[_]], delayInSecs: Int): Unit = {
     def cancelFutures(): Unit = futures.foreach (_..cancel(true))
     if (delayInSecs > 0) {
       shuffleMergeFinalizeScheduler.schedule(new Runnable {
         override def run(): Unit = {
             cancelFutures()
         }
       }, delayInSecs, TimeUnit.SECONDS));
     } else {
       cancelFutures()
     }
   }
   
   
   ```



##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2311,6 +2311,15 @@ package object config {
       .intConf
       .createWithDefault(3)

Review Comment:
   Perhaps bump this up from 3 as well to 6 or 8 (since now future cancel happens in that threadpool - assuming my proposed change below is fine) - with `PUSH_BASED_SHUFFLE_SEND_FINALIZE_RPC_THREADS` correspondingly increased.



##########
core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala:
##########
@@ -282,13 +285,19 @@ private[spark] class DAGScheduler(
       None
     }
 
-  // Use multi-threaded scheduled executor. The merge finalization task could take some time,
-  // depending on the time to establish connections to mergers, and the time to get MergeStatuses
-  // from all the mergers.
+  // Use multi-threaded scheduled executor. The merge finalization task (per stage) takes up to
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT.
   private val shuffleMergeFinalizeScheduler =
     ThreadUtils.newDaemonThreadPoolScheduledExecutor("shuffle-merge-finalizer",
       shuffleMergeFinalizeNumThreads)
 
+  // Send finalize RPC tasks to merger ESS, one thread per RPC and will be cancelled after
+  // PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. Please close the opened files in the merger ESS
+  // if finalize RPC is not received due to network issues.
+  private val shuffleSendFinalizeRPCExecutor: ExecutorService =

Review Comment:
   `RPC` -> `Rpc`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973642320


##########
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:
   Resolving, as this is addressed



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973642086


##########
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:
   Resolving, as this was reverted.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973641582


##########
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:
   Resolving comment thread



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] mridulm commented on a diff in pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37533:
URL: https://github.com/apache/spark/pull/37533#discussion_r973641546


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2309,7 +2309,17 @@ package object config {
         " shuffle is enabled.")
       .version("3.3.0")
       .intConf
-      .createWithDefault(3)
+      .createWithDefault(8)
+
+  private[spark] val PUSH_SHUFFLE_FINALIZE_RPC_THREADS =
+    ConfigBuilder("spark.shuffle.push.sendFinalizeRPCThreads")
+      .doc("Number of threads used by the driver to send finalize shuffle RPC to mergers" +
+        " location and then get MergeStatus. The thread won't stop" +
+        " PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT. The merger ESS may open too many files" +
+        " if the finalize rpc is not received.")

Review Comment:
   This is not addressed @wankunde 



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] otterc commented on pull request #37533: [SPARK-40096]Fix finalize shuffle stage slow due to connection creation slow

Posted by GitBox <gi...@apache.org>.
otterc commented on PR #37533:
URL: https://github.com/apache/spark/pull/37533#issuecomment-1224253081

   So the issue is that the wait period timer doesn't take into account the time for connection creation which is a bug. However, in this PR you are adding another major change of excluding merger nodes based on this. I don't think we should combine that with the fix for this bug because if we decide to do that then we need to pay more careful consideration of how it interacts with exclusion of executor nodes, etc.


-- 
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-unsubscribe@spark.apache.org

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


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