From 6a259fcb951cba0b305a57dfc3168e69cf1658ba Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Tue, 10 Mar 2015 13:20:58 -0700 Subject: [PATCH 01/18] Add dynamic allocation to the Spark coarse-grained scheduler. Based on #3861. --- .../mesos/CoarseMesosSchedulerBackend.scala | 91 ++++++++++++++++--- 1 file changed, 77 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 90dfe14352a8e..16d8a414da0b5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -20,6 +20,9 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File import java.util.{List => JList} import java.util.Collections +import java.util.concurrent.locks.ReentrantLock + +import com.google.common.collect.HashBiMap import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} @@ -69,9 +72,17 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveIdsWithExecutors = new HashSet[String] - val taskIdToSlaveId = new HashMap[Int, String] - val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed + val taskIdToSlaveId: HashBiMap[Int, String] = HashBiMap.create[Int, String] + val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int] // How many times tasks on each slave failed + + + // the total number of executors we aim to have + private var executorLimit: Option[Int] = None + private val pendingRemovedSlaveIds = new HashSet[String] + // private lock object protecting mutable state above. Using the intrinsic lock + // may lead to deadlocks since the superclass might also try to lock + private val stateLock = new ReentrantLock val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) @@ -88,17 +99,16 @@ private[spark] class CoarseMesosSchedulerBackend( override def start() { super.start() - synchronized { + stateLock.synchronized { new Thread("CoarseMesosSchedulerBackend driver") { setDaemon(true) override def run() { val scheduler = CoarseMesosSchedulerBackend.this val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { { + try { val ret = driver.run() logInfo("driver.run() returned with code " + ret) - } } catch { case e: Exception => logError("driver.run() failed", e) } @@ -206,7 +216,7 @@ private[spark] class CoarseMesosSchedulerBackend( * unless we've already launched more than we wanted to. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - synchronized { + stateLock.synchronized { val filters = Filters.newBuilder().setRefuseSeconds(-1).build() for (offer <- offers) { @@ -274,7 +284,7 @@ private[spark] class CoarseMesosSchedulerBackend( val taskId = status.getTaskId.getValue.toInt val state = status.getState logInfo("Mesos task " + taskId + " is now " + state) - synchronized { + stateLock.synchronized { if (isFinished(state)) { val slaveId = taskIdToSlaveId(taskId) slaveIdsWithExecutors -= slaveId @@ -292,6 +302,7 @@ private[spark] class CoarseMesosSchedulerBackend( "is Spark installed on it?") } } + executorTerminated(d, slaveId, s"Executor finished with state $state") driver.reviveOffers() // In case we'd rejected everything before but have now lost a node } } @@ -311,17 +322,33 @@ private[spark] class CoarseMesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { - logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - if (slaveIdsWithExecutors.contains(slaveId.getValue)) { - // Note that the slave ID corresponds to the executor ID on that slave - slaveIdsWithExecutors -= slaveId.getValue - removeExecutor(slaveId.getValue, "Mesos slave lost") + /** Called when a slave is lost or a Mesos task finished. Update local view on + * what tasks are running and remove the terminated slave from the list of pending + * slave IDs that we might have asked to be killed. It also notifies the driver + * that an executor was removed. + */ + private def executorTerminated(d: SchedulerDriver, slaveId: String, reason: String) { + stateLock.synchronized { + if (slaveIdsWithExecutors.contains(slaveId)) { + val slaveIdToTaskId = taskIdToSlaveId.inverse() + if (slaveIdToTaskId.contains(slaveId)) { + val taskId: Int = slaveIdToTaskId.get(slaveId) + taskIdToSlaveId.remove(taskId) + removeExecutor(sparkExecutorId(slaveId, taskId.toString), reason) + } + pendingRemovedSlaveIds -= slaveId + slaveIdsWithExecutors -= slaveId } } } + private def sparkExecutorId(slaveId: String, taskId: String) = "%s/%s".format(slaveId, taskId) + + override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { + logInfo("Mesos slave lost: " + slaveId.getValue) + executorTerminated(d, slaveId.getValue, "Mesos slave lost: " + slaveId.getValue) + } + override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int) { logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue)) slaveLost(d, s) @@ -333,4 +360,40 @@ private[spark] class CoarseMesosSchedulerBackend( super.applicationId } + override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + // We don't truly know if we can fulfill the full amount of executors + // since at coarse grain it depends on the amount of slaves available. + logInfo("Capping the total amount of executors to " + requestedTotal) + executorLimit = Option(requestedTotal) + true + } + + override def doKillExecutors(executorIds: Seq[String]): Boolean = { + if (driver == null) { + logWarning("Asked to kill executors before the executor was started.") + return false + } + + val slaveIdToTaskId = taskIdToSlaveId.inverse() + for (executorId <- executorIds) { + val slaveId = executorId.split("/")(0) + if (slaveIdToTaskId.contains(slaveId)) { + driver.killTask( + TaskID.newBuilder().setValue(slaveIdToTaskId.get(slaveId).toString).build) + pendingRemovedSlaveIds += slaveId + } else { + logWarning("Unable to find executor Id '" + executorId + "' in Mesos scheduler") + } + } + + assert(pendingRemovedSlaveIds.size <= taskIdToSlaveId.size) + + // We cannot simply decrement from the existing executor limit as we may not able to + // launch as much executors as the limit. But we assume if we are notified to kill + // executors, that means the scheduler wants to set the limit that is less than + // the amount of the executors that has been launched. Therefore, we take the existing + // amount of executors launched and deduct the executors killed as the new limit. + executorLimit = Option(taskIdToSlaveId.size - pendingRemovedSlaveIds.size) + true + } } From 01709d5c55508b9e0fb46202b6325244499dc0a6 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Tue, 10 Mar 2015 17:08:33 -0700 Subject: [PATCH 02/18] First unit test for CoarseMesosSchedulerBackend. --- .../mesos/CoarseMesosSchedulerBackend.scala | 36 +-- .../CoarseMesosSchedulerBackendSuite.scala | 206 ++++++++++++++++++ 2 files changed, 228 insertions(+), 14 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 16d8a414da0b5..bcf6349e31ce6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -77,8 +77,12 @@ private[spark] class CoarseMesosSchedulerBackend( // the total number of executors we aim to have - private var executorLimit: Option[Int] = None - private val pendingRemovedSlaveIds = new HashSet[String] + private[mesos] var executorLimitOption: Option[Int] = None + + /** Return the current executor limit, which may be [[Int.MaxValue]] before properly initialized. */ + def executorLimit = executorLimitOption.getOrElse(Int.MaxValue) + + private[mesos] val pendingRemovedSlaveIds = new HashSet[String] // private lock object protecting mutable state above. Using the intrinsic lock // may lead to deadlocks since the superclass might also try to lock @@ -153,12 +157,6 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(sc.env.actorSystem), - SparkEnv.driverActorSystemName, - conf.get("spark.driver.host"), - conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) val uri = conf.get("spark.executor.uri", null) if (uri == null) { @@ -166,7 +164,7 @@ private[spark] class CoarseMesosSchedulerBackend( command.setValue( "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" .format(prefixEnv, runScript) + - s" --driver-url $driverUrl" + + s" --driver-url $driverURL" + s" --executor-id ${offer.getSlaveId.getValue}" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + @@ -178,7 +176,7 @@ private[spark] class CoarseMesosSchedulerBackend( command.setValue( s"cd $basename*; $prefixEnv " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + - s" --driver-url $driverUrl" + + s" --driver-url $driverURL" + s" --executor-id ${offer.getSlaveId.getValue}" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + @@ -188,6 +186,15 @@ private[spark] class CoarseMesosSchedulerBackend( command.build() } + protected def driverURL: String = { + AkkaUtils.address( + AkkaUtils.protocol(sc.env.actorSystem), + SparkEnv.driverActorSystemName, + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), + CoarseGrainedSchedulerBackend.ACTOR_NAME) + } + override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { @@ -220,10 +227,11 @@ private[spark] class CoarseMesosSchedulerBackend( val filters = Filters.newBuilder().setRefuseSeconds(-1).build() for (offer <- offers) { - val slaveId = offer.getSlaveId.toString + val slaveId = offer.getSlaveId.getValue val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && + if (taskIdToSlaveId.size < executorLimitOption.getOrElse(Int.MaxValue) && + totalCoresAcquired < maxCores && mem >= MemoryUtils.calculateTotalMemory(sc) && cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && @@ -364,7 +372,7 @@ private[spark] class CoarseMesosSchedulerBackend( // We don't truly know if we can fulfill the full amount of executors // since at coarse grain it depends on the amount of slaves available. logInfo("Capping the total amount of executors to " + requestedTotal) - executorLimit = Option(requestedTotal) + executorLimitOption = Option(requestedTotal) true } @@ -393,7 +401,7 @@ private[spark] class CoarseMesosSchedulerBackend( // executors, that means the scheduler wants to set the limit that is less than // the amount of the executors that has been launched. Therefore, we take the existing // amount of executors launched and deduct the executors killed as the new limit. - executorLimit = Option(taskIdToSlaveId.size - pendingRemovedSlaveIds.size) + executorLimitOption = Option(taskIdToSlaveId.size - pendingRemovedSlaveIds.size) true } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..eca4e315ac9ae --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.mesos + +import java.util +import java.util.Collections + +import akka.actor.ActorSystem +import com.typesafe.config.Config +import org.apache.mesos.Protos.Value.Scalar +import org.apache.mesos.Protos._ +import org.apache.mesos.SchedulerDriver +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MemoryUtils} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkEnv, SparkContext} +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.mockito.{ArgumentCaptor, Matchers} +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + +import scala.collection.mutable + +class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { + + def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { + val builder = Offer.newBuilder() + builder.addResourcesBuilder() + .setName("mem") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(mem)) + builder.addResourcesBuilder() + .setName("cpus") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(cpu)) + builder.setId(OfferID.newBuilder().setValue(offerId).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host${slaveId}").build() + } + + test("mesos supports killing and limiting executors") { + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] + val se = mock[SparkEnv] + val sparkConf = new SparkConf + val sc = mock[SparkContext] + when(sc.executorMemory).thenReturn(100) + when(sc.getSparkHome()).thenReturn(Option("/path")) + val emptyHashMap = new mutable.HashMap[String,String] + when(sc.executorEnvs).thenReturn(emptyHashMap) + when(sc.conf).thenReturn(sparkConf) + when(sc.env).thenReturn(se) + when(taskScheduler.sc).thenReturn(sc) + + sparkConf.set("spark.driver.host", "driverHost") + sparkConf.set("spark.driver.port", "1234") + + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + val minCpu = 4 + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) + + val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) + when( + driver.launchTasks( + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + capture.capture(), + any(classOf[Filters]) + ) + ).thenReturn(Status.valueOf(1)) + + val taskID0 = TaskID.newBuilder().setValue("0").build() + when( + driver.killTask(taskID0) + ).thenReturn(Status.valueOf(1)) + + when( + driver.declineOffer(OfferID.newBuilder().setValue("o2").build()) + ).thenReturn(Status.valueOf(1)) + + val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { + override def driverURL = "driverURL" + } + backend.driver = driver + backend.resourceOffers(driver, mesosOffers) + + // Calling doKillExecutors should invoke driver.killTask. + assert(backend.doKillExecutors(Seq("s1/0"))) + verify(driver, times(1)).killTask(taskID0) + assert(backend.executorLimit === 0) + + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) + backend.resourceOffers(driver, mesosOffers2) + + // Verify we didn't launch any new executor + assert(backend.slaveIdsWithExecutors.size === 1) + + when( + driver.launchTasks( + Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), + capture.capture(), + any(classOf[Filters]) + ) + ).thenReturn(Status.valueOf(1)) + + backend.doRequestTotalExecutors(2) + backend.resourceOffers(driver, mesosOffers2) + + assert(backend.slaveIdsWithExecutors.size === 2) + backend.slaveLost(driver, SlaveID.newBuilder().setValue("s1").build()) + assert(backend.slaveIdsWithExecutors.size === 1) + } + + // test("mesos supports killing and relaunching tasks with executors") { + // val driver = mock[SchedulerDriver] + // val taskScheduler = mock[TaskSchedulerImpl] + + // val se = mock[SparkEnv] + // val actorSystem = mock[ActorSystem] + // val sparkConf = new SparkConf + // when(se.actorSystem).thenReturn(actorSystem) + // EasyMock.replay(se) + // val sc = mock[SparkContext] + // when(sc.executorMemory).thenReturn(100).anyTimes() + // when(sc.getSparkHome()).thenReturn(Option("/path")).anyTimes() + // when(sc.executorEnvs).thenReturn(new mutable.HashMap).anyTimes() + // when(sc.conf).thenReturn(sparkConf).anyTimes() + // when(sc.env).thenReturn(se) + // EasyMock.replay(sc) + + // when(taskScheduler.sc).thenReturn(sc) + // EasyMock.replay(taskScheduler) + + // // Enable shuffle service so it will require extra resources + // sparkConf.set("spark.shuffle.service.enabled", "true") + // sparkConf.set("spark.driver.host", "driverHost") + // sparkConf.set("spark.driver.port", "1234") + + // val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 + // val minCpu = 4 + + // val mesosOffers = new java.util.ArrayList[Offer] + // mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) + + // when( + // driver.launchTasks( + // EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), + // EasyMock.anyObject(), + // EasyMock.anyObject(classOf[Filters]) + // ) + // ).thenReturn(Status.valueOf(1)).once + + // val offer2 = createOffer("o2", "s1", minMem, 1); + + // when( + // driver.launchTasks( + // EasyMock.eq(Collections.singleton(offer2.getId)), + // EasyMock.anyObject(), + // EasyMock.anyObject(classOf[Filters]) + // ) + // ).thenReturn(Status.valueOf(1)).once + + // when(driver.reviveOffers()).thenReturn(Status.valueOf(1)).once + + // EasyMock.replay(driver) + + // val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") + // backend.driver = driver + // backend.resourceOffers(driver, mesosOffers) + + // // Simulate task killed, but executor is still running + // val status = TaskStatus.newBuilder() + // .setTaskId(TaskID.newBuilder().setValue("0").build()) + // .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) + // .setState(TaskState.TASK_KILLED) + // .build + + // backend.statusUpdate(driver, status) + // assert(backend.slaveStatuses("s1").taskRunning.equals(false)) + // assert(backend.slaveStatuses("s1").executorRunning.equals(true)) + + // mesosOffers.clear() + // mesosOffers.add(offer2) + // backend.resourceOffers(driver, mesosOffers) + // assert(backend.slaveStatuses("s1").taskRunning.equals(true)) + // assert(backend.slaveStatuses("s1").executorRunning.equals(true)) + + // EasyMock.verify(driver) + // } +} From 1f5c33904bf3d376e9515ce486fae920b7040966 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Tue, 10 Mar 2015 21:56:42 -0700 Subject: [PATCH 03/18] Properly decline offer. --- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index bcf6349e31ce6..d3fbda8231fb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -256,8 +256,7 @@ private[spark] class CoarseMesosSchedulerBackend( Collections.singleton(offer.getId), Collections.singletonList(task), filters) } else { // Filter it out - d.launchTasks( - Collections.singleton(offer.getId), Collections.emptyList[MesosTaskInfo](), filters) + driver.declineOffer(offer.getId) } } } From 0954a75eeba49320b45452baa2cdb2e2d52a1611 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Tue, 10 Mar 2015 21:57:02 -0700 Subject: [PATCH 04/18] Added a second test and properly verified the first one. --- .../CoarseMesosSchedulerBackendSuite.scala | 175 +++++++----------- 1 file changed, 71 insertions(+), 104 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala index eca4e315ac9ae..cd1e2d678d492 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -26,11 +26,11 @@ import org.apache.mesos.Protos.Value.Scalar import org.apache.mesos.Protos._ import org.apache.mesos.SchedulerDriver import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MemoryUtils} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkEnv, SparkContext} +import org.apache.spark.scheduler.cluster.mesos.{ CoarseMesosSchedulerBackend, MemoryUtils } +import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } import org.mockito.Matchers._ import org.mockito.Mockito._ -import org.mockito.{ArgumentCaptor, Matchers} +import org.mockito.{ ArgumentCaptor, Matchers } import org.scalatest.FunSuite import org.scalatest.mock.MockitoSugar @@ -60,7 +60,7 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) when(sc.getSparkHome()).thenReturn(Option("/path")) - val emptyHashMap = new mutable.HashMap[String,String] + val emptyHashMap = new mutable.HashMap[String, String] when(sc.executorEnvs).thenReturn(emptyHashMap) when(sc.conf).thenReturn(sparkConf) when(sc.env).thenReturn(se) @@ -75,29 +75,17 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w val mesosOffers = new java.util.ArrayList[Offer] mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) - val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) - when( - driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).thenReturn(Status.valueOf(1)) - val taskID0 = TaskID.newBuilder().setValue("0").build() - when( - driver.killTask(taskID0) - ).thenReturn(Status.valueOf(1)) - - when( - driver.declineOffer(OfferID.newBuilder().setValue("o2").build()) - ).thenReturn(Status.valueOf(1)) val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { override def driverURL = "driverURL" } backend.driver = driver backend.resourceOffers(driver, mesosOffers) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) // Calling doKillExecutors should invoke driver.killTask. assert(backend.doKillExecutors(Seq("s1/0"))) @@ -108,99 +96,78 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) backend.resourceOffers(driver, mesosOffers2) + verify(driver, times(1)) + .declineOffer(OfferID.newBuilder().setValue("o2").build()) + // Verify we didn't launch any new executor assert(backend.slaveIdsWithExecutors.size === 1) - when( - driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).thenReturn(Status.valueOf(1)) - backend.doRequestTotalExecutors(2) backend.resourceOffers(driver, mesosOffers2) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) assert(backend.slaveIdsWithExecutors.size === 2) backend.slaveLost(driver, SlaveID.newBuilder().setValue("s1").build()) assert(backend.slaveIdsWithExecutors.size === 1) } - // test("mesos supports killing and relaunching tasks with executors") { - // val driver = mock[SchedulerDriver] - // val taskScheduler = mock[TaskSchedulerImpl] - - // val se = mock[SparkEnv] - // val actorSystem = mock[ActorSystem] - // val sparkConf = new SparkConf - // when(se.actorSystem).thenReturn(actorSystem) - // EasyMock.replay(se) - // val sc = mock[SparkContext] - // when(sc.executorMemory).thenReturn(100).anyTimes() - // when(sc.getSparkHome()).thenReturn(Option("/path")).anyTimes() - // when(sc.executorEnvs).thenReturn(new mutable.HashMap).anyTimes() - // when(sc.conf).thenReturn(sparkConf).anyTimes() - // when(sc.env).thenReturn(se) - // EasyMock.replay(sc) - - // when(taskScheduler.sc).thenReturn(sc) - // EasyMock.replay(taskScheduler) - - // // Enable shuffle service so it will require extra resources - // sparkConf.set("spark.shuffle.service.enabled", "true") - // sparkConf.set("spark.driver.host", "driverHost") - // sparkConf.set("spark.driver.port", "1234") - - // val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 - // val minCpu = 4 - - // val mesosOffers = new java.util.ArrayList[Offer] - // mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) - - // when( - // driver.launchTasks( - // EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), - // EasyMock.anyObject(), - // EasyMock.anyObject(classOf[Filters]) - // ) - // ).thenReturn(Status.valueOf(1)).once - - // val offer2 = createOffer("o2", "s1", minMem, 1); - - // when( - // driver.launchTasks( - // EasyMock.eq(Collections.singleton(offer2.getId)), - // EasyMock.anyObject(), - // EasyMock.anyObject(classOf[Filters]) - // ) - // ).thenReturn(Status.valueOf(1)).once - - // when(driver.reviveOffers()).thenReturn(Status.valueOf(1)).once - - // EasyMock.replay(driver) - - // val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") - // backend.driver = driver - // backend.resourceOffers(driver, mesosOffers) - - // // Simulate task killed, but executor is still running - // val status = TaskStatus.newBuilder() - // .setTaskId(TaskID.newBuilder().setValue("0").build()) - // .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) - // .setState(TaskState.TASK_KILLED) - // .build - - // backend.statusUpdate(driver, status) - // assert(backend.slaveStatuses("s1").taskRunning.equals(false)) - // assert(backend.slaveStatuses("s1").executorRunning.equals(true)) - - // mesosOffers.clear() - // mesosOffers.add(offer2) - // backend.resourceOffers(driver, mesosOffers) - // assert(backend.slaveStatuses("s1").taskRunning.equals(true)) - // assert(backend.slaveStatuses("s1").executorRunning.equals(true)) - - // EasyMock.verify(driver) - // } + test("mesos supports killing and relaunching tasks with executors") { + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] + val se = mock[SparkEnv] + val sparkConf = new SparkConf + val sc = mock[SparkContext] + when(sc.executorMemory).thenReturn(100) + when(sc.getSparkHome()).thenReturn(Option("/path")) + val emptyHashMap = new mutable.HashMap[String, String] + when(sc.executorEnvs).thenReturn(emptyHashMap) + when(sc.conf).thenReturn(sparkConf) + when(sc.env).thenReturn(se) + when(taskScheduler.sc).thenReturn(sc) + + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 + val minCpu = 4 + + val mesosOffers = new java.util.ArrayList[Offer] + val offer1 = createOffer("o1", "s1", minMem, minCpu) + mesosOffers.add(offer1) + + val offer2 = createOffer("o2", "s1", minMem, 1); + + val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { + override val driverURL = "" + } + backend.driver = driver + backend.resourceOffers(driver, mesosOffers) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer1.getId)), + anyObject(), + anyObject[Filters]) + + // Simulate task killed, executor no longer running + val status = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue("0").build()) + .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) + .setState(TaskState.TASK_KILLED) + .build + + backend.statusUpdate(driver, status) + assert(!backend.slaveIdsWithExecutors.contains("s1")) + + mesosOffers.clear() + mesosOffers.add(offer2) + backend.resourceOffers(driver, mesosOffers) + assert(backend.slaveIdsWithExecutors.contains("s1")) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer2.getId)), + anyObject(), + anyObject[Filters]) + + verify(driver, times(1)).reviveOffers() + } } From 129dc3605ecfc6e59e66e1392dbf8e4110718b39 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Wed, 11 Mar 2015 10:18:46 -0700 Subject: [PATCH 05/18] Refactored duplicated code. --- .../mesos/CoarseMesosSchedulerBackend.scala | 6 ++-- .../CoarseMesosSchedulerBackendSuite.scala | 28 ++++++++----------- 2 files changed, 15 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d3fbda8231fb2..262c7ce0a82db 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -164,7 +164,7 @@ private[spark] class CoarseMesosSchedulerBackend( command.setValue( "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" .format(prefixEnv, runScript) + - s" --driver-url $driverURL" + + s" --driver-url $driverUrl" + s" --executor-id ${offer.getSlaveId.getValue}" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + @@ -176,7 +176,7 @@ private[spark] class CoarseMesosSchedulerBackend( command.setValue( s"cd $basename*; $prefixEnv " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + - s" --driver-url $driverURL" + + s" --driver-url $driverUrl" + s" --executor-id ${offer.getSlaveId.getValue}" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + @@ -186,7 +186,7 @@ private[spark] class CoarseMesosSchedulerBackend( command.build() } - protected def driverURL: String = { + protected def driverUrl: String = { AkkaUtils.address( AkkaUtils.protocol(sc.env.actorSystem), SparkEnv.driverActorSystemName, diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala index cd1e2d678d492..b3a24da68119d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -38,7 +38,7 @@ import scala.collection.mutable class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { - def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { + private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -52,11 +52,11 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host${slaveId}").build() } - test("mesos supports killing and limiting executors") { + + private def mockEnvironment(sparkConf: SparkConf): (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { val driver = mock[SchedulerDriver] val taskScheduler = mock[TaskSchedulerImpl] val se = mock[SparkEnv] - val sparkConf = new SparkConf val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) when(sc.getSparkHome()).thenReturn(Option("/path")) @@ -65,6 +65,12 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w when(sc.conf).thenReturn(sparkConf) when(sc.env).thenReturn(se) when(taskScheduler.sc).thenReturn(sc) + (sc, taskScheduler, driver) + } + + test("mesos supports killing and limiting executors") { + val sparkConf = new SparkConf + val (sc, taskScheduler, driver) = mockEnvironment(sparkConf) sparkConf.set("spark.driver.host", "driverHost") sparkConf.set("spark.driver.port", "1234") @@ -78,7 +84,7 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w val taskID0 = TaskID.newBuilder().setValue("0").build() val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { - override def driverURL = "driverURL" + override val driverUrl = "" } backend.driver = driver backend.resourceOffers(driver, mesosOffers) @@ -115,18 +121,8 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w } test("mesos supports killing and relaunching tasks with executors") { - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - val se = mock[SparkEnv] val sparkConf = new SparkConf - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) - when(sc.getSparkHome()).thenReturn(Option("/path")) - val emptyHashMap = new mutable.HashMap[String, String] - when(sc.executorEnvs).thenReturn(emptyHashMap) - when(sc.conf).thenReturn(sparkConf) - when(sc.env).thenReturn(se) - when(taskScheduler.sc).thenReturn(sc) + val (sc, taskScheduler, driver) = mockEnvironment(sparkConf) val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 val minCpu = 4 @@ -138,7 +134,7 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w val offer2 = createOffer("o2", "s1", minMem, 1); val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { - override val driverURL = "" + override protected val driverUrl: String = "" } backend.driver = driver backend.resourceOffers(driver, mesosOffers) From 5f03fea59b32a686b6dc5ed1107be882221d5cf4 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Wed, 11 Mar 2015 11:18:47 -0700 Subject: [PATCH 06/18] Fix lines longer than 100 characters. --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 262c7ce0a82db..dc7becffbed67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -73,13 +73,17 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveIdsWithExecutors = new HashSet[String] val taskIdToSlaveId: HashBiMap[Int, String] = HashBiMap.create[Int, String] - val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int] // How many times tasks on each slave failed + // How many times tasks on each slave failed + val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int] // the total number of executors we aim to have private[mesos] var executorLimitOption: Option[Int] = None - /** Return the current executor limit, which may be [[Int.MaxValue]] before properly initialized. */ + /** + * Return the current executor limit, which may be [[Int.MaxValue]] + * before properly initialized. + */ def executorLimit = executorLimitOption.getOrElse(Int.MaxValue) private[mesos] val pendingRemovedSlaveIds = new HashSet[String] From 1bf833534bd9f707ffd276aa8fcc4b3b80e8b859 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Tue, 24 Mar 2015 11:34:06 -0500 Subject: [PATCH 07/18] Renamed the *MesosSchedulerBackend types to more descriptive CoarseGrained* and FineGrained*. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +++--- .../org/apache/spark/executor/MesosExecutorBackend.scala | 2 +- ...end.scala => CoarseGrainedMesosSchedulerBackend.scala} | 8 ++++---- ...ckend.scala => FineGrainedMesosSchedulerBackend.scala} | 6 +++--- .../apache/spark/SparkContextSchedulerCreationSuite.scala | 8 ++++---- ...cala => CoarseGrainedMesosSchedulerBackendSuite.scala} | 8 ++++---- .../scheduler/mesos/MesosSchedulerBackendSuite.scala | 6 +++--- 7 files changed, 22 insertions(+), 22 deletions(-) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{CoarseMesosSchedulerBackend.scala => CoarseGrainedMesosSchedulerBackend.scala} (98%) rename core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/{MesosSchedulerBackend.scala => FineGrainedMesosSchedulerBackend.scala} (98%) rename core/src/test/scala/org/apache/spark/scheduler/mesos/{CoarseMesosSchedulerBackendSuite.scala => CoarseGrainedMesosSchedulerBackendSuite.scala} (93%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1a0bee4e3aea9..e2f5caf4e473b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -57,7 +57,7 @@ import org.apache.spark.rdd._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseGrainedMesosSchedulerBackend, FineGrainedMesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ import org.apache.spark.ui.{SparkUI, ConsoleProgressBar} @@ -2244,9 +2244,9 @@ object SparkContext extends Logging { val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false) val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, url) + new CoarseGrainedMesosSchedulerBackend(scheduler, sc, url) } else { - new MesosSchedulerBackend(scheduler, sc, url) + new FineGrainedMesosSchedulerBackend(scheduler, sc, url) } scheduler.initialize(backend) (backend, scheduler) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index cfd672e1d8a97..c4849be3d432a 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -54,7 +54,7 @@ private[spark] class MesosExecutorBackend frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend. + // Get num cores for this task from ExecutorInfo, created in FineGrainedMesosSchedulerBackend. val cpusPerTask = executorInfo.getResourcesList .find(_.getName == "cpus") .map(_.getScalar.getValue.toInt) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala index dc7becffbed67..1601ccc6086eb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala @@ -43,10 +43,10 @@ import org.apache.spark.util.{Utils, AkkaUtils} * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable * latency. * - * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to + * Unfortunately this has a bit of duplication from FineGrainedMesosSchedulerBackend, but it seems hard to * remove this. */ -private[spark] class CoarseMesosSchedulerBackend( +private[spark] class CoarseGrainedMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String) @@ -108,10 +108,10 @@ private[spark] class CoarseMesosSchedulerBackend( super.start() stateLock.synchronized { - new Thread("CoarseMesosSchedulerBackend driver") { + new Thread("CoarseGrainedMesosSchedulerBackend driver") { setDaemon(true) override def run() { - val scheduler = CoarseMesosSchedulerBackend.this + val scheduler = CoarseGrainedMesosSchedulerBackend.this val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala index cfb6592e14aa8..cb38415dce74e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks * from multiple apps can run on different cores) and in time (a core can switch ownership). */ -private[spark] class MesosSchedulerBackend( +private[spark] class FineGrainedMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String) @@ -74,10 +74,10 @@ private[spark] class MesosSchedulerBackend( synchronized { classLoader = Thread.currentThread.getContextClassLoader - new Thread("MesosSchedulerBackend driver") { + new Thread("FineGrainedMesosSchedulerBackend driver") { setDaemon(true) override def run() { - val scheduler = MesosSchedulerBackend.this + val scheduler = FineGrainedMesosSchedulerBackend.this val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index bbed8ddc6bafc..82eed7f3b4431 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseGrainedMesosSchedulerBackend, FineGrainedMesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite @@ -166,14 +166,14 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) + testMesos("mesos://localhost:1234", classOf[FineGrainedMesosSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) + testMesos("mesos://localhost:1234", classOf[CoarseGrainedMesosSchedulerBackend], coarse = true) } test("mesos with zookeeper") { - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) + testMesos("zk://localhost:1234,localhost:2345", classOf[FineGrainedMesosSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala similarity index 93% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala index b3a24da68119d..aca362c3b9270 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala @@ -26,7 +26,7 @@ import org.apache.mesos.Protos.Value.Scalar import org.apache.mesos.Protos._ import org.apache.mesos.SchedulerDriver import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.scheduler.cluster.mesos.{ CoarseMesosSchedulerBackend, MemoryUtils } +import org.apache.spark.scheduler.cluster.mesos.{ CoarseGrainedMesosSchedulerBackend, MemoryUtils } import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -36,7 +36,7 @@ import org.scalatest.mock.MockitoSugar import scala.collection.mutable -class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { +class CoarseGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() @@ -83,7 +83,7 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w val taskID0 = TaskID.newBuilder().setValue("0").build() - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { + val backend = new CoarseGrainedMesosSchedulerBackend(taskScheduler, sc, "master") { override val driverUrl = "" } backend.driver = driver @@ -133,7 +133,7 @@ class CoarseMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext w val offer2 = createOffer("o2", "s1", minMem, 1); - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { + val backend = new CoarseGrainedMesosSchedulerBackend(taskScheduler, sc, "master") { override protected val driverUrl: String = "" } backend.driver = driver diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index afbaa9ade811f..c937944b27d31 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils} +import org.apache.spark.scheduler.cluster.mesos.{FineGrainedMesosSchedulerBackend, MemoryUtils} class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { @@ -60,7 +60,7 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val mesosSchedulerBackend = new FineGrainedMesosSchedulerBackend(taskScheduler, sc, "master") // uri is null. val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id") @@ -109,7 +109,7 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo mesosOffers.add(createOffer(2, minMem - 1, minCpu)) mesosOffers.add(createOffer(3, minMem, minCpu)) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val backend = new FineGrainedMesosSchedulerBackend(taskScheduler, sc, "master") val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) expectedWorkerOffers.append(new WorkerOffer( From 62cc9d8b8fc65c475f9d128d0a34e37a2d38e13d Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 26 Mar 2015 18:11:13 -0500 Subject: [PATCH 08/18] Saving work to refactor Mesos coarse-grained handling and tests. --- .../CoarseGrainedSchedulerBackend.scala | 19 +- .../CoarseGrainedMesosSchedulerBackend.scala | 220 ++++------------ .../mesos/CommonMesosSchedulerBackend.scala | 246 ++++++++++++++++++ ...rseGrainedMesosSchedulerBackendSuite.scala | 137 +--------- .../MesosSchedulerBackendSuiteHelper.scala | 182 +++++++++++++ 5 files changed, 496 insertions(+), 308 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6f77fa32ce37b..0324e47bfe6a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -75,7 +75,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] - override def preStart() { + override def preStart(): Unit = { // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -163,21 +163,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } // Make fake resource offers on all executors - def makeOffers() { + def makeOffers(): Unit = { launchTasks(scheduler.resourceOffers(executorDataMap.map { case (id, executorData) => new WorkerOffer(id, executorData.executorHost, executorData.freeCores) }.toSeq)) } // Make fake resource offers on just one executor - def makeOffers(executorId: String) { + def makeOffers(executorId: String): Unit = { val executorData = executorDataMap(executorId) launchTasks(scheduler.resourceOffers( Seq(new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores)))) } // Launch tasks returned by a set of resource offers - def launchTasks(tasks: Seq[Seq[TaskDescription]]) { + def launchTasks(tasks: Seq[Seq[TaskDescription]]): Unit = { for (task <- tasks.flatten) { val ser = SparkEnv.get.closureSerializer.newInstance() val serializedTask = ser.serialize(task) @@ -227,7 +227,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste var driverActor: ActorRef = null val taskIdsOnSlave = new HashMap[String, HashSet[String]] - override def start() { + override def start(): Unit = { val properties = new ArrayBuffer[(String, String)] for ((key, value) <- scheduler.sc.conf.getAll) { if (key.startsWith("spark.")) { @@ -239,7 +239,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } - def stopExecutors() { + def stopExecutors(): Unit = { try { if (driverActor != null) { logInfo("Shutting down all executors") @@ -252,7 +252,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } } - override def stop() { + override def stop(): Unit = { stopExecutors() try { if (driverActor != null) { @@ -265,11 +265,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } } - override def reviveOffers() { + override def reviveOffers(): Unit = { driverActor ! ReviveOffers } - override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { driverActor ! KillTask(taskId, executorId, interruptThread) } @@ -363,6 +363,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste */ final override def killExecutors(executorIds: Seq[String]): Boolean = synchronized { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") + println(s"killExecutors: ids = ${executorIds}") val filteredExecutorIds = new ArrayBuffer[String] executorIds.foreach { id => if (executorDataMap.contains(id)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala index 1601ccc6086eb..9b9f33db05ef7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala @@ -25,7 +25,7 @@ import java.util.concurrent.locks.ReentrantLock import com.google.common.collect.HashBiMap import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, HashSet} +import scala.collection.mutable.{HashMap => MutableHashMap, HashSet => MutableHashSet} import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ @@ -47,147 +47,66 @@ import org.apache.spark.util.{Utils, AkkaUtils} * remove this. */ private[spark] class CoarseGrainedMesosSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String) + val scheduler: TaskSchedulerImpl, + val sc: SparkContext, + val master: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + with CommonMesosSchedulerBackend with MScheduler with Logging { val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() + // Maximum number of cores to acquire (TODO: we'll need more flexible controls here.) + private[mesos] val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt - // Driver for talking to Mesos - var driver: SchedulerDriver = null + /** Cores we have acquired with each Mesos task ID */ + private[mesos] val coresByTaskId = MutableHashMap.empty[Int, Int] - // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt + private[mesos] var totalCoresAcquired = 0 - // Cores we have acquired with each Mesos task ID - val coresByTaskId = new HashMap[Int, Int] - var totalCoresAcquired = 0 + // How many times tasks on each slave failed? + private[mesos] val failuresBySlaveId = MutableHashMap.empty[String, Int] - val slaveIdsWithExecutors = new HashSet[String] - - val taskIdToSlaveId: HashBiMap[Int, String] = HashBiMap.create[Int, String] - // How many times tasks on each slave failed - val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int] - - - // the total number of executors we aim to have - private[mesos] var executorLimitOption: Option[Int] = None - - /** - * Return the current executor limit, which may be [[Int.MaxValue]] - * before properly initialized. - */ - def executorLimit = executorLimitOption.getOrElse(Int.MaxValue) - - private[mesos] val pendingRemovedSlaveIds = new HashSet[String] - - // private lock object protecting mutable state above. Using the intrinsic lock - // may lead to deadlocks since the superclass might also try to lock - private val stateLock = new ReentrantLock + private[mesos] val pendingRemovedSlaveIds = MutableHashSet.empty[String] val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) - var nextMesosTaskId = 0 - - @volatile var appId: String = _ - + /** Return a new task id for coarse-grained mode. */ def newMesosTaskId(): Int = { val id = nextMesosTaskId nextMesosTaskId += 1 id } - override def start() { - super.start() + // ==== Definitions for start(): - stateLock.synchronized { - new Thread("CoarseGrainedMesosSchedulerBackend driver") { - setDaemon(true) - override def run() { - val scheduler = CoarseGrainedMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { - val ret = driver.run() - logInfo("driver.run() returned with code " + ret) - } catch { - case e: Exception => logError("driver.run() failed", e) - } - } - }.start() + // Nothing to do + protected def preStart(): Unit = {} - waitForRegister() - } + // Nothing to do + protected def postStart(): Unit = {} + + /** @see CommonMesosSchedulerBackend.doStart() */ + override def start() = { + super.start() + doStart() + } + + /** @see CommonMesosSchedulerBackend.doStop() */ + override def stop() { + super.stop() + doStop() } def createCommand(offer: Offer, numCores: Int): CommandInfo = { - val executorSparkHome = conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } - val environment = Environment.newBuilder() - val extraClassPath = conf.getOption("spark.executor.extraClassPath") - extraClassPath.foreach { cp => - environment.addVariables( - Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) - } - val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "") - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - - val uri = conf.get("spark.executor.uri", null) - if (uri == null) { - val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath - command.setValue( - "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" - .format(prefixEnv, runScript) + - s" --driver-url $driverUrl" + - s" --executor-id ${offer.getSlaveId.getValue}" + - s" --hostname ${offer.getHostname}" + - s" --cores $numCores" + - s" --app-id $appId") - } else { - // Grab everything to the first '.'. We'll use that and '*' to - // glob the directory "correctly". - val basename = uri.split('/').last.split('.').head - command.setValue( - s"cd $basename*; $prefixEnv " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + + val extraCommandArguments = s" --driver-url $driverUrl" + s" --executor-id ${offer.getSlaveId.getValue}" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + - s" --app-id $appId") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) - } - command.build() + s" --app-id $appId" + createCommandInfo(extraCommandArguments) } protected def driverUrl: String = { @@ -199,29 +118,10 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( CoarseGrainedSchedulerBackend.ACTOR_NAME) } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} - - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - appId = frameworkId.getValue - logInfo("Registered as framework ID " + appId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } + override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) = { + doRegistered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) } - def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) { - registeredLock.wait() - } - } - } - - override def disconnected(d: SchedulerDriver) {} - - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} - /** * Method called by Mesos to offer resources on slaves. We respond by launching an executor, * unless we've already launched more than we wanted to. @@ -234,7 +134,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( val slaveId = offer.getSlaveId.getValue val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (taskIdToSlaveId.size < executorLimitOption.getOrElse(Int.MaxValue) && + if (taskIdToSlaveId.size < executorLimit && totalCoresAcquired < maxCores && mem >= MemoryUtils.calculateTotalMemory(sc) && cpus >= 1 && @@ -266,14 +166,6 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( } } - /** Helper function to pull out a resource from a Mesos Resources protobuf */ - private def getResource(res: JList[Resource], name: String): Double = { - for (r <- res if r.getName == name) { - return r.getScalar.getValue - } - 0 - } - /** Build a Mesos resource protobuf object */ private def createResource(resourceName: String, quantity: Double): Protos.Resource = { Resource.newBuilder() @@ -283,15 +175,8 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - private def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: SchedulerDriver, status: TaskStatus): Unit = { val taskId = status.getTaskId.getValue.toInt val state = status.getState logInfo("Mesos task " + taskId + " is now " + state) @@ -319,19 +204,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( } } - override def error(d: SchedulerDriver, message: String) { - logError("Mesos error: " + message) - scheduler.error(message) - } - - override def stop() { - super.stop() - if (driver != null) { - driver.stop() - } - } - - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + override def error(d: SchedulerDriver, message: String) = doError(d, message) /** Called when a slave is lost or a Mesos task finished. Update local view on * what tasks are running and remove the terminated slave from the list of pending @@ -343,7 +216,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( if (slaveIdsWithExecutors.contains(slaveId)) { val slaveIdToTaskId = taskIdToSlaveId.inverse() if (slaveIdToTaskId.contains(slaveId)) { - val taskId: Int = slaveIdToTaskId.get(slaveId) + val taskId: Long = slaveIdToTaskId.get(slaveId) taskIdToSlaveId.remove(taskId) removeExecutor(sparkExecutorId(slaveId, taskId.toString), reason) } @@ -356,8 +229,9 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( private def sparkExecutorId(slaveId: String, taskId: String) = "%s/%s".format(slaveId, taskId) override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { - logInfo("Mesos slave lost: " + slaveId.getValue) - executorTerminated(d, slaveId.getValue, "Mesos slave lost: " + slaveId.getValue) + val sid = slaveId.getValue + logInfo("Mesos slave lost: " + sid) + executorTerminated(d, sid, "Mesos slave lost: " + sid) } override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int) { @@ -365,17 +239,11 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( slaveLost(d, s) } - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { // We don't truly know if we can fulfill the full amount of executors // since at coarse grain it depends on the amount of slaves available. logInfo("Capping the total amount of executors to " + requestedTotal) - executorLimitOption = Option(requestedTotal) + executorLimit = requestedTotal true } @@ -404,7 +272,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( // executors, that means the scheduler wants to set the limit that is less than // the amount of the executors that has been launched. Therefore, we take the existing // amount of executors launched and deduct the executors killed as the new limit. - executorLimitOption = Option(taskIdToSlaveId.size - pendingRemovedSlaveIds.size) + executorLimit = taskIdToSlaveId.size - pendingRemovedSlaveIds.size true } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala new file mode 100644 index 0000000000000..6961ce98892f3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import java.io.File +import java.util.{List => JList} +import java.util.Collections +import java.util.concurrent.locks.ReentrantLock + +import com.google.common.collect.HashBiMap + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap => MutableHashMap, HashSet => MutableHashSet} + +import org.apache.mesos.{Scheduler => MScheduler} +import org.apache.mesos._ +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} + +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} +import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.{Utils, AkkaUtils} + +/** + * Shared code between {@link FineGrainedMesosSchedulerBackend} and + * {@link CoarseGrainedMesosSchedulerBackend}. + * TODO: This should really be private to the mesos package, but we need visibility + * in the tests, which are not in the same package. Fix the latter. + */ +trait CommonMesosSchedulerBackend extends SchedulerBackend { + + self: MScheduler with Logging => + + // TODO Move these declarations somewhere else? + def resourceOffers(d: SchedulerDriver, offers: JList[Offer]): Unit + def slaveLost(d: SchedulerDriver, slaveId: SlaveID): Unit + def statusUpdate(d: SchedulerDriver, status: TaskStatus): Unit + def requestExecutors(numAdditionalExecutors: Int): Boolean + def requestTotalExecutors(numAdditionalExecutors: Int): Boolean + def doKillExecutors(executorIds: Seq[String]): Boolean + + val scheduler: TaskSchedulerImpl + val sc: SparkContext + val master: String + + def executorBackendName: String = this.getClass.getName + def executorSimpleBackendName: String = this.getClass.getSimpleName + + /** Driver for talking to Mesos */ + var driver: SchedulerDriver = null + + // the total number of executors we aim to have + private[mesos] var executorLimit = Int.MaxValue + + /** + * Return the current executor limit, which may be [[Int.MaxValue]]. + */ + def getExecutorLimit = executorLimit + + private[spark] val slaveIdsWithExecutors = MutableHashSet.empty[String] + + private[spark] val taskIdToSlaveId = HashBiMap.create[Long, String] + + var nextMesosTaskId = 0 + + @volatile var appId: String = _ + + // Lock used to wait for scheduler to be registered + private var isRegistered = false + private val registeredLock = new Object() + + // Protected lock object protecting other mutable state. Using the intrinsic lock + // may lead to deadlocks since the superclass might also try to lock + protected val stateLock = new ReentrantLock + + // ==== Declarations for doStart(): + + protected def preStart(): Unit + protected def postStart(): Unit + + /** + * We would like to override start here and we almost can, except + * unfortunately, we have to call super.start in + * CoarseGrainedMesosSchedulerBackend.start, to invoke + * CoarseGrainedSchedulerBackend.start), which is concrete. + * However, for FineGrainedMesosSchedulerBackend, we _can't_ call + * super.start, because SchedulerBackend.start is abstract. + * So, all the common logic is implemented in this helper method and each + * concrete class overrides start itself. + */ + protected def doStart(): Unit = { + preStart() + + stateLock.synchronized { + new Thread(s"$executorSimpleBackendName driver") { + setDaemon(true) + override def run() { + // i.e., val scheduler = CoarseGrainedMesosSchedulerBackend.this + val scheduler = self + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() + driver = new MesosSchedulerDriver(scheduler, fwInfo, master) + try { + val ret = driver.run() + logInfo("driver.run() returned with code " + ret) + } catch { + case e: Exception => logError("driver.run() failed", e) + } + } + }.start() + + waitForRegister() + postStart() + } + } + + /** Like start, we must override stop the same way. */ + protected def doStop(): Unit = { + if (driver != null) { + driver.stop() + } + } + + def createCommandInfo(extraCommandArguments: String): CommandInfo = { + val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") + .orElse(sc.getSparkHome()) + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } + val environment = Environment.newBuilder() + sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => + environment.addVariables( + Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + } + val extraJavaOpts = sc.conf.get("spark.executor.extraJavaOptions", "") + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + Utils.libraryPathEnvPrefix(Seq(p)) + }.getOrElse("") + + environment.addVariables( + Environment.Variable.newBuilder() + .setName("SPARK_EXECUTOR_OPTS") + .setValue(extraJavaOpts) + .build()) + + sc.executorEnvs.foreach { case (key, value) => + environment.addVariables(Environment.Variable.newBuilder() + .setName(key) + .setValue(value) + .build()) + } + val command = CommandInfo.newBuilder() + .setEnvironment(environment) + + val uri = sc.conf.get("spark.executor.uri", null) + if (uri == null) { + val executorPath= new File(executorSparkHome, "./bin/spark-class").getCanonicalPath + command.setValue( + "%s \"%s\" %s %s" + .format(prefixEnv, executorPath, executorBackendName, extraCommandArguments)) + } else { + // Grab everything to the first '.'. We'll use that and '*' to + // glob the directory "correctly". + val basename = uri.split('/').last.split('.').head + command.setValue( + s"cd $basename*; $prefixEnv " + + "./bin/spark-class $executorBackendName" + extraCommandArguments) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + } + command.build() + } + + /** Handle rescinding of an offer from Mesos. */ + override def offerRescinded(d: SchedulerDriver, o: OfferID): Unit = {} + + /** + * Implements registered for coarse grained, but the fine grained + * implementation wraps it in the separate class loader. + */ + protected def doRegistered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo): Unit = { + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) + registeredLock.synchronized { + isRegistered = true + registeredLock.notifyAll() + } + } + + /** Busy-wait for registration to complete. */ + def waitForRegister(): Unit = { + registeredLock.synchronized { + while (!isRegistered) { + registeredLock.wait() + } + } + } + + override def disconnected(d: SchedulerDriver): Unit = {} + + override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo): Unit = {} + + + /** Helper function to pull out a resource from a Mesos Resources protobuf */ + protected def getResource(res: JList[Resource], name: String): Double = { + for (r <- res if r.getName == name) { + return r.getScalar.getValue + } + 0 + } + + /** Check whether a Mesos task state represents a finished task */ + protected def isFinished(state: MesosTaskState): Boolean = { + state == MesosTaskState.TASK_FINISHED || + state == MesosTaskState.TASK_FAILED || + state == MesosTaskState.TASK_KILLED || + state == MesosTaskState.TASK_LOST + } + + /** + * Implements error for coarse grained, but the fine grained + * implementation wraps it in the separate class loader. + */ + protected def doError(d: SchedulerDriver, message: String): Unit = { + logError("Mesos error: " + message) + scheduler.error(message) + } + + override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]): Unit = {} +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala index aca362c3b9270..80c567e2731ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala @@ -36,134 +36,25 @@ import org.scalatest.mock.MockitoSugar import scala.collection.mutable -class CoarseGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { - - private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(mem)) - builder.addResourcesBuilder() - .setName("cpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(offerId).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host${slaveId}").build() - } - - - private def mockEnvironment(sparkConf: SparkConf): (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - val se = mock[SparkEnv] - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) - when(sc.getSparkHome()).thenReturn(Option("/path")) - val emptyHashMap = new mutable.HashMap[String, String] - when(sc.executorEnvs).thenReturn(emptyHashMap) - when(sc.conf).thenReturn(sparkConf) - when(sc.env).thenReturn(se) - when(taskScheduler.sc).thenReturn(sc) - (sc, taskScheduler, driver) - } - - test("mesos supports killing and limiting executors") { - val sparkConf = new SparkConf - val (sc, taskScheduler, driver) = mockEnvironment(sparkConf) - - sparkConf.set("spark.driver.host", "driverHost") - sparkConf.set("spark.driver.port", "1234") - - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt - val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) - - val taskID0 = TaskID.newBuilder().setValue("0").build() - - val backend = new CoarseGrainedMesosSchedulerBackend(taskScheduler, sc, "master") { +class CoarseGrainedMesosSchedulerBackendSuite + extends FunSuite + with MesosSchedulerBackendSuiteHelper + with LocalSparkContext + with MockitoSugar { + + protected def makeTestMesosSchedulerBackend( + taskScheduler: TaskSchedulerImpl, + sc: SparkContext): CoarseGrainedMesosSchedulerBackend = { + new CoarseGrainedMesosSchedulerBackend(taskScheduler, sc, "master") { override val driverUrl = "" } - backend.driver = driver - backend.resourceOffers(driver, mesosOffers) - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), - any[util.Collection[TaskInfo]], - any[Filters]) - - // Calling doKillExecutors should invoke driver.killTask. - assert(backend.doKillExecutors(Seq("s1/0"))) - verify(driver, times(1)).killTask(taskID0) - assert(backend.executorLimit === 0) - - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) - backend.resourceOffers(driver, mesosOffers2) - - verify(driver, times(1)) - .declineOffer(OfferID.newBuilder().setValue("o2").build()) - - // Verify we didn't launch any new executor - assert(backend.slaveIdsWithExecutors.size === 1) - - backend.doRequestTotalExecutors(2) - backend.resourceOffers(driver, mesosOffers2) - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), - any[util.Collection[TaskInfo]], - any[Filters]) + } - assert(backend.slaveIdsWithExecutors.size === 2) - backend.slaveLost(driver, SlaveID.newBuilder().setValue("s1").build()) - assert(backend.slaveIdsWithExecutors.size === 1) + test("mesos supports killing and limiting executors") { + killAndLimitExecutors() } test("mesos supports killing and relaunching tasks with executors") { - val sparkConf = new SparkConf - val (sc, taskScheduler, driver) = mockEnvironment(sparkConf) - - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 - val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - val offer1 = createOffer("o1", "s1", minMem, minCpu) - mesosOffers.add(offer1) - - val offer2 = createOffer("o2", "s1", minMem, 1); - - val backend = new CoarseGrainedMesosSchedulerBackend(taskScheduler, sc, "master") { - override protected val driverUrl: String = "" - } - backend.driver = driver - backend.resourceOffers(driver, mesosOffers) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer1.getId)), - anyObject(), - anyObject[Filters]) - - // Simulate task killed, executor no longer running - val status = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue("0").build()) - .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) - .setState(TaskState.TASK_KILLED) - .build - - backend.statusUpdate(driver, status) - assert(!backend.slaveIdsWithExecutors.contains("s1")) - - mesosOffers.clear() - mesosOffers.add(offer2) - backend.resourceOffers(driver, mesosOffers) - assert(backend.slaveIdsWithExecutors.contains("s1")) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer2.getId)), - anyObject(), - anyObject[Filters]) - - verify(driver, times(1)).reviveOffers() + killAndRelaunchTasks() } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala new file mode 100644 index 0000000000000..8bea0a5efb0c7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.mesos + +import java.util +import java.util.Collections + +import akka.actor.ActorSystem +import com.typesafe.config.Config +import org.apache.mesos.Protos.Value.Scalar +import org.apache.mesos.Protos._ +import org.apache.mesos.SchedulerDriver +import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.mesos.{ CommonMesosSchedulerBackend, MemoryUtils } +import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.mockito.{ ArgumentCaptor, Matchers } +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + +import scala.collection.mutable + +trait MesosSchedulerBackendSuiteHelper { + self: FunSuite with LocalSparkContext with MockitoSugar => + + private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { + val builder = Offer.newBuilder() + builder.addResourcesBuilder() + .setName("mem") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(mem)) + builder.addResourcesBuilder() + .setName("cpus") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(cpu)) + builder.setId(OfferID.newBuilder().setValue(offerId).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host$slaveId").build() + } + + + private def mockEnvironment(): (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { + val sparkConf = new SparkConf + sparkConf.set("spark.driver.host", "driverHost") + sparkConf.set("spark.driver.port", "1234") + + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] + val se = mock[SparkEnv] + val sc = mock[SparkContext] + when(sc.executorMemory).thenReturn(100) + when(sc.getSparkHome()).thenReturn(Option("/path")) + + val emptyHashMap = mutable.HashMap.empty[String, String] + when(sc.executorEnvs).thenReturn(emptyHashMap) + when(sc.conf).thenReturn(sparkConf) + when(sc.env).thenReturn(se) + when(taskScheduler.sc).thenReturn(sc) + (sc, taskScheduler, driver) + } + + // Simulate task killed, executor no longer running + private def makeKilledTaskStatus = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue("0").build()) + .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) + .setState(TaskState.TASK_KILLED) + .build + + + protected def makeTestMesosSchedulerBackend( + taskScheduler: TaskSchedulerImpl, + sc: SparkContext): CommonMesosSchedulerBackend + + def killAndLimitExecutors() { + val (sc, taskScheduler, driver) = mockEnvironment() + + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + val minCpu = 4 + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) + + val taskID0 = TaskID.newBuilder().setValue("0").build() + + val backend = makeTestMesosSchedulerBackend(taskScheduler, sc) + backend.driver = driver + backend.resourceOffers(driver, mesosOffers) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) + + // Calling doKillExecutors should invoke driver.killTask. + // TODO: Shouldn't we call killExecutors?? + assert(backend.doKillExecutors(Seq("s1/0"))) + verify(driver, times(1)).killTask(taskID0) + // Must invoke the status update explicitly here. + backend.statusUpdate(driver, makeKilledTaskStatus) + + // Verify we don't have any executors. + assert(backend.slaveIdsWithExecutors.size === 0) + // Verify that the executor limit is now 0. + assert(backend.getExecutorLimit === 0) + + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) + backend.resourceOffers(driver, mesosOffers2) + + verify(driver, times(1)) + .declineOffer(OfferID.newBuilder().setValue("o2").build()) + + // Verify we didn't launch any new executor + assert(backend.slaveIdsWithExecutors.size === 0) + assert(backend.getExecutorLimit === 0) + + // Now allow one executor: + backend.requestExecutors(1) + backend.resourceOffers(driver, mesosOffers2) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) + + assert(backend.slaveIdsWithExecutors.size === 1) + backend.slaveLost(driver, SlaveID.newBuilder().setValue("s2").build()) + assert(backend.slaveIdsWithExecutors.size === 0) + } + + def killAndRelaunchTasks() { + val (sc, taskScheduler, driver) = mockEnvironment() + + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 + val minCpu = 4 + + val offer1 = createOffer("o1", "s1", minMem, minCpu) + val offer2 = createOffer("o2", "s2", minMem, 1) + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(offer1) // Just offer 1 now. + + val backend = makeTestMesosSchedulerBackend(taskScheduler, sc) + backend.driver = driver + backend.resourceOffers(driver, mesosOffers) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer1.getId)), + anyObject(), + anyObject[Filters]) + assert(backend.slaveIdsWithExecutors.contains("s1")) + + backend.statusUpdate(driver, makeKilledTaskStatus) + assert(!backend.slaveIdsWithExecutors.contains("s1")) + + mesosOffers.clear() + mesosOffers.add(offer2) + backend.resourceOffers(driver, mesosOffers) + assert(!backend.slaveIdsWithExecutors.contains("s1")) + assert( backend.slaveIdsWithExecutors.contains("s2")) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer2.getId)), + anyObject(), + anyObject[Filters]) + + verify(driver, times(1)).reviveOffers() + } +} From d49be1013770ea341b7a803d78401ae5d8d4213a Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 26 Mar 2015 18:18:15 -0500 Subject: [PATCH 09/18] Test refinement. --- .../mesos/MesosSchedulerBackendSuiteHelper.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala index 8bea0a5efb0c7..f3e34ebcea31c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -75,11 +75,12 @@ trait MesosSchedulerBackendSuiteHelper { } // Simulate task killed, executor no longer running - private def makeKilledTaskStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue("0").build()) - .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) - .setState(TaskState.TASK_KILLED) - .build + private def makeKilledTaskStatus(taskId: String, slaveId: String) = + TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(taskId).build()) + .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) + .setState(TaskState.TASK_KILLED) + .build protected def makeTestMesosSchedulerBackend( @@ -110,7 +111,7 @@ trait MesosSchedulerBackendSuiteHelper { assert(backend.doKillExecutors(Seq("s1/0"))) verify(driver, times(1)).killTask(taskID0) // Must invoke the status update explicitly here. - backend.statusUpdate(driver, makeKilledTaskStatus) + backend.statusUpdate(driver, makeKilledTaskStatus("0", "s1")) // Verify we don't have any executors. assert(backend.slaveIdsWithExecutors.size === 0) @@ -163,7 +164,7 @@ trait MesosSchedulerBackendSuiteHelper { anyObject[Filters]) assert(backend.slaveIdsWithExecutors.contains("s1")) - backend.statusUpdate(driver, makeKilledTaskStatus) + backend.statusUpdate(driver, makeKilledTaskStatus("0", "s1")) assert(!backend.slaveIdsWithExecutors.contains("s1")) mesosOffers.clear() From ebb7f1befa0acf6ae72e8405c2d457aff21fd521 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 2 Apr 2015 16:31:32 -0500 Subject: [PATCH 10/18] Better testing of the state transitions for the coarse-grained Mesos scheduler backend. --- .../CoarseGrainedMesosSchedulerBackend.scala | 14 +- .../mesos/CommonMesosSchedulerBackend.scala | 19 +- .../FineGrainedMesosSchedulerBackend.scala | 202 +++++------------- ...rseGrainedMesosSchedulerBackendSuite.scala | 23 +- .../MesosSchedulerBackendSuiteHelper.scala | 138 ++++++++---- 5 files changed, 179 insertions(+), 217 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala index 9b9f33db05ef7..6709dcb0368c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala @@ -43,14 +43,14 @@ import org.apache.spark.util.{Utils, AkkaUtils} * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable * latency. * - * Unfortunately this has a bit of duplication from FineGrainedMesosSchedulerBackend, but it seems hard to - * remove this. + * Unfortunately, there is some duplication with FineGrainedMesosSchedulerBackend + * that is hard to remove. */ private[spark] class CoarseGrainedMesosSchedulerBackend( val scheduler: TaskSchedulerImpl, - val sc: SparkContext, + val sparkContext: SparkContext, val master: String) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sparkContext.env.actorSystem) with CommonMesosSchedulerBackend with MScheduler with Logging { @@ -111,7 +111,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( protected def driverUrl: String = { AkkaUtils.address( - AkkaUtils.protocol(sc.env.actorSystem), + AkkaUtils.protocol(sparkContext.env.actorSystem), SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), @@ -136,7 +136,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( val cpus = getResource(offer.getResourcesList, "cpus").toInt if (taskIdToSlaveId.size < executorLimit && totalCoresAcquired < maxCores && - mem >= MemoryUtils.calculateTotalMemory(sc) && + mem >= MemoryUtils.calculateTotalMemory(sparkContext) && cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { @@ -154,7 +154,7 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", - MemoryUtils.calculateTotalMemory(sc))) + MemoryUtils.calculateTotalMemory(sparkContext))) .build() d.launchTasks( Collections.singleton(offer.getId), Collections.singletonList(task), filters) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala index 6961ce98892f3..cd2f92a414ebe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala @@ -55,7 +55,7 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { def doKillExecutors(executorIds: Seq[String]): Boolean val scheduler: TaskSchedulerImpl - val sc: SparkContext + val sparkContext: SparkContext val master: String def executorBackendName: String = this.getClass.getName @@ -112,7 +112,8 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { override def run() { // i.e., val scheduler = CoarseGrainedMesosSchedulerBackend.this val scheduler = self - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() + val fwInfo = FrameworkInfo.newBuilder(). + setUser(sparkContext.sparkUser).setName(sparkContext.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { val ret = driver.run() @@ -136,21 +137,21 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { } def createCommandInfo(extraCommandArguments: String): CommandInfo = { - val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) + val executorSparkHome = sparkContext.conf.getOption("spark.mesos.executor.home") + .orElse(sparkContext.getSparkHome()) .getOrElse { throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") } val environment = Environment.newBuilder() - sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => + sparkContext.conf.getOption("spark.executor.extraClassPath").foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = sc.conf.get("spark.executor.extraJavaOptions", "") + val extraJavaOpts = sparkContext.conf.get("spark.executor.extraJavaOptions", "") // Set the environment variable through a command prefix // to append to the existing value of the variable - val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = sparkContext.conf.getOption("spark.executor.extraLibraryPath").map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -160,7 +161,7 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { .setValue(extraJavaOpts) .build()) - sc.executorEnvs.foreach { case (key, value) => + sparkContext.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) @@ -169,7 +170,7 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.get("spark.executor.uri", null) + val uri = sparkContext.conf.get("spark.executor.uri", null) if (uri == null) { val executorPath= new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala index cb38415dce74e..8274e4f0dbb3b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala @@ -40,101 +40,50 @@ import org.apache.spark.util.Utils * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks * from multiple apps can run on different cores) and in time (a core can switch ownership). + * + * Unfortunately, there is some duplication with CoarseGrainedMesosSchedulerBackend + * that is hard to remove. */ private[spark] class FineGrainedMesosSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String) - extends SchedulerBackend + val scheduler: TaskSchedulerImpl, + val sparkContext: SparkContext, + val master: String) + extends CommonMesosSchedulerBackend with MScheduler with Logging { - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() - - // Driver for talking to Mesos - var driver: SchedulerDriver = null - - // Which slave IDs we have executors on - val slaveIdsWithExecutors = new HashSet[String] - val taskIdToSlaveId = new HashMap[Long, String] - // An ExecutorInfo for our tasks var execArgs: Array[Byte] = null var classLoader: ClassLoader = null // The listener bus to publish executor added/removed events. - val listenerBus = sc.listenerBus - - @volatile var appId: String = _ - - override def start() { - synchronized { - classLoader = Thread.currentThread.getContextClassLoader - - new Thread("FineGrainedMesosSchedulerBackend driver") { - setDaemon(true) - override def run() { - val scheduler = FineGrainedMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { - val ret = driver.run() - logInfo("driver.run() returned with code " + ret) - } catch { - case e: Exception => logError("driver.run() failed", e) - } - } - }.start() + val listenerBus = sparkContext.listenerBus - waitForRegister() - } + // ==== Definitions for start(): + + protected val backendName: String = "FineGrainedMesosSchedulerBackend" + + // Initialize the classLoader. + protected def preStart(): Unit = { + classLoader = Thread.currentThread.getContextClassLoader + } + + // Nothing to do + protected def postStart(): Unit = {} + + /** @see CommonMesosSchedulerBackend.doStart() */ + override def start(): Unit = { + doStart() + } + + /** @see CommonMesosSchedulerBackend.doStop() */ + override def stop(): Unit = { + doStop() } def createExecutorInfo(execId: String): MesosExecutorInfo = { - val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } - val environment = Environment.newBuilder() - sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => - environment.addVariables( - Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) - } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").getOrElse("") - - val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - val uri = sc.conf.get("spark.executor.uri", null) - val executorBackendName = classOf[MesosExecutorBackend].getName - if (uri == null) { - 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". - val basename = uri.split('/').last.split('.').head - command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) - } + val command = createCommandInfo("") val cpus = Resource.newBuilder() .setName("cpus") .setType(Value.Type.SCALAR) @@ -146,7 +95,7 @@ private[spark] class FineGrainedMesosSchedulerBackend( .setType(Value.Type.SCALAR) .setScalar( Value.Scalar.newBuilder() - .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) + .setValue(MemoryUtils.calculateTotalMemory(sparkContext)).build()) .build() MesosExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -164,7 +113,7 @@ private[spark] class FineGrainedMesosSchedulerBackend( private def createExecArg(): Array[Byte] = { if (execArgs == null) { val props = new HashMap[String, String] - for ((key,value) <- sc.conf.getAll) { + for ((key,value) <- sparkContext.conf.getAll) { props(key) = value } // Serialize the map as an array of (String, String) pairs @@ -173,28 +122,14 @@ private[spark] class FineGrainedMesosSchedulerBackend( execArgs } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} - - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + /** TODO: is wrapping in the separate class loader necessary? */ + override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo): Unit = { inClassLoader() { - appId = frameworkId.getValue - logInfo("Registered as framework ID " + appId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } - } - } - - def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) { - registeredLock.wait() - } + doRegistered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) } } - private def inClassLoader()(fun: => Unit) = { + private def inClassLoader()(fun: => Unit): Unit = { val oldClassLoader = Thread.currentThread.getContextClassLoader Thread.currentThread.setContextClassLoader(classLoader) try { @@ -204,16 +139,12 @@ private[spark] class FineGrainedMesosSchedulerBackend( } } - override def disconnected(d: SchedulerDriver) {} - - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} - /** * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]): Unit = { inClassLoader() { // Fail-fast on offers we know will be rejected val (usableOffers, unUsableOffers) = offers.partition { o => @@ -221,7 +152,7 @@ private[spark] class FineGrainedMesosSchedulerBackend( val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? - (mem >= MemoryUtils.calculateTotalMemory(sc) && + (mem >= MemoryUtils.calculateTotalMemory(sparkContext) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || (slaveIdsWithExecutors.contains(slaveId) && @@ -287,14 +218,6 @@ private[spark] class FineGrainedMesosSchedulerBackend( } } - /** Helper function to pull out a resource from a Mesos Resources protobuf */ - def getResource(res: JList[Resource], name: String): Double = { - for (r <- res if r.getName == name) { - return r.getScalar.getValue - } - 0 - } - /** Turn a Spark TaskDescription into a Mesos task */ def createMesosTask(task: TaskDescription, slaveId: String): MesosTaskInfo = { val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build() @@ -313,19 +236,11 @@ private[spark] class FineGrainedMesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + override def statusUpdate(d: SchedulerDriver, status: TaskStatus): Unit = { inClassLoader() { val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) - synchronized { + stateLock.synchronized { if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { // We lost the executor on this slave, so remember that it's gone removeExecutor(taskIdToSlaveId(tid), "Lost executor") @@ -338,36 +253,28 @@ private[spark] class FineGrainedMesosSchedulerBackend( } } - override def error(d: SchedulerDriver, message: String) { + /** TODO: is wrapping in the separate class loader necessary? */ + override def error(d: SchedulerDriver, message: String): Unit = { inClassLoader() { - logError("Mesos error: " + message) - scheduler.error(message) - } - } - - override def stop() { - if (driver != null) { - driver.stop() + doError(d, message) } } - override def reviveOffers() { + override def reviveOffers(): Unit = { driver.reviveOffers() } - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - /** * Remove executor associated with slaveId in a thread safe manner. */ - private def removeExecutor(slaveId: String, reason: String) = { - synchronized { + private def removeExecutor(slaveId: String, reason: String): Unit = { + stateLock.synchronized { listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), slaveId, reason)) slaveIdsWithExecutors -= slaveId } } - private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { + private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason): Unit = { inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) removeExecutor(slaveId.getValue, reason.toString) @@ -375,12 +282,12 @@ private[spark] class FineGrainedMesosSchedulerBackend( } } - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { + override def slaveLost(d: SchedulerDriver, slaveId: SlaveID): Unit = { recordSlaveLost(d, slaveId, SlaveLost()) } override def executorLost(d: SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, status: Int) { + slaveId: SlaveID, status: Int): Unit = { logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, slaveId.getValue)) recordSlaveLost(d, slaveId, ExecutorExited(status)) @@ -393,13 +300,12 @@ private[spark] class FineGrainedMesosSchedulerBackend( ) } - // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + // TODO: Not currently used. + def requestExecutors(numAdditionalExecutors: Int): Boolean = false + def requestTotalExecutors(numAdditionalExecutors: Int): Boolean = false + def doKillExecutors(executorIds: Seq[String]): Boolean = false - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } + // TODO: query Mesos for number of cores + override def defaultParallelism() = sparkContext.conf.getInt("spark.default.parallelism", 8) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala index 80c567e2731ec..e5708b7d5852a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala @@ -43,18 +43,29 @@ class CoarseGrainedMesosSchedulerBackendSuite with MockitoSugar { protected def makeTestMesosSchedulerBackend( - taskScheduler: TaskSchedulerImpl, - sc: SparkContext): CoarseGrainedMesosSchedulerBackend = { - new CoarseGrainedMesosSchedulerBackend(taskScheduler, sc, "master") { + taskScheduler: TaskSchedulerImpl): CoarseGrainedMesosSchedulerBackend = { + new CoarseGrainedMesosSchedulerBackend(taskScheduler, taskScheduler.sc, "master") { override val driverUrl = "" } } - test("mesos supports killing and limiting executors") { - killAndLimitExecutors() + test("When Mesos offers resources, a Mesos executor is created.") { + makeMesosExecutorsTest + } + + test("When a Mesos executor is killed, the maximum number of allowed Mesos executors is deprecated by one") { + killMesosExecutorDeprecateByOneTest() + } + + test("The maximum number of allowed Mesos executors can be increased by explicitly requesting new Mesos executors") { + increaseAllowedMesosExecutorsTest() + } + + test("Losing a slave and its Mesos executor doesn't change the maximum allowed number of Mesos executors") { + slaveLostDoesntChangeMaxAllowedMesosExecutorsTest() } test("mesos supports killing and relaunching tasks with executors") { - killAndRelaunchTasks() + killAndRelaunchTasksTest() } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala index f3e34ebcea31c..e57292c96f52e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -39,7 +39,7 @@ import scala.collection.mutable trait MesosSchedulerBackendSuiteHelper { self: FunSuite with LocalSparkContext with MockitoSugar => - private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { + private def makeOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -53,14 +53,16 @@ trait MesosSchedulerBackendSuiteHelper { .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host$slaveId").build() } + private def makeOffersList(offers: Offer*): java.util.ArrayList[Offer] = { + val mesosOffers = new java.util.ArrayList[Offer] + for (o <- offers) mesosOffers.add(o) + mesosOffers + } - private def mockEnvironment(): (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { + private def mockSparkContext: SparkContext = { val sparkConf = new SparkConf sparkConf.set("spark.driver.host", "driverHost") sparkConf.set("spark.driver.port", "1234") - - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] val se = mock[SparkEnv] val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) @@ -70,66 +72,102 @@ trait MesosSchedulerBackendSuiteHelper { when(sc.executorEnvs).thenReturn(emptyHashMap) when(sc.conf).thenReturn(sparkConf) when(sc.env).thenReturn(se) + sc + } + + private def mockEnvironment: (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { + val sc = mockSparkContext + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.sc).thenReturn(sc) (sc, taskScheduler, driver) } - // Simulate task killed, executor no longer running + private def makeBackendAndDriver: (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (sc, taskScheduler, driver) = mockEnvironment + val backend = makeTestMesosSchedulerBackend(taskScheduler) + backend.driver = driver + (backend, driver) + } + + private def makeTaskID( id: String): TaskID = TaskID.newBuilder().setValue(id).build() + private def makeSlaveID(id: String): SlaveID = SlaveID.newBuilder().setValue(id).build() + private def makeOfferID(id: String): OfferID = OfferID.newBuilder().setValue(id).build() + + // Simulate task killed message, signaling that an executor is no longer running. private def makeKilledTaskStatus(taskId: String, slaveId: String) = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) + .setTaskId(makeTaskID(taskId)) + .setSlaveId(makeSlaveID(slaveId)) .setState(TaskState.TASK_KILLED) .build + private def minMemMinCPU(sc: SparkContext, extraMemory: Int = 0, numCores: Int = 4): (Int,Int) = + (MemoryUtils.calculateTotalMemory(sc).toInt + extraMemory, numCores) protected def makeTestMesosSchedulerBackend( - taskScheduler: TaskSchedulerImpl, - sc: SparkContext): CommonMesosSchedulerBackend + taskScheduler: TaskSchedulerImpl): CommonMesosSchedulerBackend - def killAndLimitExecutors() { - val (sc, taskScheduler, driver) = mockEnvironment() + val (taskIDVal1, slaveIDVal1) = ("0", "s1") + val (taskIDVal2, slaveIDVal2) = ("1", "s2") - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt - val minCpu = 4 + def makeMesosExecutorsTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (backend, driver) = makeBackendAndDriver - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) - val taskID0 = TaskID.newBuilder().setValue("0").build() + val mesosOffers1 = makeOffersList(makeOffer(taskIDVal1, slaveIDVal1, minMem, minCPU)) - val backend = makeTestMesosSchedulerBackend(taskScheduler, sc) - backend.driver = driver - backend.resourceOffers(driver, mesosOffers) + backend.resourceOffers(driver, mesosOffers1) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + Matchers.eq(Collections.singleton(mesosOffers1.get(0).getId)), any[util.Collection[TaskInfo]], any[Filters]) + // Verify we have one executor and the executor limit is 1. + assert(backend.slaveIdsWithExecutors.size === 1) + assert(backend.getExecutorLimit >= 1) + + (backend, driver) // Return so this test can be embedded in others. + } + + def killMesosExecutorDeprecateByOneTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (backend, driver) = makeMesosExecutorsTest() + // Calling doKillExecutors should invoke driver.killTask. - // TODO: Shouldn't we call killExecutors?? - assert(backend.doKillExecutors(Seq("s1/0"))) - verify(driver, times(1)).killTask(taskID0) + val taskID1 = makeTaskID(taskIDVal1) + assert(backend.doKillExecutors(Seq(s"$slaveIDVal1/$taskIDVal1"))) + verify(driver, times(1)).killTask(taskID1) // Must invoke the status update explicitly here. - backend.statusUpdate(driver, makeKilledTaskStatus("0", "s1")) + // TODO: can we mock other parts of the API so this can be called automatically? + backend.statusUpdate(driver, makeKilledTaskStatus(taskIDVal1, slaveIDVal1)) // Verify we don't have any executors. assert(backend.slaveIdsWithExecutors.size === 0) // Verify that the executor limit is now 0. assert(backend.getExecutorLimit === 0) - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) + val mesosOffers2 = makeOffersList(makeOffer(taskIDVal2, slaveIDVal2, minMem, minCPU)) backend.resourceOffers(driver, mesosOffers2) verify(driver, times(1)) - .declineOffer(OfferID.newBuilder().setValue("o2").build()) + .declineOffer(makeOfferID(taskIDVal2)) // Verify we didn't launch any new executor assert(backend.slaveIdsWithExecutors.size === 0) assert(backend.getExecutorLimit === 0) - // Now allow one executor: + (backend, driver) // Return so this test can be embedded in others. + } + + def increaseAllowedMesosExecutorsTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (backend, driver) = killMesosExecutorDeprecateByOneTest() + + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) + val mesosOffers2 = makeOffersList(makeOffer(taskIDVal2, slaveIDVal2, minMem, minCPU)) + + // Now allow one more executor: backend.requestExecutors(1) backend.resourceOffers(driver, mesosOffers2) verify(driver, times(1)).launchTasks( @@ -138,40 +176,46 @@ trait MesosSchedulerBackendSuiteHelper { any[Filters]) assert(backend.slaveIdsWithExecutors.size === 1) - backend.slaveLost(driver, SlaveID.newBuilder().setValue("s2").build()) - assert(backend.slaveIdsWithExecutors.size === 0) - } + assert(backend.getExecutorLimit >= 1) - def killAndRelaunchTasks() { - val (sc, taskScheduler, driver) = mockEnvironment() + (backend, driver) // Return so this test can be embedded in others. + } - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + 1024 - val minCpu = 4 + def slaveLostDoesntChangeMaxAllowedMesosExecutorsTest(): Unit = { + val (backend, driver) = increaseAllowedMesosExecutorsTest() - val offer1 = createOffer("o1", "s1", minMem, minCpu) - val offer2 = createOffer("o2", "s2", minMem, 1) + backend.slaveLost(driver, makeSlaveID(slaveIDVal2)) + assert(backend.slaveIdsWithExecutors.size === 0) + assert(backend.getExecutorLimit >= 1) + } - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(offer1) // Just offer 1 now. + def killAndRelaunchTasksTest(): Unit = { + val (backend, driver) = makeBackendAndDriver + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext, 1024) + val offer1 = makeOffer(taskIDVal1, slaveIDVal1, minMem, minCPU) + val mesosOffers = makeOffersList(offer1) - val backend = makeTestMesosSchedulerBackend(taskScheduler, sc) - backend.driver = driver backend.resourceOffers(driver, mesosOffers) verify(driver, times(1)).launchTasks( Matchers.eq(Collections.singleton(offer1.getId)), anyObject(), anyObject[Filters]) - assert(backend.slaveIdsWithExecutors.contains("s1")) + assert(backend.slaveIdsWithExecutors.contains(slaveIDVal1)) - backend.statusUpdate(driver, makeKilledTaskStatus("0", "s1")) - assert(!backend.slaveIdsWithExecutors.contains("s1")) + backend.statusUpdate(driver, makeKilledTaskStatus(taskIDVal1, slaveIDVal1)) + assert(!backend.slaveIdsWithExecutors.contains(slaveIDVal1)) + assert(backend.slaveIdsWithExecutors.size === 0) + assert(backend.getExecutorLimit >= 1) + val offer2 = makeOffer(taskIDVal2, slaveIDVal2, minMem, 1) mesosOffers.clear() mesosOffers.add(offer2) backend.resourceOffers(driver, mesosOffers) - assert(!backend.slaveIdsWithExecutors.contains("s1")) - assert( backend.slaveIdsWithExecutors.contains("s2")) + assert(!backend.slaveIdsWithExecutors.contains(slaveIDVal1)) + assert( backend.slaveIdsWithExecutors.contains(slaveIDVal2)) + assert(backend.slaveIdsWithExecutors.size === 1) + assert(backend.getExecutorLimit >= 1) verify(driver, times(1)).launchTasks( Matchers.eq(Collections.singleton(offer2.getId)), From 829bb48f505de2d15f5f1291251540b6244cc2ae Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 2 Apr 2015 17:05:01 -0500 Subject: [PATCH 11/18] Moved the Mesos test package to match the corresponding "main" package. --- .../mesos/CoarseGrainedMesosSchedulerBackendSuite.scala | 0 .../mesos/FineGrainedMesosSchedulerBackendSuite.scala} | 0 .../{ => cluster}/mesos/MesosSchedulerBackendSuiteHelper.scala | 0 .../scheduler/{ => cluster}/mesos/MesosTaskLaunchDataSuite.scala | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename core/src/test/scala/org/apache/spark/scheduler/{ => cluster}/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala (100%) rename core/src/test/scala/org/apache/spark/scheduler/{mesos/MesosSchedulerBackendSuite.scala => cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala} (100%) rename core/src/test/scala/org/apache/spark/scheduler/{ => cluster}/mesos/MesosSchedulerBackendSuiteHelper.scala (100%) rename core/src/test/scala/org/apache/spark/scheduler/{ => cluster}/mesos/MesosTaskLaunchDataSuite.scala (100%) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala similarity index 100% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala similarity index 100% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala similarity index 100% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuiteHelper.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala similarity index 100% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala From d621849621e6b5ec18b42ff063c8ef3552888a9f Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 2 Apr 2015 17:05:32 -0500 Subject: [PATCH 12/18] typo. --- .../cluster/mesos/FineGrainedMesosSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala index 8274e4f0dbb3b..34acee5d7717e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala @@ -122,7 +122,7 @@ private[spark] class FineGrainedMesosSchedulerBackend( execArgs } - /** TODO: is wrapping in the separate class loader necessary? */ + /** TODO: Is wrapping in the separate class loader necessary? */ override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo): Unit = { inClassLoader() { doRegistered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) From a95d2aac596bc1505285d280d6538cc32ea41e06 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 2 Apr 2015 17:05:45 -0500 Subject: [PATCH 13/18] Test name matches class name. --- .../cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala index c937944b27d31..d53ce5bea2669 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala @@ -40,7 +40,7 @@ import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.cluster.mesos.{FineGrainedMesosSchedulerBackend, MemoryUtils} -class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { +class FineGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { test("check spark-class location correctly") { val conf = new SparkConf From fbaf278adbf4182518284c40af16507482ade773 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Thu, 2 Apr 2015 18:03:42 -0500 Subject: [PATCH 14/18] Refactored common propoerties. --- .../CoarseGrainedMesosSchedulerBackend.scala | 4 ++++ .../mesos/CommonMesosSchedulerBackend.scala | 22 +++++++++---------- .../FineGrainedMesosSchedulerBackend.scala | 2 +- ...rseGrainedMesosSchedulerBackendSuite.scala | 3 +-- ...ineGrainedMesosSchedulerBackendSuite.scala | 11 +++++----- .../MesosSchedulerBackendSuiteHelper.scala | 3 +-- .../mesos/MesosTaskLaunchDataSuite.scala | 2 +- 7 files changed, 23 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala index 6709dcb0368c8..04cb3fbd18884 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackend.scala @@ -70,8 +70,12 @@ private[spark] class CoarseGrainedMesosSchedulerBackend( private[mesos] val pendingRemovedSlaveIds = MutableHashSet.empty[String] + protected val executorBackend = this.getClass + val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) + var nextMesosTaskId = 0 + /** Return a new task id for coarse-grained mode. */ def newMesosTaskId(): Int = { val id = nextMesosTaskId diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala index cd2f92a414ebe..d8886b80cf9da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala @@ -58,9 +58,6 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { val sparkContext: SparkContext val master: String - def executorBackendName: String = this.getClass.getName - def executorSimpleBackendName: String = this.getClass.getSimpleName - /** Driver for talking to Mesos */ var driver: SchedulerDriver = null @@ -72,11 +69,14 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { */ def getExecutorLimit = executorLimit - private[spark] val slaveIdsWithExecutors = MutableHashSet.empty[String] + protected val executorBackend: Class[_] + + private[mesos] val taskIdToSlaveId = HashBiMap.create[Long, String] - private[spark] val taskIdToSlaveId = HashBiMap.create[Long, String] + private[mesos] val slaveIdsWithExecutors = MutableHashSet.empty[String] - var nextMesosTaskId = 0 + private def executorBackendName: String = executorBackend.getName + private def executorSimpleBackendName: String = executorBackend.getSimpleName @volatile var appId: String = _ @@ -173,16 +173,14 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { val uri = sparkContext.conf.get("spark.executor.uri", null) if (uri == null) { val executorPath= new File(executorSparkHome, "./bin/spark-class").getCanonicalPath - command.setValue( - "%s \"%s\" %s %s" - .format(prefixEnv, executorPath, executorBackendName, extraCommandArguments)) + command.setValue("%s \"%s\" %s %s".format( + prefixEnv, executorPath, executorBackendName, extraCommandArguments)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue( - s"cd $basename*; $prefixEnv " + - "./bin/spark-class $executorBackendName" + extraCommandArguments) + command.setValue("cd %s*; %s \"%s\" %s %s".format( + basename, prefixEnv, "./bin/spark-class", executorBackendName, extraCommandArguments)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala index 34acee5d7717e..d69e2a4d534ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala @@ -62,7 +62,7 @@ private[spark] class FineGrainedMesosSchedulerBackend( // ==== Definitions for start(): - protected val backendName: String = "FineGrainedMesosSchedulerBackend" + protected val executorBackend = classOf[MesosExecutorBackend] // Initialize the classLoader. protected def preStart(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala index e5708b7d5852a..c17a413f2dc18 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.util import java.util.Collections @@ -26,7 +26,6 @@ import org.apache.mesos.Protos.Value.Scalar import org.apache.mesos.Protos._ import org.apache.mesos.SchedulerDriver import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.scheduler.cluster.mesos.{ CoarseGrainedMesosSchedulerBackend, MemoryUtils } import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } import org.mockito.Matchers._ import org.mockito.Mockito._ diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala index d53ce5bea2669..1eb66c1985ce6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer import java.util @@ -38,11 +38,10 @@ import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.cluster.mesos.{FineGrainedMesosSchedulerBackend, MemoryUtils} class FineGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { - test("check spark-class location correctly") { + test("The spark-class location is correctly computed") { val conf = new SparkConf conf.set("spark.mesos.executor.home" , "/mesos-home") @@ -64,15 +63,15 @@ class FineGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkCont // uri is null. val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo.getCommand.getValue === s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") + 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}") + assert(executorInfo1.getCommand.getValue === s"""cd test-app-1*; "./bin/spark-class" ${classOf[MesosExecutorBackend].getName} """) } - test("mesos resource offers result in launching tasks") { + test("When Mesos resource offers are received, tasks are launched") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala index e57292c96f52e..ff45788469b8b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.util import java.util.Collections @@ -26,7 +26,6 @@ import org.apache.mesos.Protos.Value.Scalar import org.apache.mesos.Protos._ import org.apache.mesos.SchedulerDriver import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.mesos.{ CommonMesosSchedulerBackend, MemoryUtils } import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } import org.mockito.Matchers._ import org.mockito.Mockito._ diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala index 86a42a7398e4d..f2cb02a06870c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer From 108d2dc9f75f9461935e6c31d8e38ec0de177222 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Sat, 4 Apr 2015 08:15:44 -0500 Subject: [PATCH 15/18] Refactoring of the fine-grained mesos scheduler and tests. --- .../mesos/CommonMesosSchedulerBackend.scala | 2 - .../FineGrainedMesosSchedulerBackend.scala | 36 ++++---- ...ineGrainedMesosSchedulerBackendSuite.scala | 82 ++++++++----------- .../MesosSchedulerBackendSuiteHelper.scala | 38 +++++---- 4 files changed, 71 insertions(+), 87 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala index d8886b80cf9da..36d41c150d3be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala @@ -39,8 +39,6 @@ import org.apache.spark.util.{Utils, AkkaUtils} /** * Shared code between {@link FineGrainedMesosSchedulerBackend} and * {@link CoarseGrainedMesosSchedulerBackend}. - * TODO: This should really be private to the mesos package, but we need visibility - * in the tests, which are not in the same package. Fix the latter. */ trait CommonMesosSchedulerBackend extends SchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala index d69e2a4d534ad..03ef8aeb35c7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala @@ -150,24 +150,18 @@ private[spark] class FineGrainedMesosSchedulerBackend( val (usableOffers, unUsableOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") - val slaveId = o.getSlaveId.getValue + val minMemory = MemoryUtils.calculateTotalMemory(sparkContext) // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? - (mem >= MemoryUtils.calculateTotalMemory(sparkContext) && - // need at least 1 for executor, 1 for task - cpus >= 2 * scheduler.CPUS_PER_TASK) || - (slaveIdsWithExecutors.contains(slaveId) && - cpus >= scheduler.CPUS_PER_TASK) + (mem >= minMemory && cpus >= 2 * scheduler.CPUS_PER_TASK) || + (slaveHasExecutor(o) && cpus >= scheduler.CPUS_PER_TASK) } val workerOffers = usableOffers.map { o => - val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { - getResource(o.getResourcesList, "cpus").toInt - } else { - // If the executor doesn't exist yet, subtract CPU for executor - // TODO(pwendell): Should below just subtract "1"? - getResource(o.getResourcesList, "cpus").toInt - - scheduler.CPUS_PER_TASK - } + // If the executor doesn't exist yet, subtract CPU for executor + // TODO(pwendell): Should below just subtract "1"? + val cpus1 = getResource(o.getResourcesList, "cpus").toInt + val cpus = if (slaveHasExecutor(o)) cpus1 else cpus1 - scheduler.CPUS_PER_TASK + new WorkerOffer( o.getSlaveId.getValue, o.getHostname, @@ -199,11 +193,12 @@ private[spark] class FineGrainedMesosSchedulerBackend( val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? mesosTasks.foreach { case (slaveId, tasks) => - slaveIdToWorkerOffer.get(slaveId).foreach(o => - listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, + slaveIdToWorkerOffer.get(slaveId).foreach { o => + val executorAdded = SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, // TODO: Add support for log urls for Mesos - new ExecutorInfo(o.host, o.cores, Map.empty))) - ) + new ExecutorInfo(o.host, o.cores, Map.empty)) + listenerBus.post(executorAdded) + } d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } @@ -264,6 +259,11 @@ private[spark] class FineGrainedMesosSchedulerBackend( driver.reviveOffers() } + protected def slaveHasExecutor(o: Offer) = { + val slaveId = o.getSlaveId.getValue + slaveIdsWithExecutors.contains(slaveId) + } + /** * Remove executor associated with slaveId in a thread safe manner. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala index 1eb66c1985ce6..c1a5e92b05336 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala @@ -39,23 +39,26 @@ import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo -class FineGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { +class FineGrainedMesosSchedulerBackendSuite + extends FunSuite + with MesosSchedulerBackendSuiteHelper + with LocalSparkContext + with MockitoSugar { + + protected def makeTestMesosSchedulerBackend( + taskScheduler: TaskSchedulerImpl): FineGrainedMesosSchedulerBackend = { + new FineGrainedMesosSchedulerBackend(taskScheduler, taskScheduler.sc, "master") + } test("The spark-class location is correctly computed") { - val conf = new SparkConf - conf.set("spark.mesos.executor.home" , "/mesos-home") + val sc = makeMockSparkContext() + sc.conf.set("spark.mesos.executor.home" , "/mesos-home") - val listenerBus = mock[LiveListenerBus] - listenerBus.post( + sc.listenerBus.post( SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - val sc = mock[SparkContext] when(sc.getSparkHome()).thenReturn(Option("/spark-home")) - when(sc.conf).thenReturn(conf) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.executorMemory).thenReturn(100) - when(sc.listenerBus).thenReturn(listenerBus) val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.CPUS_PER_TASK).thenReturn(2) @@ -66,49 +69,27 @@ class FineGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkCont 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") + sc.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("When Mesos resource offers are received, tasks are launched") { - def createOffer(id: Int, mem: Int, cpu: Int) = { - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(mem)) - builder.addResourcesBuilder() - .setName("cpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() - } - - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + val (backend, driver) = makeBackendAndDriver() + val taskScheduler = backend.scheduler - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) + val sc = taskScheduler.sc // a mocked object when(sc.getSparkHome()).thenReturn(Option("/path")) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.conf).thenReturn(new SparkConf) - when(sc.listenerBus).thenReturn(listenerBus) - - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt - val minCpu = 4 + sc.listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host_s1", 2, Map.empty))) - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer(1, minMem, minCpu)) - mesosOffers.add(createOffer(2, minMem - 1, minCpu)) - mesosOffers.add(createOffer(3, minMem, minCpu)) + val (minMem, minCpu) = minMemMinCPU(sc) - val backend = new FineGrainedMesosSchedulerBackend(taskScheduler, sc, "master") + val mesosOffers = makeOffersList( + makeOffer("o1", "s1", minMem, minCpu), + makeOffer("o2", "s2", minMem - 1, minCpu), + makeOffer("o3", "s3", minMem, minCpu)) val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) expectedWorkerOffers.append(new WorkerOffer( @@ -145,17 +126,18 @@ class FineGrainedMesosSchedulerBackendSuite extends FunSuite with LocalSparkCont ) verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId) verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId) - assert(capture.getValue.size() == 1) + assert(capture.getValue.size() === 1) val taskInfo = capture.getValue.iterator().next() - assert(taskInfo.getName.equals("n1")) + assert(taskInfo.getName === "n1") val cpus = taskInfo.getResourcesList.get(0) - assert(cpus.getName.equals("cpus")) - assert(cpus.getScalar.getValue.equals(2.0)) - assert(taskInfo.getSlaveId.getValue.equals("s1")) + assert(cpus.getName === "cpus") + val actual = cpus.getScalar.getValue - 2.0 + val delta = 0.00001 + assert(actual >= - delta && actual <= delta) + assert(taskInfo.getSlaveId.getValue === "s1") // Unwanted resources offered on an existing node. Make sure they are declined - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer(1, minMem, minCpu)) + val mesosOffers2 = makeOffersList(makeOffer("o1", "s1", minMem, minCpu)) reset(taskScheduler) reset(driver) when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq())) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala index ff45788469b8b..40abb5b9df8ab 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -25,7 +25,7 @@ import com.typesafe.config.Config import org.apache.mesos.Protos.Value.Scalar import org.apache.mesos.Protos._ import org.apache.mesos.SchedulerDriver -import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl} +import org.apache.spark.scheduler.{ LiveListenerBus, SchedulerBackend, TaskSchedulerImpl } import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -38,7 +38,10 @@ import scala.collection.mutable trait MesosSchedulerBackendSuiteHelper { self: FunSuite with LocalSparkContext with MockitoSugar => - private def makeOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { + protected def makeTestMesosSchedulerBackend( + taskScheduler: TaskSchedulerImpl): CommonMesosSchedulerBackend + + protected def makeOffer(offerId: String, slaveId: String, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -49,16 +52,16 @@ trait MesosSchedulerBackendSuiteHelper { .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) builder.setId(OfferID.newBuilder().setValue(offerId).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host$slaveId").build() + .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host_$slaveId").build() } - private def makeOffersList(offers: Offer*): java.util.ArrayList[Offer] = { + protected def makeOffersList(offers: Offer*): java.util.ArrayList[Offer] = { val mesosOffers = new java.util.ArrayList[Offer] for (o <- offers) mesosOffers.add(o) mesosOffers } - private def mockSparkContext: SparkContext = { + protected def makeMockSparkContext(): SparkContext = { val sparkConf = new SparkConf sparkConf.set("spark.driver.host", "driverHost") sparkConf.set("spark.driver.port", "1234") @@ -71,42 +74,43 @@ trait MesosSchedulerBackendSuiteHelper { when(sc.executorEnvs).thenReturn(emptyHashMap) when(sc.conf).thenReturn(sparkConf) when(sc.env).thenReturn(se) + + val listenerBus = mock[LiveListenerBus] + when(sc.listenerBus).thenReturn(listenerBus) + sc } - private def mockEnvironment: (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { - val sc = mockSparkContext + protected def makeMockEnvironment(): (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { + val sc = makeMockSparkContext() val driver = mock[SchedulerDriver] val taskScheduler = mock[TaskSchedulerImpl] when(taskScheduler.sc).thenReturn(sc) (sc, taskScheduler, driver) } - private def makeBackendAndDriver: (CommonMesosSchedulerBackend, SchedulerDriver) = { - val (sc, taskScheduler, driver) = mockEnvironment + protected def makeBackendAndDriver(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (sc, taskScheduler, driver) = makeMockEnvironment() val backend = makeTestMesosSchedulerBackend(taskScheduler) backend.driver = driver (backend, driver) } - private def makeTaskID( id: String): TaskID = TaskID.newBuilder().setValue(id).build() - private def makeSlaveID(id: String): SlaveID = SlaveID.newBuilder().setValue(id).build() - private def makeOfferID(id: String): OfferID = OfferID.newBuilder().setValue(id).build() + protected def makeTaskID( id: String): TaskID = TaskID.newBuilder().setValue(id).build() + protected def makeSlaveID(id: String): SlaveID = SlaveID.newBuilder().setValue(id).build() + protected def makeOfferID(id: String): OfferID = OfferID.newBuilder().setValue(id).build() // Simulate task killed message, signaling that an executor is no longer running. - private def makeKilledTaskStatus(taskId: String, slaveId: String) = + protected def makeKilledTaskStatus(taskId: String, slaveId: String) = TaskStatus.newBuilder() .setTaskId(makeTaskID(taskId)) .setSlaveId(makeSlaveID(slaveId)) .setState(TaskState.TASK_KILLED) .build - private def minMemMinCPU(sc: SparkContext, extraMemory: Int = 0, numCores: Int = 4): (Int,Int) = + protected def minMemMinCPU(sc: SparkContext, extraMemory: Int = 0, numCores: Int = 4): (Int,Int) = (MemoryUtils.calculateTotalMemory(sc).toInt + extraMemory, numCores) - protected def makeTestMesosSchedulerBackend( - taskScheduler: TaskSchedulerImpl): CommonMesosSchedulerBackend - val (taskIDVal1, slaveIDVal1) = ("0", "s1") val (taskIDVal2, slaveIDVal2) = ("1", "s2") From 15f0ec0e51c0527a7391e9c873ba2460ac9afde1 Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Sat, 4 Apr 2015 08:54:36 -0500 Subject: [PATCH 16/18] Refactored the test. --- ...ineGrainedMesosSchedulerBackendSuite.scala | 80 +++++++++++++------ .../MesosSchedulerBackendSuiteHelper.scala | 6 +- 2 files changed, 60 insertions(+), 26 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala index c1a5e92b05336..26159c9c4b3ef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer import java.util import java.util.Collections +import java.util.{ ArrayList => JArrayList } import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -50,6 +51,16 @@ class FineGrainedMesosSchedulerBackendSuite new FineGrainedMesosSchedulerBackend(taskScheduler, taskScheduler.sc, "master") } + def makeTestOffers(sc: SparkContext): (Offer, Offer, Offer, Offer) = { + val (minMem, minCpu) = minMemMinCPU(sc) + val goodOffer1 = makeOffer("o1", "s1", minMem, minCpu) + val badOffer1 = makeOffer("o2", "s2", minMem - 1, minCpu) // memory will be too small. + val goodOffer2 = makeOffer("o3", "s3", minMem, minCpu) + val badOffer2 = makeOffer("o4", "s4", minMem, minCpu - 1) // CPUs will be too small. + (goodOffer1, badOffer1, goodOffer2, badOffer2) + } + + test("The spark-class location is correctly computed") { val sc = makeMockSparkContext() sc.conf.set("spark.mesos.executor.home" , "/mesos-home") @@ -74,8 +85,8 @@ class FineGrainedMesosSchedulerBackendSuite assert(executorInfo1.getCommand.getValue === s"""cd test-app-1*; "./bin/spark-class" ${classOf[MesosExecutorBackend].getName} """) } - test("When Mesos resource offers are received, tasks are launched") { - + protected def offerResourcesHelper(): + (CommonMesosSchedulerBackend, SchedulerDriver, ArgumentCaptor[util.Collection[TaskInfo]], JArrayList[Offer]) = { val (backend, driver) = makeBackendAndDriver() val taskScheduler = backend.scheduler @@ -84,24 +95,22 @@ class FineGrainedMesosSchedulerBackendSuite sc.listenerBus.post( SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host_s1", 2, Map.empty))) - val (minMem, minCpu) = minMemMinCPU(sc) - - val mesosOffers = makeOffersList( - makeOffer("o1", "s1", minMem, minCpu), - makeOffer("o2", "s2", minMem - 1, minCpu), - makeOffer("o3", "s3", minMem, minCpu)) + val (goodOffer1, badOffer1, goodOffer2, badOffer2) = makeTestOffers(sc) + val mesosOffers = makeOffersList(goodOffer1, badOffer1, goodOffer2, badOffer2) val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, + goodOffer1.getSlaveId.getValue, + goodOffer1.getHostname, 2 )) expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(2).getSlaveId.getValue, - mesosOffers.get(2).getHostname, + goodOffer2.getSlaveId.getValue, + goodOffer2.getHostname, 2 )) + + // The mock taskScheduler will only accept the first offer. val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) @@ -109,23 +118,31 @@ class FineGrainedMesosSchedulerBackendSuite val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) when( driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + Matchers.eq(Collections.singleton(goodOffer1.getId)), capture.capture(), any(classOf[Filters]) ) ).thenReturn(Status.valueOf(1)) - when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1)) - when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(badOffer1.getId)).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(goodOffer2.getId)).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(badOffer2.getId)).thenReturn(Status.valueOf(1)) backend.resourceOffers(driver, mesosOffers) + (backend, driver, capture, mesosOffers) + } + + test("When acceptable Mesos resource offers are received, tasks are launched for for them") { + + val (backend, driver, capture, mesosOffers) = offerResourcesHelper() + val goodOffer1 = mesosOffers.get(0) + verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + Matchers.eq(Collections.singleton(goodOffer1.getId)), capture.capture(), any(classOf[Filters]) ) - verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId) - verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId) + assert(capture.getValue.size() === 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName === "n1") @@ -135,16 +152,33 @@ class FineGrainedMesosSchedulerBackendSuite val delta = 0.00001 assert(actual >= - delta && actual <= delta) assert(taskInfo.getSlaveId.getValue === "s1") + } + + test("When unacceptable Mesos resource offers are received, no tasks are launched for them") { + + val (backend, driver, capture, mesosOffers) = offerResourcesHelper() + val badOffer1 = mesosOffers.get(1) + val goodOffer2 = mesosOffers.get(2) + val badOffer2 = mesosOffers.get(3) - // Unwanted resources offered on an existing node. Make sure they are declined - val mesosOffers2 = makeOffersList(makeOffer("o1", "s1", minMem, minCpu)) + verify(driver, times(1)).declineOffer(badOffer1.getId) + verify(driver, times(1)).declineOffer(goodOffer2.getId) + verify(driver, times(1)).declineOffer(badOffer2.getId) + } + + test("When acceptable Mesos resource offers are received for a node that already has an executor, they are declined") { + + val (backend, driver, capture, mesosOffers) = offerResourcesHelper() + val goodOffer1 = mesosOffers.get(0) + val taskScheduler = backend.scheduler reset(taskScheduler) reset(driver) + when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq())) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(goodOffer1.getId)).thenReturn(Status.valueOf(1)) - backend.resourceOffers(driver, mesosOffers2) - verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId) + backend.resourceOffers(driver, makeOffersList(goodOffer1)) + verify(driver, times(1)).declineOffer(goodOffer1.getId) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala index 40abb5b9df8ab..e2ec4fa887f21 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.util import java.util.Collections - +import java.util.{ ArrayList => JArrayList } import akka.actor.ActorSystem import com.typesafe.config.Config import org.apache.mesos.Protos.Value.Scalar @@ -55,8 +55,8 @@ trait MesosSchedulerBackendSuiteHelper { .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host_$slaveId").build() } - protected def makeOffersList(offers: Offer*): java.util.ArrayList[Offer] = { - val mesosOffers = new java.util.ArrayList[Offer] + protected def makeOffersList(offers: Offer*): JArrayList[Offer] = { + val mesosOffers = new JArrayList[Offer] for (o <- offers) mesosOffers.add(o) mesosOffers } From 74de3e8e6ec667040bc8a4094035e5e3678e1c1b Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Sat, 4 Apr 2015 09:36:01 -0500 Subject: [PATCH 17/18] Final refactoring before code review. --- ...rseGrainedMesosSchedulerBackendSuite.scala | 119 +++++++++++++++++- ...ineGrainedMesosSchedulerBackendSuite.scala | 8 +- .../MesosSchedulerBackendSuiteHelper.scala | 116 ----------------- 3 files changed, 123 insertions(+), 120 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala index c17a413f2dc18..e7c1fbd3278f5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseGrainedMesosSchedulerBackendSuite.scala @@ -48,8 +48,125 @@ class CoarseGrainedMesosSchedulerBackendSuite } } + val (taskIDVal1, slaveIDVal1) = ("0", "s1") + val (taskIDVal2, slaveIDVal2) = ("1", "s2") + + def makeMesosExecutorsTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (backend, driver) = makeBackendAndDriver + + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) + + val mesosOffers1 = makeOffersList(makeOffer(taskIDVal1, slaveIDVal1, minMem, minCPU)) + + backend.resourceOffers(driver, mesosOffers1) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers1.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) + + // Verify we have one executor and the executor limit is 1. + assert(backend.slaveIdsWithExecutors.size === 1) + assert(backend.getExecutorLimit >= 1) + + (backend, driver) // Return so this test can be embedded in others. + } + + def killMesosExecutorDeprecateByOneTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (backend, driver) = makeMesosExecutorsTest() + + // Calling doKillExecutors should invoke driver.killTask. + val taskID1 = makeTaskID(taskIDVal1) + assert(backend.doKillExecutors(Seq(s"$slaveIDVal1/$taskIDVal1"))) + verify(driver, times(1)).killTask(taskID1) + // Must invoke the status update explicitly here. + // TODO: can we mock other parts of the API so this can be called automatically? + backend.statusUpdate(driver, makeKilledTaskStatus(taskIDVal1, slaveIDVal1)) + + // Verify we don't have any executors. + assert(backend.slaveIdsWithExecutors.size === 0) + // Verify that the executor limit is now 0. + assert(backend.getExecutorLimit === 0) + + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) + val mesosOffers2 = makeOffersList(makeOffer(taskIDVal2, slaveIDVal2, minMem, minCPU)) + backend.resourceOffers(driver, mesosOffers2) + + verify(driver, times(1)) + .declineOffer(makeOfferID(taskIDVal2)) + + // Verify we didn't launch any new executor + assert(backend.slaveIdsWithExecutors.size === 0) + assert(backend.getExecutorLimit === 0) + + (backend, driver) // Return so this test can be embedded in others. + } + + def increaseAllowedMesosExecutorsTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { + val (backend, driver) = killMesosExecutorDeprecateByOneTest() + + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) + val mesosOffers2 = makeOffersList(makeOffer(taskIDVal2, slaveIDVal2, minMem, minCPU)) + + // Now allow one more executor: + backend.requestExecutors(1) + backend.resourceOffers(driver, mesosOffers2) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) + + assert(backend.slaveIdsWithExecutors.size === 1) + assert(backend.getExecutorLimit >= 1) + + (backend, driver) // Return so this test can be embedded in others. + } + + def slaveLostDoesntChangeMaxAllowedMesosExecutorsTest(): Unit = { + val (backend, driver) = increaseAllowedMesosExecutorsTest() + + backend.slaveLost(driver, makeSlaveID(slaveIDVal2)) + assert(backend.slaveIdsWithExecutors.size === 0) + assert(backend.getExecutorLimit >= 1) + } + + def killAndRelaunchTasksTest(): Unit = { + val (backend, driver) = makeBackendAndDriver + val (minMem, minCPU) = minMemMinCPU(backend.sparkContext, 1024) + val offer1 = makeOffer(taskIDVal1, slaveIDVal1, minMem, minCPU) + val mesosOffers = makeOffersList(offer1) + + backend.resourceOffers(driver, mesosOffers) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer1.getId)), + anyObject(), + anyObject[Filters]) + assert(backend.slaveIdsWithExecutors.contains(slaveIDVal1)) + + backend.statusUpdate(driver, makeKilledTaskStatus(taskIDVal1, slaveIDVal1)) + assert(!backend.slaveIdsWithExecutors.contains(slaveIDVal1)) + assert(backend.slaveIdsWithExecutors.size === 0) + assert(backend.getExecutorLimit >= 1) + + val offer2 = makeOffer(taskIDVal2, slaveIDVal2, minMem, 1) + mesosOffers.clear() + mesosOffers.add(offer2) + backend.resourceOffers(driver, mesosOffers) + assert(!backend.slaveIdsWithExecutors.contains(slaveIDVal1)) + assert( backend.slaveIdsWithExecutors.contains(slaveIDVal2)) + assert(backend.slaveIdsWithExecutors.size === 1) + assert(backend.getExecutorLimit >= 1) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer2.getId)), + anyObject(), + anyObject[Filters]) + + verify(driver, times(1)).reviveOffers() + } + test("When Mesos offers resources, a Mesos executor is created.") { - makeMesosExecutorsTest + makeMesosExecutorsTest() } test("When a Mesos executor is killed, the maximum number of allowed Mesos executors is deprecated by one") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala index 26159c9c4b3ef..a21501ed29950 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala @@ -87,8 +87,10 @@ class FineGrainedMesosSchedulerBackendSuite protected def offerResourcesHelper(): (CommonMesosSchedulerBackend, SchedulerDriver, ArgumentCaptor[util.Collection[TaskInfo]], JArrayList[Offer]) = { + val (backend, driver) = makeBackendAndDriver() val taskScheduler = backend.scheduler + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) val sc = taskScheduler.sc // a mocked object when(sc.getSparkHome()).thenReturn(Option("/path")) @@ -111,9 +113,9 @@ class FineGrainedMesosSchedulerBackendSuite )) // The mock taskScheduler will only accept the first offer. - val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) + val expectedTaskDescriptions = Seq(new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))) + + when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(expectedTaskDescriptions)) val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) when( diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala index e2ec4fa887f21..1f3dbe69fc49e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -111,120 +111,4 @@ trait MesosSchedulerBackendSuiteHelper { protected def minMemMinCPU(sc: SparkContext, extraMemory: Int = 0, numCores: Int = 4): (Int,Int) = (MemoryUtils.calculateTotalMemory(sc).toInt + extraMemory, numCores) - val (taskIDVal1, slaveIDVal1) = ("0", "s1") - val (taskIDVal2, slaveIDVal2) = ("1", "s2") - - def makeMesosExecutorsTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { - val (backend, driver) = makeBackendAndDriver - - val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) - - val mesosOffers1 = makeOffersList(makeOffer(taskIDVal1, slaveIDVal1, minMem, minCPU)) - - backend.resourceOffers(driver, mesosOffers1) - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers1.get(0).getId)), - any[util.Collection[TaskInfo]], - any[Filters]) - - // Verify we have one executor and the executor limit is 1. - assert(backend.slaveIdsWithExecutors.size === 1) - assert(backend.getExecutorLimit >= 1) - - (backend, driver) // Return so this test can be embedded in others. - } - - def killMesosExecutorDeprecateByOneTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { - val (backend, driver) = makeMesosExecutorsTest() - - // Calling doKillExecutors should invoke driver.killTask. - val taskID1 = makeTaskID(taskIDVal1) - assert(backend.doKillExecutors(Seq(s"$slaveIDVal1/$taskIDVal1"))) - verify(driver, times(1)).killTask(taskID1) - // Must invoke the status update explicitly here. - // TODO: can we mock other parts of the API so this can be called automatically? - backend.statusUpdate(driver, makeKilledTaskStatus(taskIDVal1, slaveIDVal1)) - - // Verify we don't have any executors. - assert(backend.slaveIdsWithExecutors.size === 0) - // Verify that the executor limit is now 0. - assert(backend.getExecutorLimit === 0) - - val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) - val mesosOffers2 = makeOffersList(makeOffer(taskIDVal2, slaveIDVal2, minMem, minCPU)) - backend.resourceOffers(driver, mesosOffers2) - - verify(driver, times(1)) - .declineOffer(makeOfferID(taskIDVal2)) - - // Verify we didn't launch any new executor - assert(backend.slaveIdsWithExecutors.size === 0) - assert(backend.getExecutorLimit === 0) - - (backend, driver) // Return so this test can be embedded in others. - } - - def increaseAllowedMesosExecutorsTest(): (CommonMesosSchedulerBackend, SchedulerDriver) = { - val (backend, driver) = killMesosExecutorDeprecateByOneTest() - - val (minMem, minCPU) = minMemMinCPU(backend.sparkContext) - val mesosOffers2 = makeOffersList(makeOffer(taskIDVal2, slaveIDVal2, minMem, minCPU)) - - // Now allow one more executor: - backend.requestExecutors(1) - backend.resourceOffers(driver, mesosOffers2) - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), - any[util.Collection[TaskInfo]], - any[Filters]) - - assert(backend.slaveIdsWithExecutors.size === 1) - assert(backend.getExecutorLimit >= 1) - - (backend, driver) // Return so this test can be embedded in others. - } - - def slaveLostDoesntChangeMaxAllowedMesosExecutorsTest(): Unit = { - val (backend, driver) = increaseAllowedMesosExecutorsTest() - - backend.slaveLost(driver, makeSlaveID(slaveIDVal2)) - assert(backend.slaveIdsWithExecutors.size === 0) - assert(backend.getExecutorLimit >= 1) - } - - def killAndRelaunchTasksTest(): Unit = { - val (backend, driver) = makeBackendAndDriver - val (minMem, minCPU) = minMemMinCPU(backend.sparkContext, 1024) - val offer1 = makeOffer(taskIDVal1, slaveIDVal1, minMem, minCPU) - val mesosOffers = makeOffersList(offer1) - - backend.resourceOffers(driver, mesosOffers) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer1.getId)), - anyObject(), - anyObject[Filters]) - assert(backend.slaveIdsWithExecutors.contains(slaveIDVal1)) - - backend.statusUpdate(driver, makeKilledTaskStatus(taskIDVal1, slaveIDVal1)) - assert(!backend.slaveIdsWithExecutors.contains(slaveIDVal1)) - assert(backend.slaveIdsWithExecutors.size === 0) - assert(backend.getExecutorLimit >= 1) - - val offer2 = makeOffer(taskIDVal2, slaveIDVal2, minMem, 1) - mesosOffers.clear() - mesosOffers.add(offer2) - backend.resourceOffers(driver, mesosOffers) - assert(!backend.slaveIdsWithExecutors.contains(slaveIDVal1)) - assert( backend.slaveIdsWithExecutors.contains(slaveIDVal2)) - assert(backend.slaveIdsWithExecutors.size === 1) - assert(backend.getExecutorLimit >= 1) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer2.getId)), - anyObject(), - anyObject[Filters]) - - verify(driver, times(1)).reviveOffers() - } } From e5ab37dc6a22987a8b0bd5ceac6614226b57821f Mon Sep 17 00:00:00 2001 From: Dean Wampler Date: Mon, 6 Apr 2015 16:12:41 -0500 Subject: [PATCH 18/18] More tests for fine-grained dynamic allocation. --- .../mesos/CommonMesosSchedulerBackend.scala | 4 + .../FineGrainedMesosSchedulerBackend.scala | 12 +-- ...ineGrainedMesosSchedulerBackendSuite.scala | 78 +++++++++++++++---- .../MesosSchedulerBackendSuiteHelper.scala | 20 +++-- 4 files changed, 85 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala index 36d41c150d3be..6635ca6f7546f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CommonMesosSchedulerBackend.scala @@ -73,6 +73,10 @@ trait CommonMesosSchedulerBackend extends SchedulerBackend { private[mesos] val slaveIdsWithExecutors = MutableHashSet.empty[String] + def slaveHasExecutor(slaveId: String) = { + slaveIdsWithExecutors.contains(slaveId) + } + private def executorBackendName: String = executorBackend.getName private def executorSimpleBackendName: String = executorBackend.getSimpleName diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala index 03ef8aeb35c7b..27e719e633926 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackend.scala @@ -148,20 +148,21 @@ private[spark] class FineGrainedMesosSchedulerBackend( inClassLoader() { // Fail-fast on offers we know will be rejected val (usableOffers, unUsableOffers) = offers.partition { o => + val slaveId = o.getSlaveId.getValue val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val minMemory = MemoryUtils.calculateTotalMemory(sparkContext) // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= minMemory && cpus >= 2 * scheduler.CPUS_PER_TASK) || - (slaveHasExecutor(o) && cpus >= scheduler.CPUS_PER_TASK) + (slaveHasExecutor(slaveId) && cpus >= scheduler.CPUS_PER_TASK) } val workerOffers = usableOffers.map { o => // If the executor doesn't exist yet, subtract CPU for executor // TODO(pwendell): Should below just subtract "1"? + val slaveId = o.getSlaveId.getValue val cpus1 = getResource(o.getResourcesList, "cpus").toInt - val cpus = if (slaveHasExecutor(o)) cpus1 else cpus1 - scheduler.CPUS_PER_TASK - + val cpus = if (slaveHasExecutor(slaveId)) cpus1 else cpus1 - scheduler.CPUS_PER_TASK new WorkerOffer( o.getSlaveId.getValue, o.getHostname, @@ -259,11 +260,6 @@ private[spark] class FineGrainedMesosSchedulerBackend( driver.reviveOffers() } - protected def slaveHasExecutor(o: Offer) = { - val slaveId = o.getSlaveId.getValue - slaveIdsWithExecutors.contains(slaveId) - } - /** * Remove executor associated with slaveId in a thread safe manner. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala index a21501ed29950..4faac8d6662ef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/FineGrainedMesosSchedulerBackendSuite.scala @@ -28,6 +28,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.mesos.SchedulerDriver import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.Scalar +import org.apache.mesos.Protos.{TaskState => MesosTaskState} import org.mockito.Mockito._ import org.mockito.Matchers._ import org.mockito.{ArgumentCaptor, Matchers} @@ -51,7 +52,7 @@ class FineGrainedMesosSchedulerBackendSuite new FineGrainedMesosSchedulerBackend(taskScheduler, taskScheduler.sc, "master") } - def makeTestOffers(sc: SparkContext): (Offer, Offer, Offer, Offer) = { + protected def makeTestOffers(sc: SparkContext): (Offer, Offer, Offer, Offer) = { val (minMem, minCpu) = minMemMinCPU(sc) val goodOffer1 = makeOffer("o1", "s1", minMem, minCpu) val badOffer1 = makeOffer("o2", "s2", minMem - 1, minCpu) // memory will be too small. @@ -60,6 +61,18 @@ class FineGrainedMesosSchedulerBackendSuite (goodOffer1, badOffer1, goodOffer2, badOffer2) } + protected def checkLaunchTask( + driver: SchedulerDriver, offer: Offer, expectedValue: Int): ArgumentCaptor[util.Collection[TaskInfo]] = { + val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) + when( + driver.launchTasks( + Matchers.eq(Collections.singleton(offer.getId)), + capture.capture(), + any(classOf[Filters]) + ) + ).thenReturn(Status.valueOf(expectedValue)) + capture + } test("The spark-class location is correctly computed") { val sc = makeMockSparkContext() @@ -85,6 +98,11 @@ class FineGrainedMesosSchedulerBackendSuite assert(executorInfo1.getCommand.getValue === s"""cd test-app-1*; "./bin/spark-class" ${classOf[MesosExecutorBackend].getName} """) } + // The mock taskScheduler will only accept the first offer. + private val expectedTaskId1 = 1L + private val expectedTaskDescriptions = + Seq(Seq(new TaskDescription(expectedTaskId1, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))))) + protected def offerResourcesHelper(): (CommonMesosSchedulerBackend, SchedulerDriver, ArgumentCaptor[util.Collection[TaskInfo]], JArrayList[Offer]) = { @@ -112,19 +130,9 @@ class FineGrainedMesosSchedulerBackendSuite 2 )) - // The mock taskScheduler will only accept the first offer. - val expectedTaskDescriptions = Seq(new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))) - - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(expectedTaskDescriptions)) + when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(expectedTaskDescriptions) - val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) - when( - driver.launchTasks( - Matchers.eq(Collections.singleton(goodOffer1.getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).thenReturn(Status.valueOf(1)) + val capture = checkLaunchTask(driver, goodOffer1, 1) when(driver.declineOffer(badOffer1.getId)).thenReturn(Status.valueOf(1)) when(driver.declineOffer(goodOffer2.getId)).thenReturn(Status.valueOf(1)) when(driver.declineOffer(badOffer2.getId)).thenReturn(Status.valueOf(1)) @@ -134,7 +142,7 @@ class FineGrainedMesosSchedulerBackendSuite (backend, driver, capture, mesosOffers) } - test("When acceptable Mesos resource offers are received, tasks are launched for for them") { + test("When acceptable Mesos resource offers are received, tasks are launched for them") { val (backend, driver, capture, mesosOffers) = offerResourcesHelper() val goodOffer1 = mesosOffers.get(0) @@ -173,14 +181,52 @@ class FineGrainedMesosSchedulerBackendSuite val (backend, driver, capture, mesosOffers) = offerResourcesHelper() val goodOffer1 = mesosOffers.get(0) val taskScheduler = backend.scheduler - reset(taskScheduler) + resetTaskScheduler(taskScheduler) reset(driver) when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq())) - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) when(driver.declineOffer(goodOffer1.getId)).thenReturn(Status.valueOf(1)) backend.resourceOffers(driver, makeOffersList(goodOffer1)) verify(driver, times(1)).declineOffer(goodOffer1.getId) } + + test("When acceptable Mesos resource offers are received for a node that had an executor that is now gone, they are accepted") { + + val (backend, driver, capture, mesosOffers) = offerResourcesHelper() + val goodOffer1 = mesosOffers.get(0) + val goodOffer2 = mesosOffers.get(1) + val slaveId1 = goodOffer1.getSlaveId.getValue + val taskScheduler = backend.scheduler + + resetTaskScheduler(taskScheduler) + reset(driver) + + // First, reconfirm that offers are rejected while the executor exists. + when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq())) + when(driver.declineOffer(goodOffer1.getId)).thenReturn(Status.valueOf(1)) + + backend.resourceOffers(driver, makeOffersList(goodOffer1)) + verify(driver, times(1)).declineOffer(goodOffer1.getId) + + // Now, kill the executor, re-offer, and confirm an offer is now accepted (again). + resetTaskScheduler(taskScheduler) + reset(driver) + + val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1) + expectedWorkerOffers.append(new WorkerOffer( + goodOffer1.getSlaveId.getValue, + goodOffer1.getHostname, + 2 + )) + + when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(expectedTaskDescriptions) + + backend.statusUpdate(driver, makeKilledTaskStatus(expectedTaskId1.toString, slaveId1, MesosTaskState.TASK_LOST)) + assert(backend.slaveHasExecutor(slaveId1) === false) + checkLaunchTask(driver, goodOffer1, 1) + + backend.resourceOffers(driver, makeOffersList(goodOffer1)) + verify(driver, times(0)).declineOffer(goodOffer1.getId) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala index 1f3dbe69fc49e..759c5465533a4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuiteHelper.scala @@ -23,7 +23,7 @@ import java.util.{ ArrayList => JArrayList } import akka.actor.ActorSystem import com.typesafe.config.Config import org.apache.mesos.Protos.Value.Scalar -import org.apache.mesos.Protos._ +import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.SchedulerDriver import org.apache.spark.scheduler.{ LiveListenerBus, SchedulerBackend, TaskSchedulerImpl } import org.apache.spark.{ LocalSparkContext, SparkConf, SparkEnv, SparkContext } @@ -51,8 +51,10 @@ trait MesosSchedulerBackendSuiteHelper { .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(offerId).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId)).setHostname(s"host_$slaveId").build() + builder.setId(OfferID.newBuilder().setValue(offerId).build()) + .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(slaveId)) + .setHostname(s"host_$slaveId").build() } protected def makeOffersList(offers: Offer*): JArrayList[Offer] = { @@ -81,6 +83,14 @@ trait MesosSchedulerBackendSuiteHelper { sc } + protected def resetTaskScheduler(taskScheduler: TaskSchedulerImpl): TaskSchedulerImpl = { + val sc = taskScheduler.sc + reset(taskScheduler) + when(taskScheduler.sc).thenReturn(sc) + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) + taskScheduler + } + protected def makeMockEnvironment(): (SparkContext, TaskSchedulerImpl, SchedulerDriver) = { val sc = makeMockSparkContext() val driver = mock[SchedulerDriver] @@ -101,11 +111,11 @@ trait MesosSchedulerBackendSuiteHelper { protected def makeOfferID(id: String): OfferID = OfferID.newBuilder().setValue(id).build() // Simulate task killed message, signaling that an executor is no longer running. - protected def makeKilledTaskStatus(taskId: String, slaveId: String) = + protected def makeKilledTaskStatus(taskId: String, slaveId: String, state: MesosTaskState = MesosTaskState.TASK_KILLED) = TaskStatus.newBuilder() .setTaskId(makeTaskID(taskId)) .setSlaveId(makeSlaveID(slaveId)) - .setState(TaskState.TASK_KILLED) + .setState(state) .build protected def minMemMinCPU(sc: SparkContext, extraMemory: Int = 0, numCores: Int = 4): (Int,Int) =