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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]