Skip to content

Commit 0fa7780

Browse files
Luc Bourliertnachen
authored andcommitted
Launch task through the mesos scheduler
1 parent 5b7a12b commit 0fa7780

File tree

5 files changed

+67
-35
lines changed

5 files changed

+67
-35
lines changed

core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,8 @@ object MesosClusterDispatcher {
6666
def main(args: Array[String]) {
6767
val conf = new SparkConf
6868
val dispatcherArgs = new ClusterDispatcherArguments(args, conf)
69+
conf.setMaster(dispatcherArgs.masterUrl)
70+
conf.setAppName("Mesos Cluster Dispatcher")
6971
val scheduler = new MesosClusterScheduler(conf)
7072
scheduler.start()
7173
new MesosClusterDispatcher(
@@ -74,6 +76,10 @@ object MesosClusterDispatcher {
7476
dispatcherArgs.webUiPort,
7577
conf,
7678
scheduler).start()
79+
this.synchronized {
80+
// block indefinitely
81+
this.wait() // TODO: bad
82+
}
7783
}
7884

7985
class ClusterDispatcherArguments(args: Array[String], conf: SparkConf) {
@@ -103,7 +109,7 @@ object MesosClusterDispatcher {
103109
System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)")
104110
System.exit(1)
105111
}
106-
masterUrl = value
112+
masterUrl = value.stripPrefix("mesos://")
107113
parse(tail)
108114

109115
case ("--help") :: tail =>

core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverOutputPage.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -55,15 +55,15 @@ class DriverOutputPage(parent: MesosClusterUI) extends WebUIPage("") {
5555
<tr>
5656
<td>{submission.submissionId}</td>
5757
<td>{submission.submitDate}</td>
58-
<td>{submission.desc.command.mainClass}</td>
58+
<td>{submission.req.desc.command.mainClass}</td>
5959
</tr>
6060
}
6161

6262
def driverRow(state: ClusterTaskState): Seq[Node] = {
6363
<tr>
6464
<td>{state.submission.submissionId}</td>
6565
<td>{state.submission.submitDate}</td>
66-
<td>{state.submission.desc.command.mainClass}</td>
66+
<td>{state.submission.req.desc.command.mainClass}</td>
6767
<td>{state.startDate}</td>
6868
<td>{state.slaveId}</td>
6969
<td>{state.taskId}</td>

core/src/main/scala/org/apache/spark/deploy/rest/MesosRestServer.scala

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,10 +23,10 @@ import javax.servlet.http.HttpServletResponse
2323
import org.apache.spark.deploy.DriverDescription
2424
import org.apache.spark.deploy.ClientArguments._
2525
import org.apache.spark.deploy.Command
26-
2726
import org.apache.spark.{SparkConf, SPARK_VERSION => sparkVersion}
2827
import org.apache.spark.util.Utils
2928
import org.apache.spark.scheduler.cluster.mesos.ClusterScheduler
29+
import org.apache.spark.scheduler.cluster.mesos.DriverRequest
3030

3131
/**
3232
* A server that responds to requests submitted by the [[RestClient]].
@@ -74,7 +74,7 @@ class MesosSubmitRequestServlet(
7474
* This does not currently consider fields used by python applications since python
7575
* is not supported in mesos cluster mode yet.
7676
*/
77-
private def buildDriverDescription(request: CreateSubmissionRequest): DriverDescription = {
77+
private def buildDriverRequest(request: CreateSubmissionRequest): DriverRequest = {
7878
// Required fields, including the main class because python is not yet supported
7979
val appResource = Option(request.appResource).getOrElse {
8080
throw new SubmitRestMissingFieldException("Application jar is missing.")
@@ -109,8 +109,9 @@ class MesosSubmitRequestServlet(
109109
val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY)
110110
val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES)
111111

112-
new DriverDescription(
113-
appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command)
112+
DriverRequest(new DriverDescription(
113+
appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command),
114+
conf)
114115
}
115116

116117
protected override def handleSubmit(
@@ -119,7 +120,7 @@ class MesosSubmitRequestServlet(
119120
responseServlet: HttpServletResponse): SubmitRestProtocolResponse = {
120121
requestMessage match {
121122
case submitRequest: CreateSubmissionRequest =>
122-
val driverDescription = buildDriverDescription(submitRequest)
123+
val driverDescription = buildDriverRequest(submitRequest)
123124
val response = scheduler.submitDriver(driverDescription)
124125
val submitResponse = new CreateSubmissionResponse
125126
submitResponse.serverSparkVersion = sparkVersion

core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala

Lines changed: 48 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -18,30 +18,28 @@
1818
package org.apache.spark.scheduler.cluster.mesos
1919

2020
import java.text.SimpleDateFormat
21-
2221
import java.util.concurrent.atomic.AtomicLong
2322
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
2423
import java.util.{List => JList}
2524
import java.util.{Collections, Date}
26-
2725
import org.apache.mesos.{SchedulerDriver, Scheduler}
2826
import org.apache.mesos.Protos._
29-
3027
import org.apache.spark.deploy.DriverDescription
3128
import org.apache.spark.deploy.master.DriverState
3229
import org.apache.spark.deploy.master.DriverState.DriverState
33-
3430
import org.apache.spark.SparkConf
3531
import org.apache.spark.util.Utils
36-
3732
import scala.collection.mutable
3833
import scala.collection.JavaConversions._
3934
import org.apache.mesos.Protos.Environment.Variable
35+
import org.apache.spark.SparkException
36+
import java.io.File
4037

38+
case class DriverRequest(desc: DriverDescription, conf: SparkConf)
4139

4240
private[spark] class DriverSubmission(
4341
val submissionId: String,
44-
val desc: DriverDescription,
42+
val req: DriverRequest,
4543
val submitDate: Date) {
4644

4745
def canEqual(other: Any): Boolean = other.isInstanceOf[DriverSubmission]
@@ -76,7 +74,7 @@ private[spark] case class ClusterSchedulerState(
7674
finishedDrivers: Iterable[ClusterTaskState])
7775

7876
private[spark] trait ClusterScheduler {
79-
def submitDriver(desc: DriverDescription): SubmitResponse
77+
def submitDriver(desc: DriverRequest): SubmitResponse
8078
def killDriver(submissionId: String): KillResponse
8179
def getStatus(submissionId: String): StatusResponse
8280
def getState(): ClusterSchedulerState
@@ -88,7 +86,6 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
8886
val master = conf.get("spark.master")
8987
val appName = conf.get("spark.app.name")
9088
val capacity = conf.getInt("spark.mesos.driver.capacity", 200)
91-
val executorUri = conf.getOption("spark.executor.uri")
9289
val stateLock = new Object
9390
val launchedDrivers = new mutable.HashMap[String, ClusterTaskState]()
9491

@@ -105,10 +102,10 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
105102
createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet())
106103
}
107104

108-
def submitDriver(desc: DriverDescription): SubmitResponse = {
105+
def submitDriver(req: DriverRequest): SubmitResponse = {
109106
val submitDate: Date = new Date()
110107
val submissionId: String = newDriverId(submitDate)
111-
val submission = new DriverSubmission(submissionId, desc, submitDate)
108+
val submission = new DriverSubmission(submissionId, req, submitDate)
112109
if (queue.offer(submission)) {
113110
SubmitResponse(submissionId, true, None)
114111
} else {
@@ -147,9 +144,16 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
147144
markRegistered()
148145
}
149146

150-
private def buildCommand(desc: DriverDescription): CommandInfo = {
147+
private def buildCommand(req: DriverRequest): CommandInfo = {
148+
149+
val desc = req.desc
150+
151+
val cleanedJarUrl = desc.jarUrl.stripPrefix("file:")
152+
153+
logInfo(s"jarUrl: $cleanedJarUrl")
154+
151155
val builder = CommandInfo.newBuilder()
152-
.addUris(CommandInfo.URI.newBuilder().setValue(desc.jarUrl).build())
156+
.addUris(CommandInfo.URI.newBuilder().setValue(cleanedJarUrl).build())
153157

154158
val entries =
155159
(conf.getOption("spark.executor.extraLibraryPath").toList ++ desc.command.libraryPathEntries)
@@ -160,11 +164,7 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
160164
""
161165
}
162166

163-
val stringBuilder = new StringBuilder
164-
stringBuilder
165-
.append(desc.command.mainClass)
166-
.append(" ")
167-
.append(desc.command.arguments.mkString("\"", "\"", "\""))
167+
// TODO: add support for more spark-submit parameters
168168

169169
val envBuilder = Environment.newBuilder()
170170
desc.command.environment.foreach {
@@ -175,14 +175,35 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
175175

176176
builder.setEnvironment(envBuilder.build())
177177

178+
val executorUri = req.conf.getOption("spark.executor.uri")
178179
if (executorUri.isDefined) {
179180
builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build())
181+
180182
val basename = executorUri.get.split('/').last.split('.').head
183+
val cmd =
184+
Seq("bin/spark-submit",
185+
"--class", desc.command.mainClass,
186+
"--master", s"mesos://${conf.get("spark.master")}",
187+
s"../${desc.jarUrl.split("/").last}")
188+
.mkString(" ")
189+
181190
builder.setValue(
182-
s"cd $basename*; $prefixEnv ${stringBuilder.toString()}")
191+
s"cd $basename*; $prefixEnv $cmd")
183192
} else {
193+
val executorSparkHome = req.conf.getOption("spark.mesos.executor.home")
194+
.getOrElse {
195+
throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!")
196+
}
197+
198+
val cmd =
199+
Seq(new File(executorSparkHome, "./bin/spark-submit"),
200+
"--class", desc.command.mainClass,
201+
"--master", s"mesos://${conf.get("spark.master")}",
202+
desc.jarUrl.split("/").last)
203+
.mkString(" ")
204+
184205
builder.setValue(
185-
s"$prefixEnv ${stringBuilder.toString()}")
206+
s"$prefixEnv $cmd")
186207
}
187208

188209
builder.build
@@ -203,8 +224,8 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
203224

204225
var remainingOffers = offers
205226

206-
val driverCpu = submission.desc.cores
207-
val driverMem = submission.desc.mem
227+
val driverCpu = submission.req.desc.cores
228+
val driverMem = submission.req.desc.mem
208229

209230
// Should use the passed in driver cpu and memory.
210231
val offerOption = offers.find { o =>
@@ -216,15 +237,18 @@ private[spark] class MesosClusterScheduler(conf: SparkConf)
216237
val taskId = TaskID.newBuilder().setValue(submission.submissionId).build()
217238

218239
val cpuResource = Resource.newBuilder()
219-
.setName("cpus").setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build()
240+
.setName("cpus").setType(Value.Type.SCALAR)
241+
.setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build()
220242

221243
val memResource = Resource.newBuilder()
222-
.setName("mem").setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build()
244+
.setName("mem").setType(Value.Type.SCALAR)
245+
.setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build()
223246

224-
val commandInfo = buildCommand(submission.desc)
247+
val commandInfo = buildCommand(submission.req)
225248

226249
val taskInfo = TaskInfo.newBuilder()
227250
.setTaskId(taskId)
251+
.setName(s"driver for ${submission.req.desc.command.mainClass}")
228252
.setSlaveId(offer.getSlaveId)
229253
.setCommand(commandInfo)
230254
.addResources(cpuResource)

core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import org.scalatest.FunSuite
2121
import org.apache.spark.{SparkConf, LocalSparkContext}
2222
import org.scalatest.mock.MockitoSugar
2323
import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler
24+
import org.apache.spark.scheduler.cluster.mesos.DriverRequest
2425
import org.apache.spark.deploy.{Command, DriverDescription}
2526

2627
class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar {
@@ -35,11 +36,11 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo
3536
conf.setAppName("spark mesos")
3637
val scheduler = new MesosClusterScheduler(conf)
3738
val response =
38-
scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand))
39+
scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf))
3940
assert(response.success)
4041

4142
val response2 =
42-
scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand))
43+
scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf))
4344
assert(response2.success)
4445

4546
val state = scheduler.getState()
@@ -53,7 +54,7 @@ class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with Mo
5354
conf.setAppName("spark mesos")
5455
val scheduler = new MesosClusterScheduler(conf)
5556
val response =
56-
scheduler.submitDriver(new DriverDescription("jar", 1000, 1, true, createCommand))
57+
scheduler.submitDriver(DriverRequest(new DriverDescription("jar", 1000, 1, true, createCommand), new SparkConf))
5758
assert(response.success)
5859

5960
val killResponse = scheduler.killDriver(response.id)

0 commit comments

Comments
 (0)