Skip to content

Commit

Permalink
[SPARK-5088] Use spark-class for running executors directly
Browse files Browse the repository at this point in the history
- Fixed code if spark.executor.uri doesn't have any value
- Added test cases
  • Loading branch information
jongyoul committed Jan 19, 2015
1 parent ff57bda commit 613cb47
Show file tree
Hide file tree
Showing 2 changed files with 28 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -126,9 +126,8 @@ private[spark] class MesosSchedulerBackend(
val uri = sc.conf.get("spark.executor.uri", null)
val executorBackendName = classOf[MesosExecutorBackend].getName
if (uri == null) {
val executorPath = new File(executorSparkHome, s"/bin/spark-class $executorBackendName")
.getCanonicalPath
command.setValue("%s %s".format(prefixEnv, executorPath))
val executorPath = new File(executorSparkHome, "/bin/spark-class").getCanonicalPath
command.setValue(s"$prefixEnv $executorPath $executorBackendName")
} else {
// Grab everything to the first '.'. We'll use that and '*' to
// glob the directory "correctly".
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.scheduler.mesos

import org.apache.spark.executor.MesosExecutorBackend
import org.scalatest.FunSuite
import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext}
import org.apache.spark.scheduler.{SparkListenerExecutorAdded, LiveListenerBus,
Expand All @@ -37,6 +38,31 @@ import scala.collection.mutable.ArrayBuffer

class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar {

test("check spark-class location correctly") {
val conf = new SparkConf
conf.set("spark.mesos.executor.home" , "/mesos-home")

val sc = EasyMock.createMock(classOf[SparkContext])
EasyMock.expect(sc.getSparkHome()).andReturn(Option("/spark-home")).anyTimes()
EasyMock.expect(sc.conf).andReturn(conf).anyTimes()
EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes()
EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes()
EasyMock.replay(sc)
val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl])
EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes()
EasyMock.replay(taskScheduler)
val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master")

// uri is null.
val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id")
assert(executorInfo.getCommand.getValue === s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}")

// uri exists.
conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz")
val executorInfo1 = mesosSchedulerBackend.createExecutorInfo("test-id")
assert(executorInfo1.getCommand.getValue === s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}")
}

test("mesos resource offers result in launching tasks") {
def createOffer(id: Int, mem: Int, cpu: Int) = {
val builder = Offer.newBuilder()
Expand Down

0 comments on commit 613cb47

Please sign in to comment.