Skip to content

Commit c42ef95

Browse files
susanxhuynhMarcelo Vanzin
authored andcommitted
[SPARK-21456][MESOS] Make the driver failover_timeout configurable
## What changes were proposed in this pull request? Current behavior: in Mesos cluster mode, the driver failover_timeout is set to zero. If the driver temporarily loses connectivity with the Mesos master, the framework will be torn down and all executors killed. Proposed change: make the failover_timeout configurable via a new option, spark.mesos.driver.failoverTimeout. The default value is still zero. Note: with non-zero failover_timeout, an explicit teardown is needed in some cases. This is captured in https://issues.apache.org/jira/browse/SPARK-21458 ## How was this patch tested? Added a unit test to make sure the config option is set while creating the scheduler driver. Ran an integration test with mesosphere/spark showing that with a non-zero failover_timeout the Spark job finishes after a driver is disconnected from the master. Author: Susan X. Huynh <[email protected]> Closes #18674 from susanxhuynh/sh-mesos-failover-timeout.
1 parent c972918 commit c42ef95

File tree

4 files changed

+57
-2
lines changed

4 files changed

+57
-2
lines changed

docs/running-on-mesos.md

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -545,6 +545,17 @@ See the [configuration page](configuration.html) for information on Spark config
545545
Fetcher Cache</a>
546546
</td>
547547
</tr>
548+
<tr>
549+
<td><code>spark.mesos.driver.failoverTimeout</code></td>
550+
<td><code>0.0</code></td>
551+
<td>
552+
The amount of time (in seconds) that the master will wait for the
553+
driver to reconnect, after being temporarily disconnected, before
554+
it tears down the driver framework by killing all its
555+
executors. The default value is zero, meaning no timeout: if the
556+
driver disconnects, the master immediately tears down the framework.
557+
</td>
558+
</tr>
548559
</table>
549560

550561
# Troubleshooting and Debugging

resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,9 +58,16 @@ package object config {
5858

5959
private [spark] val DRIVER_LABELS =
6060
ConfigBuilder("spark.mesos.driver.labels")
61-
.doc("Mesos labels to add to the driver. Labels are free-form key-value pairs. Key-value" +
61+
.doc("Mesos labels to add to the driver. Labels are free-form key-value pairs. Key-value " +
6262
"pairs should be separated by a colon, and commas used to list more than one." +
6363
"Ex. key:value,key2:value2")
6464
.stringConf
6565
.createOptional
66+
67+
private [spark] val DRIVER_FAILOVER_TIMEOUT =
68+
ConfigBuilder("spark.mesos.driver.failoverTimeout")
69+
.doc("Amount of time in seconds that the master will wait to hear from the driver, " +
70+
"during a temporary disconnection, before tearing down all the executors.")
71+
.doubleConf
72+
.createWithDefault(0.0)
6673
}

resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _}
2929
import org.apache.mesos.SchedulerDriver
3030

3131
import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState}
32+
import org.apache.spark.deploy.mesos.config._
3233
import org.apache.spark.internal.config
3334
import org.apache.spark.network.netty.SparkTransportConf
3435
import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
@@ -177,7 +178,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
177178
sc.conf,
178179
sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)),
179180
None,
180-
None,
181+
Some(sc.conf.get(DRIVER_FAILOVER_TIMEOUT)),
181182
sc.conf.getOption("spark.mesos.driver.frameworkId")
182183
)
183184

resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ import org.scalatest.mock.MockitoSugar
3333
import org.scalatest.BeforeAndAfter
3434

3535
import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite}
36+
import org.apache.spark.deploy.mesos.config._
3637
import org.apache.spark.internal.config._
3738
import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
3839
import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef}
@@ -369,6 +370,41 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite
369370
backend.start()
370371
}
371372

373+
test("failover timeout is set in created scheduler driver") {
374+
val failoverTimeoutIn = 3600.0
375+
initializeSparkConf(Map(DRIVER_FAILOVER_TIMEOUT.key -> failoverTimeoutIn.toString))
376+
sc = new SparkContext(sparkConf)
377+
378+
val taskScheduler = mock[TaskSchedulerImpl]
379+
when(taskScheduler.sc).thenReturn(sc)
380+
381+
val driver = mock[SchedulerDriver]
382+
when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING)
383+
384+
val securityManager = mock[SecurityManager]
385+
386+
val backend = new MesosCoarseGrainedSchedulerBackend(
387+
taskScheduler, sc, "master", securityManager) {
388+
override protected def createSchedulerDriver(
389+
masterUrl: String,
390+
scheduler: Scheduler,
391+
sparkUser: String,
392+
appName: String,
393+
conf: SparkConf,
394+
webuiUrl: Option[String] = None,
395+
checkpoint: Option[Boolean] = None,
396+
failoverTimeout: Option[Double] = None,
397+
frameworkId: Option[String] = None): SchedulerDriver = {
398+
markRegistered()
399+
assert(failoverTimeout.isDefined)
400+
assert(failoverTimeout.get.equals(failoverTimeoutIn))
401+
driver
402+
}
403+
}
404+
405+
backend.start()
406+
}
407+
372408
test("honors unset spark.mesos.containerizer") {
373409
setBackend(Map("spark.mesos.executor.docker.image" -> "test"))
374410

0 commit comments

Comments
 (0)