Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskStat
import org.apache.spark.deploy.mesos.config._
import org.apache.spark.deploy.security.HadoopDelegationTokenManager
import org.apache.spark.internal.config
import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle}
import org.apache.spark.network.netty.SparkTransportConf
import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient
import org.apache.spark.rpc.RpcEndpointAddress
Expand Down Expand Up @@ -89,6 +90,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
// Synchronization protected by stateLock
private[this] var stopCalled: Boolean = false

private val launcherBackend = new LauncherBackend() {
override protected def onStopRequest(): Unit = {
stopSchedulerBackend()
setState(SparkAppHandle.State.KILLED)
}
}

// If shuffle service is enabled, the Spark driver will register with the shuffle service.
// This is for cleaning up shuffle files reliably.
private val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
Expand Down Expand Up @@ -174,6 +182,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
override def start() {
super.start()

if (sc.deployMode == "client") {
launcherBackend.connect()
}
val startedBefore = IdHelper.startedBefore.getAndSet(true)

val suffix = if (startedBefore) {
Expand All @@ -194,6 +205,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
sc.conf.getOption("spark.mesos.driver.frameworkId").map(_ + suffix)
)

launcherBackend.setState(SparkAppHandle.State.SUBMITTED)
startScheduler(driver)
}

Expand Down Expand Up @@ -287,15 +299,21 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
this.mesosExternalShuffleClient.foreach(_.init(appId))
this.schedulerDriver = driver
markRegistered()
launcherBackend.setAppId(appId)
launcherBackend.setState(SparkAppHandle.State.RUNNING)
}

override def sufficientResourcesRegistered(): Boolean = {
totalCoreCount.get >= maxCoresOption.getOrElse(0) * minRegisteredRatio
}

override def disconnected(d: org.apache.mesos.SchedulerDriver) {}
override def disconnected(d: org.apache.mesos.SchedulerDriver) {
launcherBackend.setState(SparkAppHandle.State.SUBMITTED)
}

override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {}
override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {
launcherBackend.setState(SparkAppHandle.State.RUNNING)
}

/**
* Method called by Mesos to offer resources on slaves. We respond by launching an executor,
Expand Down Expand Up @@ -569,6 +587,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
}

override def stop() {
stopSchedulerBackend()
launcherBackend.setState(SparkAppHandle.State.FINISHED)
launcherBackend.close()
}

private def stopSchedulerBackend() {
// Make sure we're not launching tasks during shutdown
stateLock.synchronized {
if (stopCalled) {
Expand Down