From 2494e77729ca56b6b6600608e28024272c2acbbc Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Mon, 15 Apr 2019 20:22:57 +0800 Subject: [PATCH 01/18] add gpu support to job scheduler --- .../org/apache/spark/BarrierTaskContext.scala | 2 + .../scala/org/apache/spark/SparkConf.scala | 123 ++++++- .../scala/org/apache/spark/TaskContext.scala | 8 +- .../org/apache/spark/TaskContextImpl.scala | 3 +- .../CoarseGrainedExecutorBackend.scala | 13 +- .../org/apache/spark/executor/Executor.scala | 3 +- .../spark/internal/config/package.scala | 7 +- .../SchedulerResourceInformation.scala | 68 ++++ .../org/apache/spark/scheduler/Task.scala | 7 +- .../spark/scheduler/TaskDescription.scala | 47 ++- .../spark/scheduler/TaskSchedulerImpl.scala | 23 +- .../spark/scheduler/TaskSetManager.scala | 15 +- .../apache/spark/scheduler/WorkerOffer.scala | 3 +- .../cluster/CoarseGrainedClusterMessage.scala | 15 +- .../CoarseGrainedSchedulerBackend.scala | 35 +- .../scheduler/cluster/ExecutorData.scala | 12 +- .../scheduler/cluster/ExecutorInfo.scala | 11 +- .../local/LocalSchedulerBackend.scala | 5 +- .../spark/JavaTaskContextCompileCheck.java | 4 + ...che.spark.scheduler.ExternalClusterManager | 1 + .../org/apache/spark/SparkConfSuite.scala | 79 +++++ .../org/apache/spark/SparkContextSuite.scala | 32 ++ .../apache/spark/executor/ExecutorSuite.scala | 2 + .../CoarseGrainedSchedulerBackendSuite.scala | 148 ++++++++- .../spark/scheduler/TaskContextSuite.scala | 4 +- .../scheduler/TaskDescriptionSuite.scala | 18 + .../scheduler/TaskSchedulerImplSuite.scala | 36 +- .../spark/scheduler/TaskSetManagerSuite.scala | 309 ++++++++++++------ project/MimaExcludes.scala | 3 + ...esosFineGrainedSchedulerBackendSuite.scala | 6 +- 30 files changed, 895 insertions(+), 147 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index a354f44a1be19..cf957ffcec9ad 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -185,6 +185,8 @@ class BarrierTaskContext private[spark] ( taskContext.getMetricsSources(sourceName) } + override def resources(): Map[String, ResourceInformation] = taskContext.resources() + override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted() override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason() diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 15f1730ca483f..97f37ff966808 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -21,7 +21,7 @@ import java.util.{Map => JMap} import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import scala.collection.mutable.LinkedHashSet +import scala.collection.mutable.{HashMap, LinkedHashSet} import org.apache.avro.{Schema, SchemaNormalization} @@ -507,6 +507,101 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } + /** + * Set available resources on the driver/executor. + */ + private[spark] def setResources( + resourceName: String, + resourceCount: Int, + resourceAddresses: Option[Seq[String]], + isDriver: Boolean): SparkConf = { + if (resourceAddresses.isDefined && resourceAddresses.get.size != resourceCount) { + val nodeType = if (isDriver) { + "driver" + } else { + "executor" + } + throw new SparkException(s"Specified $resourceCount $resourceName(s) on $nodeType, while " + + s"the size of device addresses is ${resourceAddresses.size}.") + } + val prefix = if (isDriver) { + s"$SPARK_DRIVER_RESOURCE_PREFIX.$resourceName" + } else { + s"$SPARK_EXECUTOR_RESOURCE_PREFIX.$resourceName" + } + set(s"$prefix.$SPARK_RESOURCE_COUNT", resourceCount.toString) + if (resourceAddresses.isDefined) { + set(s"$prefix.$SPARK_RESOURCE_ADDRESSES", resourceAddresses.get.mkString(",")) + } + this + } + + /** + * Set task resource requirement. + */ + private[spark] def setResourceRequirement( + resourceName: String, resourceCount: Int): SparkConf = { + val key = s"$SPARK_TASK_RESOURCE_PREFIX.$resourceName.$SPARK_RESOURCE_COUNT" + set(key, resourceCount.toString) + this + } + + /** + * Get available resources on the driver/executor. + */ + private[spark] def getResources(isDriver: Boolean): Map[String, ResourceInformation] = { + val prefix = if (isDriver) { + s"$SPARK_DRIVER_RESOURCE_PREFIX." + } else { + s"$SPARK_EXECUTOR_RESOURCE_PREFIX." + } + + val resourceCountMap = new HashMap[String, Int] + val resourceAddressMap = new HashMap[String, Array[String]] + getAllWithPrefix(prefix).foreach { tuple => + val keys = tuple._1.split("\\.") + if (keys.size == 2) { + val resourceName = keys.head + if (keys.last.equals(SPARK_RESOURCE_COUNT)) { + val resourceCount = tuple._2.toInt + resourceCountMap.put(resourceName, resourceCount) + } else if (keys.last.equals(SPARK_RESOURCE_ADDRESSES)) { + val resourceAddresses = tuple._2.split(",").map(_.trim()) + resourceAddressMap.put(resourceName, resourceAddresses) + } + } + } + + resourceCountMap.map { case (resourceName, resourceCount) => + if (resourceAddressMap.contains(resourceName)) { + val resourceAddresses = resourceAddressMap.get(resourceName).get + (resourceName, new ResourceInformation(resourceName, "", resourceCount, resourceAddresses)) + } else { + (resourceName, new ResourceInformation(resourceName, "", resourceCount, Array.empty)) + } + }.toMap + } + + /** + * Get task resource requirements. + */ + private[spark] def getResourceRequirements(): Map[String, Int] = { + getAllWithPrefix(s"$SPARK_TASK_RESOURCE_PREFIX.").map { tuple => + val keys = tuple._1.split("\\.") + val resourceName = keys.head + if (keys.size == 2) { + if (keys.last.equals(SPARK_RESOURCE_COUNT)) { + val resourceCount = tuple._2.toInt + (resourceName, resourceCount) + } else { + (resourceName, 0) + } + } else { + (resourceName, 0) + } + }.filter(_._2 > 0).toMap + } + /** * Checks for illegal or deprecated config settings. Throws an exception for the former. Not * idempotent - may mutate this conf object to convert deprecated settings to supported ones. @@ -591,6 +686,32 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } + val resources = getResources(false) + val resourceRequirements = getResourceRequirements() + val executorCores = get(EXECUTOR_CORES) + val taskCpus = get(CPUS_PER_TASK) + resourceRequirements.foreach { case (resourceName, resourceCount) => + if (!resources.contains(resourceName)) { + throw new SparkException(s"Task requires resource type $resourceName, which is not " + + s"available on executors.") + } else { + val resourceInformation = resources.get(resourceName).get + val executorResourceCount = resourceInformation.getCount() + if (executorResourceCount < resourceCount) { + throw new SparkException(s"Available $resourceName resource count on executors is " + + s"$executorResourceCount, which is less than the task requirement $resourceCount.") + } + + if (contains(CPUS_PER_TASK) && contains(EXECUTOR_CORES)) { + if (executorCores > 0 && taskCpus > 0 && + executorCores * resourceCount != executorResourceCount * taskCpus) { + throw new SparkException("Can't make full use of the resources allocated to each " + + "executor with current task resource requirements") + } + } + } + } + val encryptionEnabled = get(NETWORK_CRYPTO_ENABLED) || get(SASL_ENCRYPTION_ENABLED) require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 959f246f3f9f6..443f0fad5ed54 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -20,7 +20,7 @@ package org.apache.spark import java.io.Serializable import java.util.Properties -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Evolving} import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source @@ -176,6 +176,12 @@ abstract class TaskContext extends Serializable { */ def getLocalProperty(key: String): String + /** + * Other Resources allocated to the task. Currently gpus are the only resource supported. + */ + @Evolving + def resources(): Map[String, ResourceInformation] + @DeveloperApi def taskMetrics(): TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 76296c5d0abd3..8e40b7f1affc4 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -51,7 +51,8 @@ private[spark] class TaskContextImpl( localProperties: Properties, @transient private val metricsSystem: MetricsSystem, // The default value is only used in tests. - override val taskMetrics: TaskMetrics = TaskMetrics.empty) + override val taskMetrics: TaskMetrics = TaskMetrics.empty, + override val resources: Map[String, ResourceInformation] = Map.empty) extends TaskContext with Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index b262c235d06c2..221f04f5bdd84 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import java.util.Locale import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.mutable +import scala.collection.mutable.{HashMap, ListBuffer} import scala.util.{Failure, Success} import scala.util.control.NonFatal @@ -66,12 +66,16 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + private[this] val taskResources = new HashMap[Long, Map[String, ResourceInformation]] + override def onStart() { logInfo("Connecting to driver: " + driverUrl) val resources = parseOrFindResources(resourcesFile) rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) + + val resourceInfo = env.conf.getResources(false) ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, extractAttributes, resources)) }(ThreadUtils.sameThread).onComplete { @@ -151,6 +155,7 @@ private[spark] class CoarseGrainedExecutorBackend( } else { val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) + taskResources(taskDesc.taskId) = taskDesc.resources executor.launchTask(this, taskDesc) } @@ -197,7 +202,11 @@ private[spark] class CoarseGrainedExecutorBackend( } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { - val msg = StatusUpdate(executorId, taskId, state, data) + val resources = taskResources.getOrElse(taskId, Map.empty[String, ResourceInformation]) + val msg = StatusUpdate(executorId, taskId, state, data, resources) + if (TaskState.isFinished(state)) { + taskResources.remove(taskId) + } driver match { case Some(driverRef) => driverRef.send(msg) case None => logWarning(s"Drop $msg because has not yet connected to driver") diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index cc3cc1604d68b..2c035285c08ff 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -422,7 +422,8 @@ private[spark] class Executor( val res = task.run( taskAttemptId = taskId, attemptNumber = taskDescription.attemptNumber, - metricsSystem = env.metricsSystem) + metricsSystem = env.metricsSystem, + resources = taskDescription.resources) threwException = false res } { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 8ea88878c40ea..d320272c7d3c4 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -188,6 +188,7 @@ package object config { private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) .intConf + .checkValue(_ > 0, "Each executor must contain at least 1 cpu core.") .createWithDefault(1) private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) @@ -333,7 +334,11 @@ package object config { private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) - private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) + private[spark] val CPUS_PER_TASK = + ConfigBuilder("spark.task.cpus") + .intConf + .checkValue(_ > 0, "Each task must require at least 1 cpu core.") + .createWithDefault(1) private[spark] val DYN_ALLOCATION_ENABLED = ConfigBuilder("spark.dynamicAllocation.enabled").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala new file mode 100644 index 0000000000000..ecdb575fb778b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala @@ -0,0 +1,68 @@ +/* + * 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 + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.ResourceInformation +import org.apache.spark.annotation.Evolving +import org.apache.spark.internal.Logging + +/** + * Class to hold information about a type of Resource used by the scheduler. This + * is a separate class from the ResourceInformation because here its mutable because the + * scheduler has to update the count and addresses based on what its assigned and + * what is available. + */ +@Evolving +private[spark] class SchedulerResourceInformation( + private val name: String, + private val units: String, + private var count: Long, + private val addresses: ArrayBuffer[String] = ArrayBuffer.empty) extends Logging { + + def getName(): String = name + def getUnits(): String = units + def getCount(): Long = count + + def decCount(cnt: Long): Unit = { + count -= cnt + } + + def incCount(cnt: Long): Unit = { + count += cnt + } + + def getAddresses(): ArrayBuffer[String] = addresses + + def addAddresses(addrs: Array[String]): Unit = { + addresses ++= addrs + } + + def takeAddresses(count: Int): ArrayBuffer[String] = { + addresses.take(count) + } + + def removeAddresses(addrs: Array[String]): Unit = { + addresses --= addrs + } +} +private[spark] object SchedulerResourceInformation { + def empty: SchedulerResourceInformation = new SchedulerResourceInformation( + ResourceInformation.GPU, "", 0, ArrayBuffer.empty[String]) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 29b4380a9c331..bb44e9ada3813 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -74,12 +74,14 @@ private[spark] abstract class Task[T]( * * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. * @param attemptNumber how many times this task has been attempted (0 for the first attempt) + * @param resources other host resources (like gpus) that this task attempt can access * @return the result of the task along with updates of Accumulators. */ final def run( taskAttemptId: Long, attemptNumber: Int, - metricsSystem: MetricsSystem): T = { + metricsSystem: MetricsSystem, + resources: Map[String, ResourceInformation]): T = { SparkEnv.get.blockManager.registerTask(taskAttemptId) // TODO SPARK-24874 Allow create BarrierTaskContext based on partitions, instead of whether // the stage is barrier. @@ -92,7 +94,8 @@ private[spark] abstract class Task[T]( taskMemoryManager, localProperties, metricsSystem, - metrics) + metrics, + resources) context = if (isBarrier) { new BarrierTaskContext(taskContext) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index bb4a4442b9433..b137465d2a24e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -23,8 +23,10 @@ import java.nio.charset.StandardCharsets import java.util.Properties import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, Map} +import scala.collection.immutable +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import org.apache.spark.ResourceInformation import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} /** @@ -54,6 +56,7 @@ private[spark] class TaskDescription( val addedFiles: Map[String, Long], val addedJars: Map[String, Long], val properties: Properties, + val resources: immutable.Map[String, ResourceInformation], val serializedTask: ByteBuffer) { override def toString: String = "TaskDescription(TID=%d, index=%d)".format(taskId, index) @@ -68,6 +71,21 @@ private[spark] object TaskDescription { } } + private def serializeResources(map: immutable.Map[String, ResourceInformation], + dataOut: DataOutputStream): Unit = { + dataOut.writeInt(map.size) + for ((key, value) <- map) { + dataOut.writeUTF(key) + dataOut.writeUTF(value.getName()) + dataOut.writeUTF(value.getUnits()) + dataOut.writeLong(value.getCount()) + dataOut.writeInt(value.getAddresses.size) + for (identifier <- value.getAddresses()) { + dataOut.writeUTF(identifier) + } + } + } + def encode(taskDescription: TaskDescription): ByteBuffer = { val bytesOut = new ByteBufferOutputStream(4096) val dataOut = new DataOutputStream(bytesOut) @@ -95,6 +113,9 @@ private[spark] object TaskDescription { dataOut.write(bytes) } + // Write resources. + serializeResources(taskDescription.resources, dataOut) + // Write the task. The task is already serialized, so write it directly to the byte buffer. Utils.writeByteBuffer(taskDescription.serializedTask, bytesOut) @@ -112,6 +133,25 @@ private[spark] object TaskDescription { map } + private def deserializeResources(dataIn: DataInputStream): + immutable.Map[String, ResourceInformation] = { + val map = new HashMap[String, ResourceInformation]() + val mapSize = dataIn.readInt() + for (i <- 0 until mapSize) { + val resType = dataIn.readUTF() + val name = dataIn.readUTF() + val units = dataIn.readUTF() + val count = dataIn.readLong() + val numIdentifier = dataIn.readInt() + val identifiers = new ArrayBuffer[String](numIdentifier) + for (j <- 0 until numIdentifier) { + identifiers += dataIn.readUTF() + } + map(resType) = new ResourceInformation(name, units, count, identifiers.toArray) + } + map.toMap + } + def decode(byteBuffer: ByteBuffer): TaskDescription = { val dataIn = new DataInputStream(new ByteBufferInputStream(byteBuffer)) val taskId = dataIn.readLong() @@ -138,10 +178,13 @@ private[spark] object TaskDescription { properties.setProperty(key, new String(valueBytes, StandardCharsets.UTF_8)) } + // Read resources. + val resources = deserializeResources(dataIn) + // Create a sub-buffer for the serialized task into its own buffer (to be deserialized later). val serializedTask = byteBuffer.slice() new TaskDescription(taskId, attemptNumber, executorId, name, index, partitionId, taskFiles, - taskJars, properties, serializedTask) + taskJars, properties, resources, serializedTask) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 532eb322769aa..8f10a618a8e6a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -92,6 +92,9 @@ private[spark] class TaskSchedulerImpl( // CPUs to request per task val CPUS_PER_TASK = conf.get(config.CPUS_PER_TASK) + // GPUs to request per task + val GPUS_PER_TASK = conf.getResourceRequirements().getOrElse("gpu", 0) + // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. Protected by `this` private val taskSetsByStageIdAndAttempt = new HashMap[Int, HashMap[Int, TaskSetManager]] @@ -327,6 +330,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], + availableGpuIndices: Array[ArrayBuffer[String]], tasks: IndexedSeq[ArrayBuffer[TaskDescription]], addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = { var launchedTask = false @@ -335,16 +339,22 @@ private[spark] class TaskSchedulerImpl( for (i <- 0 until shuffledOffers.size) { val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK) { + if (availableCpus(i) >= CPUS_PER_TASK && availableGpuIndices(i).size >= GPUS_PER_TASK) { try { - for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { + val gpuResources = shuffledOffers(i).resources. + getOrElse("gpu", SchedulerResourceInformation.empty) + for (task <- taskSet.resourceOffer(execId, host, maxLocality, + availableGpuIndices(i), gpuResources)) { tasks(i) += task val tid = task.taskId taskIdToTaskSetManager.put(tid, taskSet) taskIdToExecutorId(tid) = execId executorIdToRunningTaskIds(execId).add(tid) availableCpus(i) -= CPUS_PER_TASK - assert(availableCpus(i) >= 0) + task.resources.get("gpu").foreach { addrs => + availableGpuIndices(i) --= addrs.getAddresses() + } + assert(availableCpus(i) >= 0 && availableGpuIndices(i).size >= 0) // Only update hosts for a barrier task. if (taskSet.isBarrier) { // The executor address is expected to be non empty. @@ -405,6 +415,10 @@ private[spark] class TaskSchedulerImpl( val shuffledOffers = shuffleOffers(filteredOffers) // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK)) + val availableResources = shuffledOffers.map(_.resources).toArray + val gpuResources = availableResources.map(_.getOrElse(ResourceInformation.GPU, + SchedulerResourceInformation.empty)) + val availableGpuIndices = gpuResources.map(_.getAddresses()) val availableCpus = shuffledOffers.map(o => o.cores).toArray val availableSlots = shuffledOffers.map(o => o.cores / CPUS_PER_TASK).sum val sortedTaskSets = rootPool.getSortedTaskSetQueue @@ -436,7 +450,8 @@ private[spark] class TaskSchedulerImpl( var launchedTaskAtCurrentMaxLocality = false do { launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet(taskSet, - currentMaxLocality, shuffledOffers, availableCpus, tasks, addressesWithDescs) + currentMaxLocality, shuffledOffers, availableCpus, + availableGpuIndices, tasks, addressesWithDescs) launchedAnyTask |= launchedTaskAtCurrentMaxLocality } while (launchedTaskAtCurrentMaxLocality) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 52323b3331d7e..c6d7fcbd588d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -21,6 +21,7 @@ import java.io.NotSerializableException import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue +import scala.collection.immutable.Map import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.math.max import scala.util.control.NonFatal @@ -467,7 +468,9 @@ private[spark] class TaskSetManager( def resourceOffer( execId: String, host: String, - maxLocality: TaskLocality.TaskLocality) + maxLocality: TaskLocality.TaskLocality, + hostGpuIndices: ArrayBuffer[String], + gpuResources: SchedulerResourceInformation) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -532,6 +535,15 @@ private[spark] class TaskSetManager( logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") + val extraResources = if (sched.GPUS_PER_TASK > 0) { + // doing minimal checking here to keep things fast + val indices = hostGpuIndices.take(sched.GPUS_PER_TASK).toArray + Map(ResourceInformation.GPU -> new ResourceInformation(gpuResources.getName(), + gpuResources.getUnits(), sched.GPUS_PER_TASK, indices)) + } else { + Map.empty[String, ResourceInformation] + } + sched.dagScheduler.taskStarted(task, info) new TaskDescription( taskId, @@ -543,6 +555,7 @@ private[spark] class TaskSetManager( addedFiles, addedJars, task.localProperties, + extraResources, serializedTask) } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala index 6ec74913e42f2..46e7d88780473 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -27,4 +27,5 @@ case class WorkerOffer( cores: Int, // `address` is an optional hostPort string, it provide more useful information than `host` // when multiple executors are launched on the same host. - address: Option[String] = None) + address: Option[String] = None, + resources: Map[String, SchedulerResourceInformation] = Map.empty) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 89425e702677a..82d51f8a169a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -69,14 +69,19 @@ private[spark] object CoarseGrainedClusterMessages { resources: Map[String, ResourceInformation]) extends CoarseGrainedClusterMessage - case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, - data: SerializableBuffer) extends CoarseGrainedClusterMessage + case class StatusUpdate( + executorId: String, + taskId: Long, + state: TaskState, + data: SerializableBuffer, + resources: Map[String, ResourceInformation] = Map.empty) + extends CoarseGrainedClusterMessage object StatusUpdate { /** Alternate factory method that takes a ByteBuffer directly for the data field */ - def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) - : StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer, + resources: Map[String, ResourceInformation]): StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), resources) } } 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 f7cf212d0bfe1..7ebfefbef16a7 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 @@ -21,7 +21,8 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.{HashMap, HashSet} +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future import org.apache.hadoop.security.UserGroupInformation @@ -139,12 +140,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } override def receive: PartialFunction[Any, Unit] = { - case StatusUpdate(executorId, taskId, state, data) => + case StatusUpdate(executorId, taskId, state, data, resources) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { executorDataMap.get(executorId) match { case Some(executorInfo) => executorInfo.freeCores += scheduler.CPUS_PER_TASK + for ((k, v) <- resources) { + executorInfo.availableResources.get(k).foreach { r => + r.incCount(v.getCount()) + r.addAddresses(v.getAddresses()) + } + } makeOffers(executorId) case None => // Ignoring the update since we don't know about the executor. @@ -210,7 +217,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) val data = new ExecutorData(executorRef, executorAddress, hostname, - cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes) + cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, + mutable.Map.empty[String, SchedulerResourceInformation] ++= resources.mapValues(v => + new SchedulerResourceInformation(v.getName(), v.getUnits(), v.getCount(), + v.getAddresses().to[ArrayBuffer]))) + // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { @@ -263,7 +274,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val workOffers = activeExecutors.map { case (id, executorData) => new WorkerOffer(id, executorData.executorHost, executorData.freeCores, - Some(executorData.executorAddress.hostPort)) + Some(executorData.executorAddress.hostPort), executorData.availableResources.toMap) }.toIndexedSeq scheduler.resourceOffers(workOffers) } @@ -289,7 +300,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val executorData = executorDataMap(executorId) val workOffers = IndexedSeq( new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores, - Some(executorData.executorAddress.hostPort))) + Some(executorData.executorAddress.hostPort), executorData.availableResources.toMap)) scheduler.resourceOffers(workOffers) } else { Seq.empty @@ -326,6 +337,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK + for ((k, v) <- task.resources) { + executorData.availableResources.get(k).foreach { r => + r.decCount(v.getCount()) + r.removeAddresses(v.getAddresses()) + } + } + logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + s"${executorData.executorHost}.") @@ -525,6 +543,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp }.sum } + // this function is for testing only + def getExecutorAvailableResources(executorId: String): + mutable.Map[String, SchedulerResourceInformation] = { + executorDataMap.get(executorId).map(_.availableResources). + getOrElse(mutable.Map.empty[String, SchedulerResourceInformation]) + } + /** * Request an additional number of executors from the cluster manager. * @return whether the request is acknowledged. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index ebe1c1eb0a357..3964d94713730 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,7 +17,11 @@ package org.apache.spark.scheduler.cluster +import scala.collection.mutable + +import org.apache.spark.ResourceInformation import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} +import org.apache.spark.scheduler.SchedulerResourceInformation /** * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. @@ -27,6 +31,8 @@ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} * @param executorHost The hostname that this executor is running on * @param freeCores The current number of cores available for work on the executor * @param totalCores The total number of cores available to the executor + * @param totalResources The information of all resources on the executor + * @param availableResources The information of the currently available resources on the executor */ private[cluster] class ExecutorData( val executorEndpoint: RpcEndpointRef, @@ -35,5 +41,7 @@ private[cluster] class ExecutorData( var freeCores: Int, override val totalCores: Int, override val logUrlMap: Map[String, String], - override val attributes: Map[String, String] -) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes) + override val attributes: Map[String, String], + override val totalResources: Map[String, ResourceInformation], + val availableResources: mutable.Map[String, SchedulerResourceInformation] +) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, totalResources) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 3197e06fcd13a..1f6d572613047 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.scheduler.cluster +import org.apache.spark.ResourceInformation import org.apache.spark.annotation.DeveloperApi /** @@ -27,10 +28,11 @@ class ExecutorInfo( val executorHost: String, val totalCores: Int, val logUrlMap: Map[String, String], - val attributes: Map[String, String]) { + val attributes: Map[String, String], + val totalResources: Map[String, ResourceInformation] = Map.empty) { def this(executorHost: String, totalCores: Int, logUrlMap: Map[String, String]) = { - this(executorHost, totalCores, logUrlMap, Map.empty) + this(executorHost, totalCores, logUrlMap, Map.empty, Map.empty) } def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -41,12 +43,13 @@ class ExecutorInfo( executorHost == that.executorHost && totalCores == that.totalCores && logUrlMap == that.logUrlMap && - attributes == that.attributes + attributes == that.attributes && + totalResources == that.totalResources case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores, logUrlMap, attributes) + val state = Seq(executorHost, totalCores, logUrlMap, attributes, totalResources) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index fde2a328f02f1..fae322a7f2be2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -81,8 +81,9 @@ private[spark] class LocalEndpoint( } def reviveOffers() { + // local mode doesn't support extra resources like GPUs right now val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores, - Some(rpcEnv.address.hostPort))) + Some(rpcEnv.address.hostPort), Map.empty)) for (task <- scheduler.resourceOffers(offers).flatten) { freeCores -= scheduler.CPUS_PER_TASK executor.launchTask(executorBackend, task) @@ -130,7 +131,7 @@ private[spark] class LocalSchedulerBackend( System.currentTimeMillis, executorEndpoint.localExecutorId, new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty, - Map.empty))) + Map.empty, Map.empty))) launcherBackend.setAppId(appId) launcherBackend.setState(SparkAppHandle.State.RUNNING) } diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java index f8e233a05a447..62a0b85915efc 100644 --- a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java +++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java @@ -40,6 +40,10 @@ public static void test() { tc.stageId(); tc.stageAttemptNumber(); tc.taskAttemptId(); + tc.resources(); + tc.taskMetrics(); + tc.taskMemoryManager(); + tc.getLocalProperties(); } /** diff --git a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager index cf8565c74e95e..1c78f1a019001 100644 --- a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -1,3 +1,4 @@ org.apache.spark.scheduler.DummyExternalClusterManager org.apache.spark.scheduler.MockExternalClusterManager org.apache.spark.DummyLocalExternalClusterManager +org.apache.spark.scheduler.CSMockExternalClusterManager diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 965c7df69cfe3..eb6441d35ee2a 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -446,6 +446,85 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(thrown.getMessage.contains(key)) } } + + test("specify resource count and addresses") { + val conf = new SparkConf() + conf.validateSettings() + + // only specify resource count + conf.setResources("gpu", 2, None, false) + assert(conf.getResources(false).get("gpu").get.getCount() == 2) + assert(conf.getResources(false).get("gpu").get.getAddresses().sameElements(Array.empty[String])) + conf.validateSettings() + + // specify resource count and addresses + conf.setResources("gpu", 2, Some(Seq("0", "1")), false) + assert(conf.getResources(false).get("gpu").get.getCount() == 2) + assert(conf.getResources(false).get("gpu").get.getAddresses().sameElements(Seq("0", "1"))) + conf.validateSettings() + + // conflict number of resource count and addresses size + intercept[SparkException] { + conf.setResources("gpu", 2, Some(Seq("0", "1", "2")), false) + } + + // specify resource on the driver + assert(conf.getResources(true).isEmpty) + conf.setResources("gpu", 2, Some(Seq("0", "1")), true) + assert(conf.getResources(true).get("gpu").get.getCount() == 2) + assert(conf.getResources(true).get("gpu").get.getAddresses().sameElements(Seq("0", "1"))) + conf.validateSettings() + } + + test("specify resource requirement") { + val conf = new SparkConf() + conf.validateSettings() + + // resource required but not on executors. + assert(conf.getResourceRequirements().isEmpty) + conf.setResourceRequirement("gpu", 2) + assert(conf.getResourceRequirements().get("gpu").get == 2) + intercept[SparkException] { + conf.validateSettings() + } + + // specify resource on executors. + conf.setResources("gpu", 2, None, false) + conf.validateSettings() + } + + test("verify available resources and resource requirements") { + val conf = new SparkConf() + conf.validateSettings() + + // resource on executors less than resource requirement + conf.setResourceRequirement("gpu", 3) + conf.setResources("gpu", 2, None, false) + intercept[SparkException] { + conf.validateSettings() + } + + // valid case + conf.setResourceRequirement("gpu", 2) + conf.setResources("gpu", 2, None, false) + conf.validateSettings() + + // resource on executors not fully used compated to resource requirement + conf.setResourceRequirement("gpu", 3) + conf.setResources("gpu", 3, None, false) + conf.set(EXECUTOR_CORES.key, "4") + conf.set(CPUS_PER_TASK.key, "1") + intercept[SparkException] { + conf.validateSettings() + } + + // valid case + conf.setResourceRequirement("gpu", 2) + conf.setResources("gpu", 8, None, false) + conf.set(EXECUTOR_CORES.key, "4") + conf.set(CPUS_PER_TASK.key, "1") + conf.validateSettings() + } } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index ded914d3d896f..d6dbae367e28f 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -843,6 +843,38 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu file.getPath() } + + test("test gpu support under local-cluster mode") { + val conf = new SparkConf() + .setResources("gpu", 3, Some(Seq("0", "1", "2")), false) + .setResources("gpu", 3, Some(Seq("0", "1", "8")), true) + .setResourceRequirement("gpu", 1) + .setMaster("local-cluster[3, 3, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + + // Ensure all executors has started + eventually(timeout(10.seconds)) { + assert(sc.statusTracker.getExecutorInfos.size == 3) + } + + val resources = sc.getResources() + assert(resources.get("gpu").get.getAddresses() === Array("0", "1", "8")) + assert(resources.get("gpu").get.getCount() === 3) + assert(resources.get("gpu").get.getName() === "gpu") + assert(resources.get("gpu").get.getUnits() === "") + + val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => + val context = TaskContext.get() + context.resources().get(ResourceInformation.GPU).get.getAddresses().iterator + } + val gpus = rdd.collect() + assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) + + eventually(timeout(10.seconds)) { + assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } + } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 8a4f7a34e5451..495321fcba745 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -24,6 +24,7 @@ import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.immutable import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map import scala.concurrent.duration._ @@ -369,6 +370,7 @@ class ExecutorSuite extends SparkFunSuite addedFiles = Map[String, Long](), addedJars = Map[String, Long](), properties = new Properties, + resources = immutable.Map[String, ResourceInformation](), serializedTask) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 4858d38cad409..634527b752f0e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -17,21 +17,27 @@ package org.apache.spark.scheduler +import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.immutable +import scala.collection.mutable import scala.concurrent.duration._ +import scala.language.postfixOps +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.when import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar._ -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite} -import org.apache.spark.internal.config.{CPUS_PER_TASK, UI} +import org.apache.spark._ +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD -import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RegisterExecutor +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.{RpcUtils, SerializableBuffer} +import org.apache.spark.util.{RpcUtils, SerializableBuffer, Utils} class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with Eventually { @@ -174,6 +180,94 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(executorAddedCount === 3) } + test("extra gpu resources from executor") { + val conf = new SparkConf() + .setResources("gpu", 3, None, false) + .setResourceRequirement("gpu", 3) + .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test + .setMaster( + "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") + .setAppName("test") + + sc = new SparkContext(conf) + val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] + val mockEndpointRef = mock[RpcEndpointRef] + val mockAddress = mock[RpcAddress] + + val logUrls = Map( + "stdout" -> "http://oldhost:8888/logs/dummy/stdout", + "stderr" -> "http://oldhost:8888/logs/dummy/stderr") + val attributes = Map( + "CLUSTER_ID" -> "cl1", + "USER" -> "dummy", + "CONTAINER_ID" -> "container1", + "LOG_FILES" -> "stdout,stderr") + val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + + s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" + val resources = Map(ResourceInformation.GPU -> + new ResourceInformation(ResourceInformation.GPU, "", 3, Array("0", "1", "3"))) + + var executorAddedCount: Int = 0 + val listener = new SparkListener() { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + executorAddedCount += 1 + assert(executorAdded.executorInfo.totalResources.get(ResourceInformation.GPU).nonEmpty) + val totalResources = executorAdded.executorInfo.totalResources. + get(ResourceInformation.GPU).get + assert(totalResources.getAddresses() === Array("0", "1", "3")) + assert(totalResources.getCount() == 3) + assert(totalResources.getName() == ResourceInformation.GPU) + assert(totalResources.getUnits() == "") + } + } + + sc.addSparkListener(listener) + + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + + val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) + val bytebuffer = java.nio.ByteBuffer.allocate(frameSize/2) + val buffer = new SerializableBuffer(bytebuffer) + + var gpuExecResources = backend.getExecutorAvailableResources("1") + + assert(gpuExecResources.get("gpu").get.getCount() === 3) + assert(gpuExecResources.get("gpu").get.getAddresses() === Array("0", "1", "3")) + + var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", + "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], + new Properties(), immutable.Map("gpu" -> new ResourceInformation("gpu", "", 1, Array("0"))), + bytebuffer))) + val ts = backend.getTaskSchedulerImpl() + // resource offer such that gpu 0 gets removed + when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(taskDescs) + + backend.driverEndpoint.send(ReviveOffers) + + eventually(timeout(5 seconds), interval(10 millis)) { + gpuExecResources = backend.getExecutorAvailableResources("1") + assert(gpuExecResources.get("gpu").get.getCount() === 2) + assert(gpuExecResources.get("gpu").get.getAddresses() === Array("1", "3")) + } + + var finishedTaskResources = Map("gpu" -> new ResourceInformation("gpu", "", 1, Array("0"))) + backend.driverEndpoint.send( + StatusUpdate("1", 1, TaskState.FINISHED, buffer, finishedTaskResources)) + + eventually(timeout(5 seconds), interval(10 millis)) { + gpuExecResources = backend.getExecutorAvailableResources("1") + assert(gpuExecResources.get("gpu").get.getCount() === 3) + assert(gpuExecResources.get("gpu").get.getAddresses() === Array("1", "3", "0")) + } + sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) + assert(executorAddedCount === 3) + } + private def testSubmitJob(sc: SparkContext, rdd: RDD[Int]): Unit = { sc.submitJob( rdd, @@ -184,3 +278,47 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo ) } } + +/** Simple cluster manager that wires up our mock backend for the gpu resource tests. */ +private class CSMockExternalClusterManager extends ExternalClusterManager { + + private var ts: TaskSchedulerImpl = _ + + private val MOCK_REGEX = """coarseclustermanager\[(.*)\]""".r + override def canCreate(masterURL: String): Boolean = MOCK_REGEX.findFirstIn(masterURL).isDefined + + override def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { + ts = mock[TaskSchedulerImpl] + when(ts.sc).thenReturn(sc) + when(ts.applicationId()).thenReturn("appid1") + when(ts.applicationAttemptId()).thenReturn(Some("attempt1")) + when(ts.schedulingMode).thenReturn(SchedulingMode.FIFO) + when(ts.nodeBlacklist()).thenReturn(Set.empty[String]) + ts + } + + override def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + masterURL match { + case MOCK_REGEX(backendClassName) => + val backendClass = Utils.classForName(backendClassName) + val ctor = backendClass.getConstructor(classOf[TaskSchedulerImpl], classOf[RpcEnv]) + ctor.newInstance(scheduler, sc.env.rpcEnv).asInstanceOf[SchedulerBackend] + } + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} + +private[spark] +class TestCoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, override val rpcEnv: RpcEnv) + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + + def getTaskSchedulerImpl(): TaskSchedulerImpl = scheduler +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 2f677776fe826..c16b552d20891 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -70,7 +70,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, closureSerializer.serialize(TaskMetrics.registered).array()) intercept[RuntimeException] { - task.run(0, 0, null) + task.run(0, 0, null, null) } assert(TaskContextSuite.completed) } @@ -92,7 +92,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, closureSerializer.serialize(TaskMetrics.registered).array()) intercept[RuntimeException] { - task.run(0, 0, null) + task.run(0, 0, null, null) } assert(TaskContextSuite.lastError.getMessage == "damn error") } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala index ba62eec0522db..c59470ef62564 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -23,6 +23,7 @@ import java.util.Properties import scala.collection.mutable.HashMap +import org.apache.spark.ResourceInformation import org.apache.spark.SparkFunSuite class TaskDescriptionSuite extends SparkFunSuite { @@ -53,6 +54,9 @@ class TaskDescriptionSuite extends SparkFunSuite { } } + val originalResources = + Map("gpu" -> new ResourceInformation("gpu", "", 3, Array("1", "2", "3"))) + // Create a dummy byte buffer for the task. val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) @@ -66,6 +70,7 @@ class TaskDescriptionSuite extends SparkFunSuite { originalFiles, originalJars, originalProperties, + originalResources, taskBuffer ) @@ -82,6 +87,19 @@ class TaskDescriptionSuite extends SparkFunSuite { assert(decodedTaskDescription.addedFiles.equals(originalFiles)) assert(decodedTaskDescription.addedJars.equals(originalJars)) assert(decodedTaskDescription.properties.equals(originalTaskDescription.properties)) + assert(equalResources(decodedTaskDescription.resources, originalTaskDescription.resources)) assert(decodedTaskDescription.serializedTask.equals(taskBuffer)) + + def equalResources(original: Map[String, ResourceInformation], + target: Map[String, ResourceInformation]): Boolean = { + original.size == target.size && original.forall { case (name, info) => + target.get(name).exists { targetInfo => + info.getName().equals(targetInfo.getName()) && + info.getUnits().equals(targetInfo.getUnits()) && + info.getCount() === targetInfo.getCount() && + info.getAddresses().sameElements(targetInfo.getAddresses()) + } + } + } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 29614058485ab..fa8e400163eec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import scala.collection.mutable.HashMap +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.duration._ import org.mockito.ArgumentMatchers.{any, anyInt, anyString, eq => meq} @@ -1238,4 +1238,38 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B tsm.handleFailedTask(tsm.taskAttempts.head.head.taskId, TaskState.FAILED, TaskKilled("test")) assert(tsm.isZombie) } + + test("Scheduler correctly accounts for GPUs per task") { + val taskCpus = 1 + val taskGpus = 1 + val executorGpus = 4 + val executorCpus = 4 + val taskScheduler = setupScheduler(config.CPUS_PER_TASK.key -> taskCpus.toString, + s"${config.SPARK_TASK_RESOURCE_PREFIX}.gpu.${config.SPARK_RESOURCE_COUNT}" -> + taskGpus.toString, + s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}.gpu.${config.SPARK_RESOURCE_COUNT}" -> + executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + val taskSet = FakeTask.createTaskSet(3) + + val numFreeCores = 2 + val gpuresources = Map("gpu" -> + new SchedulerResourceInformation("gpu", "", 4, ArrayBuffer("0", "1", "2", "3"))) + val singleCoreWorkerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, gpuresources)) + val zeroGpuWorkerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, Map.empty)) + taskScheduler.submitTasks(taskSet) + // WorkerOffer doesn't contain GPU resource, don't launch any task. + var taskDescriptions = taskScheduler.resourceOffers(zeroGpuWorkerOffers).flatten + assert(0 === taskDescriptions.length) + assert(!failedTaskSet) + // Launch tasks on executor that satisfies GPU resource requirements. + taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten + assert(2 === taskDescriptions.length) + assert(!failedTaskSet) + assert(1 === taskDescriptions(0).resources.get("gpu").get.getCount()) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get("gpu").get.getAddresses()) + assert(ArrayBuffer("1") === taskDescriptions(1).resources.get("gpu").get.getAddresses()) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 72c6ab964ccf4..66c592198845e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -210,7 +210,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) clock.advance(1) @@ -231,7 +232,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -239,7 +241,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) + assert(manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) @@ -262,12 +265,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. - assert(manager.resourceOffer("execC", "host2", ANY) === None) + assert(manager.resourceOffer("execC", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) // Because there are no alive PROCESS_LOCAL executors, the base locality level should be // NODE_LOCAL. So, we should schedule the task on this offered NODE_LOCAL executor before // any of the locality wait timers expire. - assert(manager.resourceOffer("execA", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) } test("basic delay scheduling") { @@ -282,22 +287,28 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 3) should // get chosen before the noPref task - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index == 2) // Offer host2, exec2, at NODE_LOCAL level: we should choose task 2 - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).get.index == 1) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index == 1) // Offer host2, exec2 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) clock.advance(LOCALITY_WAIT_MS) - assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) + assert(manager.resourceOffer("exec2", "host2", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index == 3) } test("we do not need to delay scheduling when we only have noPref tasks in the queue") { @@ -311,10 +322,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) - assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) - assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL) == None) - assert(manager.resourceOffer("exec3", "host2", NO_PREF).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("exec3", "host2", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) } test("delay scheduling with fallback") { @@ -332,29 +347,36 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) // Offer host1 again: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) // Offer host1 again: third task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) // Offer host2: fifth task (also on host2) should get chosen - assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 4) + assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 4) // Now that we've launched a local task, we should no longer launch the task for host3 - assert(manager.resourceOffer("exec2", "host2", ANY) === None) + assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally - assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) + assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 3) } test("delay scheduling with failed hosts") { @@ -370,28 +392,35 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) // After this, nothing should get chosen, because we have separated tasks with unavailable // preference from the noPrefPendingTasks - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2", SlaveLost()) // nothing should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) // all finished - assert(manager.resourceOffer("exec1", "host1", ANY) === None) - assert(manager.resourceOffer("exec2", "host2", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) } test("task result lost") { @@ -402,14 +431,16 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) // Tell it the task has finished but the result was lost. manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost) assert(sched.endedTasks(0) === TaskResultLost) // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) } test("repeated failures lead to task set abortion") { @@ -423,7 +454,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. (1 to manager.maxTaskFailures).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", ANY) + val offerResult = manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(offerResult.isDefined, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) @@ -459,7 +491,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -470,15 +503,20 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL) + val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -490,12 +528,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", ANY) + val offerResult = manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -506,7 +546,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) } // Despite advancing beyond the time for expiring executors from within the blacklist, @@ -514,17 +555,20 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(offerResult.isEmpty) } { - val offerResult = manager.resourceOffer("exec3", "host3", ANY) + val offerResult = manager.resourceOffer("exec3", "host3", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(offerResult.isDefined) assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isEmpty) // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) @@ -583,14 +627,16 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg manager.executorAdded() sched.addExecutor("execC", "host2") manager.executorAdded() - assert(manager.resourceOffer("exec1", "host1", ANY).isDefined) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isDefined) sched.removeExecutor("execA") manager.executorLost( "execA", "host1", ExecutorExited(143, false, "Terminated for reason unrelated to running tasks")) assert(!sched.taskSetsFailed.contains(taskSet.id)) - assert(manager.resourceOffer("execC", "host2", ANY).isDefined) + assert(manager.resourceOffer("execC", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).isDefined) sched.removeExecutor("execC") manager.executorLost( "execC", "host2", ExecutorExited(1, true, "Terminated due to issue with running tasks")) @@ -618,12 +664,15 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL - assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) + assert(manager.resourceOffer("execC", "host3", RACK_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) // Task 0 can be scheduled with ANY - assert(manager.resourceOffer("execC", "host3", ANY).get.index === 0) + assert(manager.resourceOffer("execC", "host3", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) // Offer host2 // Task 1 can be scheduled with RACK_LOCAL - assert(manager.resourceOffer("execB", "host2", RACK_LOCAL).get.index === 1) + assert(manager.resourceOffer("execB", "host2", RACK_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) } test("do not emit warning when serialized task is small") { @@ -634,7 +683,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) assert(!manager.emittedTaskSizeWarning) } @@ -648,7 +698,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) assert(manager.emittedTaskSizeWarning) } @@ -662,7 +713,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { - manager.resourceOffer("exec1", "host1", ANY) + manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) } assert(manager.isZombie) } @@ -733,12 +785,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer host1, which should be accepted as a PROCESS_LOCAL location // by the one task in the task set - val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL).get + val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL, + ArrayBuffer.empty, SchedulerResourceInformation.empty).get // Mark the task as available for speculation, and then offer another resource, // which should be used to launch a speculative copy of the task. manager.speculatableTasks += singleTask.partitionId - val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY).get + val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get assert(manager.runningTasks === 2) assert(manager.isZombie === false) @@ -823,7 +877,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -849,7 +904,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY) + val taskOption = manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -878,19 +934,25 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index == 1) manager.speculatableTasks += 1 clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) + assert(manager.resourceOffer("execA", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) // schedule the speculative task - assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) + assert(manager.resourceOffer("execB", "host2", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks - assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) + assert(manager.resourceOffer("execB", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 3) } test("node-local tasks should be scheduled right away " + @@ -907,13 +969,18 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) - assert(manager.resourceOffer("execA", "host2", NODE_LOCAL).get.index === 1) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL).get.index === 3) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL) === None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 3) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) === None) // schedule no-preference after node local ones - assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) + assert(manager.resourceOffer("execA", "host3", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) } test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") @@ -929,13 +996,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 3) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) } test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { @@ -949,13 +1022,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) - assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 2) // no-pref tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0) - assert(manager.resourceOffer("execA", "host1", ANY) == None) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) == None) } test("Ensure TaskSetManager is usable after addition of levels") { @@ -976,9 +1055,11 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.pendingTasksWithNoPrefs.size === 0) // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) - assert(manager.resourceOffer("execA", "host1", ANY) !== None) + assert(manager.resourceOffer("execA", "host1", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) !== None) clock.advance(LOCALITY_WAIT_MS * 4) - assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) + assert(manager.resourceOffer("execB.2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1", SlaveLost()) @@ -987,7 +1068,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: - assert(manager.resourceOffer("execC", "host3", ANY) !== None) + assert(manager.resourceOffer("execC", "host3", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) !== None) } test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { @@ -1039,7 +1121,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1060,7 +1143,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1099,7 +1183,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1132,7 +1217,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) sched.endedTasks(task.taskId) = endReason assert(!manager.isZombie) - val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") tasks += nextTask.get } @@ -1148,7 +1234,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(3, 4)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption5.isDefined) val speculativeTask = taskOption5.get assert(speculativeTask.index === 3 || speculativeTask.index === 4) @@ -1173,7 +1260,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.isZombie) // now run another speculative task - val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOpt6.isDefined) val speculativeTask2 = taskOpt6.get assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) @@ -1204,7 +1292,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any())).thenAnswer( (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -1249,7 +1338,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec2" -> "host1" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY)} + (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY, + ArrayBuffer.empty, SchedulerResourceInformation.empty)} } assert(taskDescs.size === 4) @@ -1286,7 +1376,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec2" -> "host2" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY)} + (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty)} } assert(taskDescs.size === 4) @@ -1322,7 +1413,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) val taskSetManagerSpy = spy(taskSetManager) - val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) + val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) // Assert the task has been black listed on the executor it was last executed on. when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( @@ -1350,9 +1442,11 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager1 = new TaskSetManager(sched, taskSet1, MAX_TASK_FAILURES, clock = new ManualClock) // all tasks from the first taskset have the same jars - val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption1.get.addedJars === addedJarsPreTaskSet) - val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption2.get.addedJars === addedJarsPreTaskSet) // even with a jar added mid-TaskSet @@ -1360,7 +1454,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sc.addJar(jarPath.toString) val addedJarsMidTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) assert(addedJarsPreTaskSet !== addedJarsMidTaskSet) - val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) // which should have the old version of the jars list assert(taskOption3.get.addedJars === addedJarsPreTaskSet) @@ -1368,7 +1463,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet2 = FakeTask.createTaskSet(1) val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) - val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF) + val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } @@ -1465,7 +1561,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -1491,7 +1588,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY) + val taskOption = manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -1536,7 +1634,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1556,7 +1655,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, + SchedulerResourceInformation.empty) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1615,16 +1715,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(FakeRackUtil.numBatchInvocation === 1) assert(FakeRackUtil.numSingleHostInvocation === 0) // with rack locality, reject an offer on a host with an unknown rack - assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL, + ArrayBuffer.empty, SchedulerResourceInformation.empty).isEmpty) (0 until 20).foreach { rackIdx => (0 until 5).foreach { offerIdx => // if we offer hosts which are not in preferred locations, // we'll reject them at NODE_LOCAL level, // but accept them at RACK_LOCAL level if they're on OK racks val hostIdx = 100 + rackIdx - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL, + ArrayBuffer.empty, SchedulerResourceInformation.empty) .isEmpty) - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL, + ArrayBuffer.empty, SchedulerResourceInformation.empty) .isDefined) } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5ccaa38c08675..98e4764bdfe48 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,9 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-27366][CORE] Support GPU Resources in Spark job scheduling + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.resources"), + // [SPARK-27410][MLLIB] Remove deprecated / no-op mllib.KMeans getRuns, setRuns ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.getRuns"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.setRuns"), diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index 3e63c35361a5b..b70be7a4b0eed 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -24,6 +24,7 @@ import java.util.Collections import java.util.Properties import scala.collection.JavaConverters._ +import scala.collection.immutable import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -35,7 +36,8 @@ import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, ResourceInformation, SparkConf, SparkContext, + SparkFunSuite} import org.apache.spark.deploy.mesos.config._ import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, @@ -262,6 +264,7 @@ class MesosFineGrainedSchedulerBackendSuite addedFiles = mutable.Map.empty[String, Long], addedJars = mutable.Map.empty[String, Long], properties = new Properties(), + resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) @@ -372,6 +375,7 @@ class MesosFineGrainedSchedulerBackendSuite addedFiles = mutable.Map.empty[String, Long], addedJars = mutable.Map.empty[String, Long], properties = new Properties(), + resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(1) From 46ceedd0ec184b0fc33a2e979c0a1b24dafd155d Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 14 May 2019 18:37:18 -0700 Subject: [PATCH 02/18] fix build failure --- .../apache/spark/ResourceInformation.scala | 5 +++++ .../scala/org/apache/spark/SparkConf.scala | 22 +++++++++---------- .../CoarseGrainedExecutorBackend.scala | 4 ++-- .../SchedulerResourceInformation.scala | 16 ++------------ .../spark/scheduler/TaskDescription.scala | 12 ++++------ .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 3 +-- .../CoarseGrainedSchedulerBackend.scala | 9 +++----- .../org/apache/spark/SparkConfSuite.scala | 9 +++----- .../org/apache/spark/SparkContextSuite.scala | 8 +++---- .../CoarseGrainedSchedulerBackendSuite.scala | 15 +++++-------- .../scheduler/TaskDescriptionSuite.scala | 8 +++---- .../scheduler/TaskSchedulerImplSuite.scala | 11 +++++----- 13 files changed, 48 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index 6a5b725ac21d7..bc969518d285a 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -35,3 +35,8 @@ class ResourceInformation( override def toString: String = s"[name: ${name}, addresses: ${addresses.mkString(",")}]" } + +object ResourceInformation { + // known types of resources + final val GPU: String = "gpu" +} diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 97f37ff966808..6f61ebe05546e 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -525,13 +525,13 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria s"the size of device addresses is ${resourceAddresses.size}.") } val prefix = if (isDriver) { - s"$SPARK_DRIVER_RESOURCE_PREFIX.$resourceName" + s"$SPARK_DRIVER_RESOURCE_PREFIX$resourceName" } else { - s"$SPARK_EXECUTOR_RESOURCE_PREFIX.$resourceName" + s"$SPARK_EXECUTOR_RESOURCE_PREFIX$resourceName" } - set(s"$prefix.$SPARK_RESOURCE_COUNT", resourceCount.toString) + set(s"$prefix$SPARK_RESOURCE_COUNT_POSTFIX", resourceCount.toString) if (resourceAddresses.isDefined) { - set(s"$prefix.$SPARK_RESOURCE_ADDRESSES", resourceAddresses.get.mkString(",")) + set(s"$prefix$SPARK_RESOURCE_ADDRESSES_POSTFIX", resourceAddresses.get.mkString(",")) } this } @@ -541,7 +541,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria */ private[spark] def setResourceRequirement( resourceName: String, resourceCount: Int): SparkConf = { - val key = s"$SPARK_TASK_RESOURCE_PREFIX.$resourceName.$SPARK_RESOURCE_COUNT" + val key = s"$SPARK_TASK_RESOURCE_PREFIX$resourceName$SPARK_RESOURCE_COUNT_POSTFIX" set(key, resourceCount.toString) this } @@ -562,10 +562,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria val keys = tuple._1.split("\\.") if (keys.size == 2) { val resourceName = keys.head - if (keys.last.equals(SPARK_RESOURCE_COUNT)) { + if (keys.last.equals(SPARK_RESOURCE_COUNT_POSTFIX)) { val resourceCount = tuple._2.toInt resourceCountMap.put(resourceName, resourceCount) - } else if (keys.last.equals(SPARK_RESOURCE_ADDRESSES)) { + } else if (keys.last.equals(SPARK_RESOURCE_ADDRESSES_POSTFIX)) { val resourceAddresses = tuple._2.split(",").map(_.trim()) resourceAddressMap.put(resourceName, resourceAddresses) } @@ -575,9 +575,9 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria resourceCountMap.map { case (resourceName, resourceCount) => if (resourceAddressMap.contains(resourceName)) { val resourceAddresses = resourceAddressMap.get(resourceName).get - (resourceName, new ResourceInformation(resourceName, "", resourceCount, resourceAddresses)) + (resourceName, new ResourceInformation(resourceName, resourceAddresses)) } else { - (resourceName, new ResourceInformation(resourceName, "", resourceCount, Array.empty)) + (resourceName, new ResourceInformation(resourceName, Array.empty)) } }.toMap } @@ -590,7 +590,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria val keys = tuple._1.split("\\.") val resourceName = keys.head if (keys.size == 2) { - if (keys.last.equals(SPARK_RESOURCE_COUNT)) { + if (keys.last.equals(SPARK_RESOURCE_COUNT_POSTFIX)) { val resourceCount = tuple._2.toInt (resourceName, resourceCount) } else { @@ -696,7 +696,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria s"available on executors.") } else { val resourceInformation = resources.get(resourceName).get - val executorResourceCount = resourceInformation.getCount() + val executorResourceCount = resourceInformation.addresses.size if (executorResourceCount < resourceCount) { throw new SparkException(s"Available $resourceName resource count on executors is " + s"$executorResourceCount, which is less than the task requirement $resourceCount.") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 221f04f5bdd84..d5f1ac1730a9a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import java.util.Locale import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.mutable.{HashMap, ListBuffer} +import scala.collection.mutable import scala.util.{Failure, Success} import scala.util.control.NonFatal @@ -66,7 +66,7 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() - private[this] val taskResources = new HashMap[Long, Map[String, ResourceInformation]] + private[this] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] override def onStart() { logInfo("Connecting to driver: " + driverUrl) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala index ecdb575fb778b..e0d85e0035c6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala @@ -32,21 +32,9 @@ import org.apache.spark.internal.Logging @Evolving private[spark] class SchedulerResourceInformation( private val name: String, - private val units: String, - private var count: Long, private val addresses: ArrayBuffer[String] = ArrayBuffer.empty) extends Logging { def getName(): String = name - def getUnits(): String = units - def getCount(): Long = count - - def decCount(cnt: Long): Unit = { - count -= cnt - } - - def incCount(cnt: Long): Unit = { - count += cnt - } def getAddresses(): ArrayBuffer[String] = addresses @@ -63,6 +51,6 @@ private[spark] class SchedulerResourceInformation( } } private[spark] object SchedulerResourceInformation { - def empty: SchedulerResourceInformation = new SchedulerResourceInformation( - ResourceInformation.GPU, "", 0, ArrayBuffer.empty[String]) + def empty: SchedulerResourceInformation = new SchedulerResourceInformation("gpu", + ArrayBuffer.empty[String]) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index b137465d2a24e..fd0d9f7e43b6e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -76,11 +76,9 @@ private[spark] object TaskDescription { dataOut.writeInt(map.size) for ((key, value) <- map) { dataOut.writeUTF(key) - dataOut.writeUTF(value.getName()) - dataOut.writeUTF(value.getUnits()) - dataOut.writeLong(value.getCount()) - dataOut.writeInt(value.getAddresses.size) - for (identifier <- value.getAddresses()) { + dataOut.writeUTF(value.name) + dataOut.writeInt(value.addresses.size) + for (identifier <- value.addresses) { dataOut.writeUTF(identifier) } } @@ -140,14 +138,12 @@ private[spark] object TaskDescription { for (i <- 0 until mapSize) { val resType = dataIn.readUTF() val name = dataIn.readUTF() - val units = dataIn.readUTF() - val count = dataIn.readLong() val numIdentifier = dataIn.readInt() val identifiers = new ArrayBuffer[String](numIdentifier) for (j <- 0 until numIdentifier) { identifiers += dataIn.readUTF() } - map(resType) = new ResourceInformation(name, units, count, identifiers.toArray) + map(resType) = new ResourceInformation(name, identifiers.toArray) } map.toMap } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8f10a618a8e6a..94f2e4f74ff54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -352,7 +352,7 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds(execId).add(tid) availableCpus(i) -= CPUS_PER_TASK task.resources.get("gpu").foreach { addrs => - availableGpuIndices(i) --= addrs.getAddresses() + availableGpuIndices(i) --= addrs.addresses } assert(availableCpus(i) >= 0 && availableGpuIndices(i).size >= 0) // Only update hosts for a barrier task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c6d7fcbd588d6..54b0107c2820c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -538,8 +538,7 @@ private[spark] class TaskSetManager( val extraResources = if (sched.GPUS_PER_TASK > 0) { // doing minimal checking here to keep things fast val indices = hostGpuIndices.take(sched.GPUS_PER_TASK).toArray - Map(ResourceInformation.GPU -> new ResourceInformation(gpuResources.getName(), - gpuResources.getUnits(), sched.GPUS_PER_TASK, indices)) + Map(ResourceInformation.GPU -> new ResourceInformation(gpuResources.getName(), indices)) } else { Map.empty[String, ResourceInformation] } 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 7ebfefbef16a7..44c975158c57b 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 @@ -148,8 +148,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorInfo.freeCores += scheduler.CPUS_PER_TASK for ((k, v) <- resources) { executorInfo.availableResources.get(k).foreach { r => - r.incCount(v.getCount()) - r.addAddresses(v.getAddresses()) + r.addAddresses(v.addresses) } } makeOffers(executorId) @@ -219,8 +218,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val data = new ExecutorData(executorRef, executorAddress, hostname, cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, mutable.Map.empty[String, SchedulerResourceInformation] ++= resources.mapValues(v => - new SchedulerResourceInformation(v.getName(), v.getUnits(), v.getCount(), - v.getAddresses().to[ArrayBuffer]))) + new SchedulerResourceInformation(v.name, v.addresses.to[ArrayBuffer]))) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -339,8 +337,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp for ((k, v) <- task.resources) { executorData.availableResources.get(k).foreach { r => - r.decCount(v.getCount()) - r.removeAddresses(v.getAddresses()) + r.removeAddresses(v.addresses) } } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index eb6441d35ee2a..5ec6f65ef148d 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -453,14 +453,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // only specify resource count conf.setResources("gpu", 2, None, false) - assert(conf.getResources(false).get("gpu").get.getCount() == 2) - assert(conf.getResources(false).get("gpu").get.getAddresses().sameElements(Array.empty[String])) + assert(conf.getResources(false).get("gpu").get.addresses.sameElements(Array.empty[String])) conf.validateSettings() // specify resource count and addresses conf.setResources("gpu", 2, Some(Seq("0", "1")), false) - assert(conf.getResources(false).get("gpu").get.getCount() == 2) - assert(conf.getResources(false).get("gpu").get.getAddresses().sameElements(Seq("0", "1"))) + assert(conf.getResources(false).get("gpu").get.addresses.sameElements(Seq("0", "1"))) conf.validateSettings() // conflict number of resource count and addresses size @@ -471,8 +469,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // specify resource on the driver assert(conf.getResources(true).isEmpty) conf.setResources("gpu", 2, Some(Seq("0", "1")), true) - assert(conf.getResources(true).get("gpu").get.getCount() == 2) - assert(conf.getResources(true).get("gpu").get.getAddresses().sameElements(Seq("0", "1"))) + assert(conf.getResources(true).get("gpu").get.addresses.sameElements(Seq("0", "1"))) conf.validateSettings() } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index d6dbae367e28f..62636009be85f 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -859,14 +859,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } val resources = sc.getResources() - assert(resources.get("gpu").get.getAddresses() === Array("0", "1", "8")) - assert(resources.get("gpu").get.getCount() === 3) - assert(resources.get("gpu").get.getName() === "gpu") - assert(resources.get("gpu").get.getUnits() === "") + assert(resources.get("gpu").get.addresses === Array("0", "1", "8")) + assert(resources.get("gpu").get.name === "gpu") val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => val context = TaskContext.get() - context.resources().get(ResourceInformation.GPU).get.getAddresses().iterator + context.resources().get(ResourceInformation.GPU).get.addresses.iterator } val gpus = rdd.collect() assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 634527b752f0e..96df5de934832 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -205,7 +205,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" val resources = Map(ResourceInformation.GPU -> - new ResourceInformation(ResourceInformation.GPU, "", 3, Array("0", "1", "3"))) + new ResourceInformation(ResourceInformation.GPU, Array("0", "1", "3"))) var executorAddedCount: Int = 0 val listener = new SparkListener() { @@ -214,10 +214,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(executorAdded.executorInfo.totalResources.get(ResourceInformation.GPU).nonEmpty) val totalResources = executorAdded.executorInfo.totalResources. get(ResourceInformation.GPU).get - assert(totalResources.getAddresses() === Array("0", "1", "3")) - assert(totalResources.getCount() == 3) - assert(totalResources.getName() == ResourceInformation.GPU) - assert(totalResources.getUnits() == "") + assert(totalResources.addresses === Array("0", "1", "3")) + assert(totalResources.name == ResourceInformation.GPU) } } @@ -236,12 +234,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo var gpuExecResources = backend.getExecutorAvailableResources("1") - assert(gpuExecResources.get("gpu").get.getCount() === 3) assert(gpuExecResources.get("gpu").get.getAddresses() === Array("0", "1", "3")) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], - new Properties(), immutable.Map("gpu" -> new ResourceInformation("gpu", "", 1, Array("0"))), + new Properties(), immutable.Map("gpu" -> new ResourceInformation("gpu", Array("0"))), bytebuffer))) val ts = backend.getTaskSchedulerImpl() // resource offer such that gpu 0 gets removed @@ -251,17 +248,15 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo eventually(timeout(5 seconds), interval(10 millis)) { gpuExecResources = backend.getExecutorAvailableResources("1") - assert(gpuExecResources.get("gpu").get.getCount() === 2) assert(gpuExecResources.get("gpu").get.getAddresses() === Array("1", "3")) } - var finishedTaskResources = Map("gpu" -> new ResourceInformation("gpu", "", 1, Array("0"))) + var finishedTaskResources = Map("gpu" -> new ResourceInformation("gpu", Array("0"))) backend.driverEndpoint.send( StatusUpdate("1", 1, TaskState.FINISHED, buffer, finishedTaskResources)) eventually(timeout(5 seconds), interval(10 millis)) { gpuExecResources = backend.getExecutorAvailableResources("1") - assert(gpuExecResources.get("gpu").get.getCount() === 3) assert(gpuExecResources.get("gpu").get.getAddresses() === Array("1", "3", "0")) } sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala index c59470ef62564..9fcf71d1aec75 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -55,7 +55,7 @@ class TaskDescriptionSuite extends SparkFunSuite { } val originalResources = - Map("gpu" -> new ResourceInformation("gpu", "", 3, Array("1", "2", "3"))) + Map("gpu" -> new ResourceInformation("gpu", Array("1", "2", "3"))) // Create a dummy byte buffer for the task. val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) @@ -94,10 +94,8 @@ class TaskDescriptionSuite extends SparkFunSuite { target: Map[String, ResourceInformation]): Boolean = { original.size == target.size && original.forall { case (name, info) => target.get(name).exists { targetInfo => - info.getName().equals(targetInfo.getName()) && - info.getUnits().equals(targetInfo.getUnits()) && - info.getCount() === targetInfo.getCount() && - info.getAddresses().sameElements(targetInfo.getAddresses()) + info.name.equals(targetInfo.name) && + info.addresses.sameElements(targetInfo.addresses) } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index fa8e400163eec..4c6452dc84e1d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1245,16 +1245,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val executorGpus = 4 val executorCpus = 4 val taskScheduler = setupScheduler(config.CPUS_PER_TASK.key -> taskCpus.toString, - s"${config.SPARK_TASK_RESOURCE_PREFIX}.gpu.${config.SPARK_RESOURCE_COUNT}" -> + s"${config.SPARK_TASK_RESOURCE_PREFIX}gpu${config.SPARK_RESOURCE_COUNT_POSTFIX}" -> taskGpus.toString, - s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}.gpu.${config.SPARK_RESOURCE_COUNT}" -> + s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${config.SPARK_RESOURCE_COUNT_POSTFIX}" -> executorGpus.toString, config.EXECUTOR_CORES.key -> executorCpus.toString) val taskSet = FakeTask.createTaskSet(3) val numFreeCores = 2 val gpuresources = Map("gpu" -> - new SchedulerResourceInformation("gpu", "", 4, ArrayBuffer("0", "1", "2", "3"))) + new SchedulerResourceInformation("gpu", ArrayBuffer("0", "1", "2", "3"))) val singleCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, gpuresources)) val zeroGpuWorkerOffers = @@ -1268,8 +1268,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten assert(2 === taskDescriptions.length) assert(!failedTaskSet) - assert(1 === taskDescriptions(0).resources.get("gpu").get.getCount()) - assert(ArrayBuffer("0") === taskDescriptions(0).resources.get("gpu").get.getAddresses()) - assert(ArrayBuffer("1") === taskDescriptions(1).resources.get("gpu").get.getAddresses()) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get("gpu").get.addresses) + assert(ArrayBuffer("1") === taskDescriptions(1).resources.get("gpu").get.addresses) } } From f27757243b0bd760f9a9e9178796918a07df075a Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 21 May 2019 11:24:22 -0700 Subject: [PATCH 03/18] code clean up --- .../apache/spark/ResourceInformation.scala | 1 + .../scala/org/apache/spark/SparkConf.scala | 122 +------ .../scala/org/apache/spark/TestUtils.scala | 11 + .../CoarseGrainedExecutorBackend.scala | 1 - .../SchedulerResourceInformation.scala | 42 ++- .../spark/scheduler/TaskSchedulerImpl.scala | 34 +- .../spark/scheduler/TaskSetManager.scala | 12 +- .../CoarseGrainedSchedulerBackend.scala | 11 +- .../org/apache/spark/SparkConfSuite.scala | 76 ----- .../org/apache/spark/SparkContextSuite.scala | 52 +-- .../CoarseGrainedSchedulerBackendSuite.scala | 84 +---- .../scheduler/TaskDescriptionSuite.scala | 3 +- .../spark/scheduler/TaskSetManagerSuite.scala | 308 ++++++------------ 13 files changed, 200 insertions(+), 557 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index bc969518d285a..4a89bf4fe313a 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -39,4 +39,5 @@ class ResourceInformation( object ResourceInformation { // known types of resources final val GPU: String = "gpu" + final val UNKNOWN: String = "unknown" } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 6f61ebe05546e..f74241b11fe20 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -21,7 +21,7 @@ import java.util.{Map => JMap} import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, LinkedHashSet} +import scala.collection.mutable.LinkedHashSet import org.apache.avro.{Schema, SchemaNormalization} @@ -507,99 +507,13 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } - /** - * Set available resources on the driver/executor. - */ - private[spark] def setResources( - resourceName: String, - resourceCount: Int, - resourceAddresses: Option[Seq[String]], - isDriver: Boolean): SparkConf = { - if (resourceAddresses.isDefined && resourceAddresses.get.size != resourceCount) { - val nodeType = if (isDriver) { - "driver" - } else { - "executor" - } - throw new SparkException(s"Specified $resourceCount $resourceName(s) on $nodeType, while " + - s"the size of device addresses is ${resourceAddresses.size}.") - } - val prefix = if (isDriver) { - s"$SPARK_DRIVER_RESOURCE_PREFIX$resourceName" - } else { - s"$SPARK_EXECUTOR_RESOURCE_PREFIX$resourceName" - } - set(s"$prefix$SPARK_RESOURCE_COUNT_POSTFIX", resourceCount.toString) - if (resourceAddresses.isDefined) { - set(s"$prefix$SPARK_RESOURCE_ADDRESSES_POSTFIX", resourceAddresses.get.mkString(",")) - } - this - } - - /** - * Set task resource requirement. - */ - private[spark] def setResourceRequirement( - resourceName: String, resourceCount: Int): SparkConf = { - val key = s"$SPARK_TASK_RESOURCE_PREFIX$resourceName$SPARK_RESOURCE_COUNT_POSTFIX" - set(key, resourceCount.toString) - this - } - - /** - * Get available resources on the driver/executor. - */ - private[spark] def getResources(isDriver: Boolean): Map[String, ResourceInformation] = { - val prefix = if (isDriver) { - s"$SPARK_DRIVER_RESOURCE_PREFIX." - } else { - s"$SPARK_EXECUTOR_RESOURCE_PREFIX." - } - - val resourceCountMap = new HashMap[String, Int] - val resourceAddressMap = new HashMap[String, Array[String]] - getAllWithPrefix(prefix).foreach { tuple => - val keys = tuple._1.split("\\.") - if (keys.size == 2) { - val resourceName = keys.head - if (keys.last.equals(SPARK_RESOURCE_COUNT_POSTFIX)) { - val resourceCount = tuple._2.toInt - resourceCountMap.put(resourceName, resourceCount) - } else if (keys.last.equals(SPARK_RESOURCE_ADDRESSES_POSTFIX)) { - val resourceAddresses = tuple._2.split(",").map(_.trim()) - resourceAddressMap.put(resourceName, resourceAddresses) - } - } - } - - resourceCountMap.map { case (resourceName, resourceCount) => - if (resourceAddressMap.contains(resourceName)) { - val resourceAddresses = resourceAddressMap.get(resourceName).get - (resourceName, new ResourceInformation(resourceName, resourceAddresses)) - } else { - (resourceName, new ResourceInformation(resourceName, Array.empty)) - } - }.toMap - } - /** * Get task resource requirements. */ - private[spark] def getResourceRequirements(): Map[String, Int] = { - getAllWithPrefix(s"$SPARK_TASK_RESOURCE_PREFIX.").map { tuple => - val keys = tuple._1.split("\\.") - val resourceName = keys.head - if (keys.size == 2) { - if (keys.last.equals(SPARK_RESOURCE_COUNT_POSTFIX)) { - val resourceCount = tuple._2.toInt - (resourceName, resourceCount) - } else { - (resourceName, 0) - } - } else { - (resourceName, 0) - } - }.filter(_._2 > 0).toMap + private[spark] def getTaskResourceRequirements(): Map[String, Int] = { + getAllWithPrefix(SPARK_TASK_RESOURCE_PREFIX) + .withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)} + .map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.length), v.toInt)}.toMap } /** @@ -686,32 +600,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } - val resources = getResources(false) - val resourceRequirements = getResourceRequirements() - val executorCores = get(EXECUTOR_CORES) - val taskCpus = get(CPUS_PER_TASK) - resourceRequirements.foreach { case (resourceName, resourceCount) => - if (!resources.contains(resourceName)) { - throw new SparkException(s"Task requires resource type $resourceName, which is not " + - s"available on executors.") - } else { - val resourceInformation = resources.get(resourceName).get - val executorResourceCount = resourceInformation.addresses.size - if (executorResourceCount < resourceCount) { - throw new SparkException(s"Available $resourceName resource count on executors is " + - s"$executorResourceCount, which is less than the task requirement $resourceCount.") - } - - if (contains(CPUS_PER_TASK) && contains(EXECUTOR_CORES)) { - if (executorCores > 0 && taskCpus > 0 && - executorCores * resourceCount != executorResourceCount * taskCpus) { - throw new SparkException("Can't make full use of the resources allocated to each " + - "executor with current task resource requirements") - } - } - } - } - val encryptionEnabled = get(NETWORK_CRYPTO_ENABLED) || get(SASL_ENCRYPTION_ENABLED) require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index c97b10ee63b18..8f1af0a5623d0 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -38,6 +38,7 @@ import com.google.common.io.{ByteStreams, Files} import org.apache.log4j.PropertyConfigurator import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.config._ import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -311,6 +312,16 @@ private[spark] object TestUtils { current ++ current.filter(_.isDirectory).flatMap(recursiveList) } + /** + * Set task resource requirement. + */ + def setTaskResourceRequirement( + conf: SparkConf, + resourceName: String, + resourceCount: Int): SparkConf = { + val key = s"$SPARK_TASK_RESOURCE_PREFIX$resourceName$SPARK_RESOURCE_COUNT_POSTFIX" + conf.set(key, resourceCount.toString) + } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index d5f1ac1730a9a..3c715cc868161 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -75,7 +75,6 @@ private[spark] class CoarseGrainedExecutorBackend( // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - val resourceInfo = env.conf.getResources(false) ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, extractAttributes, resources)) }(ThreadUtils.sameThread).onComplete { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala index e0d85e0035c6f..fc1da82de7c81 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala @@ -19,38 +19,44 @@ package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer -import org.apache.spark.ResourceInformation +import org.apache.spark.ResourceInformation.UNKNOWN import org.apache.spark.annotation.Evolving -import org.apache.spark.internal.Logging /** - * Class to hold information about a type of Resource used by the scheduler. This - * is a separate class from the ResourceInformation because here its mutable because the - * scheduler has to update the count and addresses based on what its assigned and - * what is available. + * Class to hold information about a type of Resource used by the scheduler. This is a separate + * class from the ResourceInformation because here its mutable because the scheduler has to update + * the addresses based on what its assigned and what is available. */ @Evolving private[spark] class SchedulerResourceInformation( private val name: String, - private val addresses: ArrayBuffer[String] = ArrayBuffer.empty) extends Logging { + private val availableAddresses: ArrayBuffer[String] = ArrayBuffer.empty) extends Serializable { - def getName(): String = name + private val allocatedAddresses: ArrayBuffer[String] = ArrayBuffer.empty - def getAddresses(): ArrayBuffer[String] = addresses + def getName(): String = name - def addAddresses(addrs: Array[String]): Unit = { - addresses ++= addrs - } + def getAvailableAddresses(): ArrayBuffer[String] = availableAddresses - def takeAddresses(count: Int): ArrayBuffer[String] = { - addresses.take(count) + def acquireAddresses(num: Int): Seq[String] = { + assert(num <= availableAddresses.size, s"Required to take $num $name addresses but only " + + s"${availableAddresses.size} available.") + val addrs = availableAddresses.take(num) + allocatedAddresses ++= addrs + addrs } - def removeAddresses(addrs: Array[String]): Unit = { - addresses --= addrs + def releaseAddresses(addrs: Array[String]): Unit = { + addrs.foreach { address => + assert(allocatedAddresses.contains(address), s"Try to release $name address $address, but " + + "it is not allocated.") + availableAddresses += address + allocatedAddresses -= address + } } } + private[spark] object SchedulerResourceInformation { - def empty: SchedulerResourceInformation = new SchedulerResourceInformation("gpu", - ArrayBuffer.empty[String]) + def empty: SchedulerResourceInformation = + new SchedulerResourceInformation(UNKNOWN, ArrayBuffer.empty[String]) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 94f2e4f74ff54..7e39aade530c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -92,8 +92,8 @@ private[spark] class TaskSchedulerImpl( // CPUs to request per task val CPUS_PER_TASK = conf.get(config.CPUS_PER_TASK) - // GPUs to request per task - val GPUS_PER_TASK = conf.getResourceRequirements().getOrElse("gpu", 0) + // Resources to request per task + val RESOURCES_PER_TASK = conf.getTaskResourceRequirements() // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. Protected by `this` @@ -330,7 +330,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], - availableGpuIndices: Array[ArrayBuffer[String]], + availableResources: Array[Map[String, SchedulerResourceInformation]], tasks: IndexedSeq[ArrayBuffer[TaskDescription]], addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = { var launchedTask = false @@ -339,22 +339,17 @@ private[spark] class TaskSchedulerImpl( for (i <- 0 until shuffledOffers.size) { val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK && availableGpuIndices(i).size >= GPUS_PER_TASK) { + if (availableCpus(i) >= CPUS_PER_TASK && + resourceMeetTaskRequirements(availableResources(i))) { try { - val gpuResources = shuffledOffers(i).resources. - getOrElse("gpu", SchedulerResourceInformation.empty) - for (task <- taskSet.resourceOffer(execId, host, maxLocality, - availableGpuIndices(i), gpuResources)) { + for (task <- taskSet.resourceOffer(execId, host, maxLocality, availableResources(i))) { tasks(i) += task val tid = task.taskId taskIdToTaskSetManager.put(tid, taskSet) taskIdToExecutorId(tid) = execId executorIdToRunningTaskIds(execId).add(tid) availableCpus(i) -= CPUS_PER_TASK - task.resources.get("gpu").foreach { addrs => - availableGpuIndices(i) --= addrs.addresses - } - assert(availableCpus(i) >= 0 && availableGpuIndices(i).size >= 0) + assert(availableCpus(i) >= 0) // Only update hosts for a barrier task. if (taskSet.isBarrier) { // The executor address is expected to be non empty. @@ -374,6 +369,16 @@ private[spark] class TaskSchedulerImpl( launchedTask } + /** + * Check whether the resources from the WorkerOffer are enough to run at least one task. + */ + private def resourceMeetTaskRequirements( + resources: Map[String, SchedulerResourceInformation]): Boolean = { + RESOURCES_PER_TASK.forall { case (rName, rNum) => + resources.contains(rName) && resources(rName).getAvailableAddresses().size >= rNum + } + } + /** * Called by cluster manager 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 @@ -416,9 +421,6 @@ private[spark] class TaskSchedulerImpl( // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK)) val availableResources = shuffledOffers.map(_.resources).toArray - val gpuResources = availableResources.map(_.getOrElse(ResourceInformation.GPU, - SchedulerResourceInformation.empty)) - val availableGpuIndices = gpuResources.map(_.getAddresses()) val availableCpus = shuffledOffers.map(o => o.cores).toArray val availableSlots = shuffledOffers.map(o => o.cores / CPUS_PER_TASK).sum val sortedTaskSets = rootPool.getSortedTaskSetQueue @@ -451,7 +453,7 @@ private[spark] class TaskSchedulerImpl( do { launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet(taskSet, currentMaxLocality, shuffledOffers, availableCpus, - availableGpuIndices, tasks, addressesWithDescs) + availableResources, tasks, addressesWithDescs) launchedAnyTask |= launchedTaskAtCurrentMaxLocality } while (launchedTaskAtCurrentMaxLocality) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 54b0107c2820c..a84e988a51b18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -469,8 +469,7 @@ private[spark] class TaskSetManager( execId: String, host: String, maxLocality: TaskLocality.TaskLocality, - hostGpuIndices: ArrayBuffer[String], - gpuResources: SchedulerResourceInformation) + availableResources: Map[String, SchedulerResourceInformation]) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -535,12 +534,9 @@ private[spark] class TaskSetManager( logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") - val extraResources = if (sched.GPUS_PER_TASK > 0) { - // doing minimal checking here to keep things fast - val indices = hostGpuIndices.take(sched.GPUS_PER_TASK).toArray - Map(ResourceInformation.GPU -> new ResourceInformation(gpuResources.getName(), indices)) - } else { - Map.empty[String, ResourceInformation] + val extraResources = sched.RESOURCES_PER_TASK.map { case (rName, rNum) => + val allocatedAddresses = availableResources.get(rName).get.acquireAddresses(rNum) + (rName, new ResourceInformation(rName, allocatedAddresses.toArray)) } sched.dagScheduler.taskStarted(task, info) 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 44c975158c57b..257b6a52f2e66 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 @@ -148,7 +148,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorInfo.freeCores += scheduler.CPUS_PER_TASK for ((k, v) <- resources) { executorInfo.availableResources.get(k).foreach { r => - r.addAddresses(v.addresses) + r.releaseAddresses(v.addresses) } } makeOffers(executorId) @@ -330,17 +330,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case e: Exception => logError("Exception in error callback", e) } } - } - else { + } else { val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK - for ((k, v) <- task.resources) { - executorData.availableResources.get(k).foreach { r => - r.removeAddresses(v.addresses) - } - } - logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + s"${executorData.executorHost}.") diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 5ec6f65ef148d..965c7df69cfe3 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -446,82 +446,6 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(thrown.getMessage.contains(key)) } } - - test("specify resource count and addresses") { - val conf = new SparkConf() - conf.validateSettings() - - // only specify resource count - conf.setResources("gpu", 2, None, false) - assert(conf.getResources(false).get("gpu").get.addresses.sameElements(Array.empty[String])) - conf.validateSettings() - - // specify resource count and addresses - conf.setResources("gpu", 2, Some(Seq("0", "1")), false) - assert(conf.getResources(false).get("gpu").get.addresses.sameElements(Seq("0", "1"))) - conf.validateSettings() - - // conflict number of resource count and addresses size - intercept[SparkException] { - conf.setResources("gpu", 2, Some(Seq("0", "1", "2")), false) - } - - // specify resource on the driver - assert(conf.getResources(true).isEmpty) - conf.setResources("gpu", 2, Some(Seq("0", "1")), true) - assert(conf.getResources(true).get("gpu").get.addresses.sameElements(Seq("0", "1"))) - conf.validateSettings() - } - - test("specify resource requirement") { - val conf = new SparkConf() - conf.validateSettings() - - // resource required but not on executors. - assert(conf.getResourceRequirements().isEmpty) - conf.setResourceRequirement("gpu", 2) - assert(conf.getResourceRequirements().get("gpu").get == 2) - intercept[SparkException] { - conf.validateSettings() - } - - // specify resource on executors. - conf.setResources("gpu", 2, None, false) - conf.validateSettings() - } - - test("verify available resources and resource requirements") { - val conf = new SparkConf() - conf.validateSettings() - - // resource on executors less than resource requirement - conf.setResourceRequirement("gpu", 3) - conf.setResources("gpu", 2, None, false) - intercept[SparkException] { - conf.validateSettings() - } - - // valid case - conf.setResourceRequirement("gpu", 2) - conf.setResources("gpu", 2, None, false) - conf.validateSettings() - - // resource on executors not fully used compated to resource requirement - conf.setResourceRequirement("gpu", 3) - conf.setResources("gpu", 3, None, false) - conf.set(EXECUTOR_CORES.key, "4") - conf.set(CPUS_PER_TASK.key, "1") - intercept[SparkException] { - conf.validateSettings() - } - - // valid case - conf.setResourceRequirement("gpu", 2) - conf.setResources("gpu", 8, None, false) - conf.set(EXECUTOR_CORES.key, "4") - conf.set(CPUS_PER_TASK.key, "1") - conf.validateSettings() - } } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 62636009be85f..edc7d5b53b0ed 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -843,34 +843,40 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu file.getPath() } + test("test resource scheduling under local-cluster mode") { + import org.apache.spark.TestUtils._ - test("test gpu support under local-cluster mode") { - val conf = new SparkConf() - .setResources("gpu", 3, Some(Seq("0", "1", "2")), false) - .setResources("gpu", 3, Some(Seq("0", "1", "8")), true) - .setResourceRequirement("gpu", 1) - .setMaster("local-cluster[3, 3, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + withTempDir { dir => + val resourceFile = new File(dir, "resourceDiscoverScript") + val resources = """'{"name": "gpu", "addresses": ["0", "1", "2"]}'""" + Files.write(s"echo $resources", resourceFile, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(resourceFile.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + val discoveryScript = resourceFile.getPath() - // Ensure all executors has started - eventually(timeout(10.seconds)) { - assert(sc.statusTracker.getExecutorInfos.size == 3) - } + val conf = new SparkConf() + .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX}", + discoveryScript) + .setMaster("local-cluster[3, 3, 1024]") + .setAppName("test-cluster") + setTaskResourceRequirement(conf, "gpu", 1) + sc = new SparkContext(conf) - val resources = sc.getResources() - assert(resources.get("gpu").get.addresses === Array("0", "1", "8")) - assert(resources.get("gpu").get.name === "gpu") + // Ensure all executors has started + eventually(timeout(10.seconds)) { + assert(sc.statusTracker.getExecutorInfos.size == 3) + } - val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => - val context = TaskContext.get() - context.resources().get(ResourceInformation.GPU).get.addresses.iterator - } - val gpus = rdd.collect() - assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) + val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => + val context = TaskContext.get() + context.resources().get(ResourceInformation.GPU).get.addresses.iterator + } + val gpus = rdd.collect() + assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) - eventually(timeout(10.seconds)) { - assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + eventually(timeout(10.seconds)) { + assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 96df5de934832..63daeaa8e38c6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar._ import org.apache.spark._ +import org.apache.spark.ResourceInformation.GPU import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD @@ -180,89 +181,6 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(executorAddedCount === 3) } - test("extra gpu resources from executor") { - val conf = new SparkConf() - .setResources("gpu", 3, None, false) - .setResourceRequirement("gpu", 3) - .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test - .setMaster( - "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") - .setAppName("test") - - sc = new SparkContext(conf) - val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] - val mockEndpointRef = mock[RpcEndpointRef] - val mockAddress = mock[RpcAddress] - - val logUrls = Map( - "stdout" -> "http://oldhost:8888/logs/dummy/stdout", - "stderr" -> "http://oldhost:8888/logs/dummy/stderr") - val attributes = Map( - "CLUSTER_ID" -> "cl1", - "USER" -> "dummy", - "CONTAINER_ID" -> "container1", - "LOG_FILES" -> "stdout,stderr") - val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + - s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" - val resources = Map(ResourceInformation.GPU -> - new ResourceInformation(ResourceInformation.GPU, Array("0", "1", "3"))) - - var executorAddedCount: Int = 0 - val listener = new SparkListener() { - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { - executorAddedCount += 1 - assert(executorAdded.executorInfo.totalResources.get(ResourceInformation.GPU).nonEmpty) - val totalResources = executorAdded.executorInfo.totalResources. - get(ResourceInformation.GPU).get - assert(totalResources.addresses === Array("0", "1", "3")) - assert(totalResources.name == ResourceInformation.GPU) - } - } - - sc.addSparkListener(listener) - - backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) - backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) - backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) - - val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) - val bytebuffer = java.nio.ByteBuffer.allocate(frameSize/2) - val buffer = new SerializableBuffer(bytebuffer) - - var gpuExecResources = backend.getExecutorAvailableResources("1") - - assert(gpuExecResources.get("gpu").get.getAddresses() === Array("0", "1", "3")) - - var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", - "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], - new Properties(), immutable.Map("gpu" -> new ResourceInformation("gpu", Array("0"))), - bytebuffer))) - val ts = backend.getTaskSchedulerImpl() - // resource offer such that gpu 0 gets removed - when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(taskDescs) - - backend.driverEndpoint.send(ReviveOffers) - - eventually(timeout(5 seconds), interval(10 millis)) { - gpuExecResources = backend.getExecutorAvailableResources("1") - assert(gpuExecResources.get("gpu").get.getAddresses() === Array("1", "3")) - } - - var finishedTaskResources = Map("gpu" -> new ResourceInformation("gpu", Array("0"))) - backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, finishedTaskResources)) - - eventually(timeout(5 seconds), interval(10 millis)) { - gpuExecResources = backend.getExecutorAvailableResources("1") - assert(gpuExecResources.get("gpu").get.getAddresses() === Array("1", "3", "0")) - } - sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) - assert(executorAddedCount === 3) - } - private def testSubmitJob(sc: SparkContext, rdd: RDD[Int]): Unit = { sc.submitJob( rdd, diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala index 9fcf71d1aec75..0677205e42895 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -24,6 +24,7 @@ import java.util.Properties import scala.collection.mutable.HashMap import org.apache.spark.ResourceInformation +import org.apache.spark.ResourceInformation.GPU import org.apache.spark.SparkFunSuite class TaskDescriptionSuite extends SparkFunSuite { @@ -55,7 +56,7 @@ class TaskDescriptionSuite extends SparkFunSuite { } val originalResources = - Map("gpu" -> new ResourceInformation("gpu", Array("1", "2", "3"))) + Map(GPU -> new ResourceInformation(GPU, Array("1", "2", "3"))) // Create a dummy byte buffer for the task. val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 66c592198845e..72b2b13a7da14 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -210,8 +210,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption.isDefined) clock.advance(1) @@ -232,8 +231,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -241,8 +239,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) @@ -265,14 +262,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. - assert(manager.resourceOffer("execC", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("execC", "host2", ANY, Map.empty) === None) // Because there are no alive PROCESS_LOCAL executors, the base locality level should be // NODE_LOCAL. So, we should schedule the task on this offered NODE_LOCAL executor before // any of the locality wait timers expire. - assert(manager.resourceOffer("execA", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY, Map.empty).get.index === 0) } test("basic delay scheduling") { @@ -287,28 +282,22 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty) == None) clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 3) should // get chosen before the noPref task - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index == 2) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, Map.empty).get.index == 2) // Offer host2, exec2, at NODE_LOCAL level: we should choose task 2 - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index == 1) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, Map.empty).get.index == 1) // Offer host2, exec2 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, Map.empty) == None) clock.advance(LOCALITY_WAIT_MS) - assert(manager.resourceOffer("exec2", "host2", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index == 3) + assert(manager.resourceOffer("exec2", "host2", NO_PREF, Map.empty).get.index == 3) } test("we do not need to delay scheduling when we only have noPref tasks in the queue") { @@ -322,14 +311,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) - assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) - assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) - assert(manager.resourceOffer("exec3", "host2", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL, Map.empty).get.index === 1) + assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("exec3", "host2", NO_PREF, Map.empty).get.index === 2) } test("delay scheduling with fallback") { @@ -347,36 +332,29 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) // Offer host1 again: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 1) // Offer host1 again: third task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 2) // Offer host2: fifth task (also on host2) should get chosen - assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 4) + assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty).get.index === 4) // Now that we've launched a local task, we should no longer launch the task for host3 - assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty) === None) clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally - assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 3) + assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty).get.index === 3) } test("delay scheduling with failed hosts") { @@ -392,35 +370,28 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) // After this, nothing should get chosen, because we have separated tasks with unavailable // preference from the noPrefPendingTasks - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2", SlaveLost()) // nothing should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 2) // all finished - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) - assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) + assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty) === None) } test("task result lost") { @@ -431,16 +402,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) // Tell it the task has finished but the result was lost. manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost) assert(sched.endedTasks(0) === TaskResultLost) // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) } test("repeated failures lead to task set abortion") { @@ -454,8 +423,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. (1 to manager.maxTaskFailures).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val offerResult = manager.resourceOffer("exec1", "host1", ANY, Map.empty) assert(offerResult.isDefined, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) @@ -491,8 +459,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -503,20 +470,15 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) - assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, Map.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL, Map.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, Map.empty) assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -528,14 +490,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, Map.empty).isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val offerResult = manager.resourceOffer("exec2", "host2", ANY, Map.empty) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -546,8 +506,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty).isEmpty) } // Despite advancing beyond the time for expiring executors from within the blacklist, @@ -555,20 +514,17 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty) assert(offerResult.isEmpty) } { - val offerResult = manager.resourceOffer("exec3", "host3", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val offerResult = manager.resourceOffer("exec3", "host3", ANY, Map.empty) assert(offerResult.isDefined) assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec3", "host3", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY, Map.empty).isEmpty) // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) @@ -627,16 +583,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg manager.executorAdded() sched.addExecutor("execC", "host2") manager.executorAdded() - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isDefined) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).isDefined) sched.removeExecutor("execA") manager.executorLost( "execA", "host1", ExecutorExited(143, false, "Terminated for reason unrelated to running tasks")) assert(!sched.taskSetsFailed.contains(taskSet.id)) - assert(manager.resourceOffer("execC", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).isDefined) + assert(manager.resourceOffer("execC", "host2", ANY, Map.empty).isDefined) sched.removeExecutor("execC") manager.executorLost( "execC", "host2", ExecutorExited(1, true, "Terminated due to issue with running tasks")) @@ -664,15 +618,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL - assert(manager.resourceOffer("execC", "host3", RACK_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("execC", "host3", RACK_LOCAL, Map.empty) === None) // Task 0 can be scheduled with ANY - assert(manager.resourceOffer("execC", "host3", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("execC", "host3", ANY, Map.empty).get.index === 0) // Offer host2 // Task 1 can be scheduled with RACK_LOCAL - assert(manager.resourceOffer("execB", "host2", RACK_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("execB", "host2", RACK_LOCAL, Map.empty).get.index === 1) } test("do not emit warning when serialized task is small") { @@ -683,8 +634,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) assert(!manager.emittedTaskSizeWarning) } @@ -698,8 +648,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) assert(manager.emittedTaskSizeWarning) } @@ -713,8 +662,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { - manager.resourceOffer("exec1", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + manager.resourceOffer("exec1", "host1", ANY, Map.empty) } assert(manager.isZombie) } @@ -785,14 +733,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer host1, which should be accepted as a PROCESS_LOCAL location // by the one task in the task set - val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL, - ArrayBuffer.empty, SchedulerResourceInformation.empty).get + val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL, Map.empty).get // Mark the task as available for speculation, and then offer another resource, // which should be used to launch a speculative copy of the task. manager.speculatableTasks += singleTask.partitionId - val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get + val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY, Map.empty).get assert(manager.runningTasks === 2) assert(manager.isZombie === false) @@ -877,8 +823,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer(exec, host, NO_PREF, Map.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -904,8 +849,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer("exec2", "host2", ANY, Map.empty) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -934,25 +878,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index == 1) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, Map.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF, Map.empty).get.index == 1) manager.speculatableTasks += 1 clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task - assert(manager.resourceOffer("execA", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("execA", "host1", NO_PREF, Map.empty).get.index === 2) // schedule the speculative task - assert(manager.resourceOffer("execB", "host2", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) + assert(manager.resourceOffer("execB", "host2", NO_PREF, Map.empty).get.index === 1) clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks - assert(manager.resourceOffer("execB", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 3) + assert(manager.resourceOffer("execB", "host2", ANY, Map.empty).get.index === 3) } test("node-local tasks should be scheduled right away " + @@ -969,18 +907,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) - assert(manager.resourceOffer("execA", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 3) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) === None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host2", NODE_LOCAL, Map.empty).get.index === 1) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, Map.empty).get.index === 3) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, Map.empty) === None) // schedule no-preference after node local ones - assert(manager.resourceOffer("execA", "host3", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("execA", "host3", NO_PREF, Map.empty).get.index === 2) } test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") @@ -996,19 +929,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 3) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty).get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, Map.empty).get.index === 3) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty).get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, Map.empty).get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, Map.empty) == None) } test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { @@ -1022,19 +949,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 1) - assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 2) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, Map.empty).get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL, Map.empty).get.index === 2) // no-pref tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty).get.index === 0) - assert(manager.resourceOffer("execA", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) == None) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF, Map.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY, Map.empty) == None) } test("Ensure TaskSetManager is usable after addition of levels") { @@ -1055,11 +976,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.pendingTasksWithNoPrefs.size === 0) // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) - assert(manager.resourceOffer("execA", "host1", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) !== None) + assert(manager.resourceOffer("execA", "host1", ANY, Map.empty) !== None) clock.advance(LOCALITY_WAIT_MS * 4) - assert(manager.resourceOffer("execB.2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) !== None) + assert(manager.resourceOffer("execB.2", "host2", ANY, Map.empty) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1", SlaveLost()) @@ -1068,8 +987,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: - assert(manager.resourceOffer("execC", "host3", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) !== None) + assert(manager.resourceOffer("execC", "host3", ANY, Map.empty) !== None) } test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { @@ -1121,8 +1039,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer(k, v, NO_PREF, Map.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1143,8 +1060,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1183,8 +1099,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer(k, v, NO_PREF, Map.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1217,8 +1132,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) sched.endedTasks(task.taskId) = endReason assert(!manager.isZombie) - val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF, Map.empty) assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") tasks += nextTask.get } @@ -1234,8 +1148,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(3, 4)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption5.isDefined) val speculativeTask = taskOption5.get assert(speculativeTask.index === 3 || speculativeTask.index === 4) @@ -1260,8 +1173,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.isZombie) // now run another speculative task - val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOpt6.isDefined) val speculativeTask2 = taskOpt6.get assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) @@ -1292,8 +1204,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any())).thenAnswer( (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption.isDefined) // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -1338,8 +1249,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec2" -> "host1" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY, - ArrayBuffer.empty, SchedulerResourceInformation.empty)} + (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY, Map.empty)} } assert(taskDescs.size === 4) @@ -1376,8 +1286,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec2" -> "host2" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty)} + (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY, Map.empty)} } assert(taskDescs.size === 4) @@ -1413,8 +1322,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) val taskSetManagerSpy = spy(taskSetManager) - val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY, Map.empty) // Assert the task has been black listed on the executor it was last executed on. when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( @@ -1442,11 +1350,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager1 = new TaskSetManager(sched, taskSet1, MAX_TASK_FAILURES, clock = new ManualClock) // all tasks from the first taskset have the same jars - val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption1.get.addedJars === addedJarsPreTaskSet) - val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption2.get.addedJars === addedJarsPreTaskSet) // even with a jar added mid-TaskSet @@ -1454,8 +1360,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sc.addJar(jarPath.toString) val addedJarsMidTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) assert(addedJarsPreTaskSet !== addedJarsMidTaskSet) - val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF, Map.empty) // which should have the old version of the jars list assert(taskOption3.get.addedJars === addedJarsPreTaskSet) @@ -1463,8 +1368,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet2 = FakeTask.createTaskSet(1) val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) - val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } @@ -1561,8 +1465,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer(exec, host, NO_PREF, Map.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -1588,8 +1491,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer("exec2", "host2", ANY, Map.empty) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -1634,8 +1536,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption = manager.resourceOffer(k, v, NO_PREF, Map.empty) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1655,8 +1556,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, ArrayBuffer.empty, - SchedulerResourceInformation.empty) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1716,7 +1616,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(FakeRackUtil.numSingleHostInvocation === 0) // with rack locality, reject an offer on a host with an unknown rack assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL, - ArrayBuffer.empty, SchedulerResourceInformation.empty).isEmpty) + Map.empty).isEmpty) (0 until 20).foreach { rackIdx => (0 until 5).foreach { offerIdx => // if we offer hosts which are not in preferred locations, @@ -1724,11 +1624,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // but accept them at RACK_LOCAL level if they're on OK racks val hostIdx = 100 + rackIdx assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL, - ArrayBuffer.empty, SchedulerResourceInformation.empty) - .isEmpty) + Map.empty).isEmpty) assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL, - ArrayBuffer.empty, SchedulerResourceInformation.empty) - .isDefined) + Map.empty).isDefined) } } // check no more expensive calls to the rack resolution. manager.resourceOffer() will call From 3b5c5c75b776828dc31bdee7c95559402778b588 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 21 May 2019 14:35:33 -0700 Subject: [PATCH 04/18] update tests --- .../apache/spark/scheduler/SchedulerResourceInformation.scala | 1 + core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala index fc1da82de7c81..697c66ace99db 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala @@ -42,6 +42,7 @@ private[spark] class SchedulerResourceInformation( assert(num <= availableAddresses.size, s"Required to take $num $name addresses but only " + s"${availableAddresses.size} available.") val addrs = availableAddresses.take(num) + availableAddresses --= addrs allocatedAddresses ++= addrs addrs } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index edc7d5b53b0ed..6c088377fde17 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -863,7 +863,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) // Ensure all executors has started - eventually(timeout(10.seconds)) { + eventually(timeout(60.seconds)) { assert(sc.statusTracker.getExecutorInfos.size == 3) } From d116591a7d53b6f9a4e1575149834f6956b222fc Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 21 May 2019 16:36:53 -0700 Subject: [PATCH 05/18] update --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 6c088377fde17..2fd2f18198b99 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -855,6 +855,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val discoveryScript = resourceFile.getPath() val conf = new SparkConf() + .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_COUNT_POSTFIX}", "3") .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX}", discoveryScript) .setMaster("local-cluster[3, 3, 1024]") From 9a2601f2b567b804d8d9922bae798315d42d3a20 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 23 May 2019 16:34:03 -0700 Subject: [PATCH 06/18] code clean up and add test cases --- .../apache/spark/ResourceInformation.scala | 1 - .../scala/org/apache/spark/SparkConf.scala | 4 ++-- .../scala/org/apache/spark/TaskContext.scala | 3 ++- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 1 - .../SchedulerResourceInformation.scala | 6 ------ .../spark/scheduler/TaskSchedulerImpl.scala | 4 ++-- .../CoarseGrainedSchedulerBackend.scala | 3 ++- .../org/apache/spark/SparkContextSuite.scala | 4 ++-- .../scheduler/TaskSchedulerImplSuite.scala | 17 ++++++++------- .../spark/scheduler/TaskSetManagerSuite.scala | 21 +++++++++++++++++++ 11 files changed, 41 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index 4a89bf4fe313a..bc969518d285a 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -39,5 +39,4 @@ class ResourceInformation( object ResourceInformation { // known types of resources final val GPU: String = "gpu" - final val UNKNOWN: String = "unknown" } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index f74241b11fe20..eab89ccf9e372 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -512,8 +512,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria */ private[spark] def getTaskResourceRequirements(): Map[String, Int] = { getAllWithPrefix(SPARK_TASK_RESOURCE_PREFIX) - .withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)} - .map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.length), v.toInt)}.toMap + .withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_SUFFIX)} + .map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_SUFFIX.length), v.toInt)}.toMap } /** diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 443f0fad5ed54..04248dfa2f735 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -177,7 +177,8 @@ abstract class TaskContext extends Serializable { def getLocalProperty(key: String): String /** - * Other Resources allocated to the task. Currently gpus are the only resource supported. + * Resources allocated to the task. A task may retrieve the resource name and available resource + * addresses from the map values. */ @Evolving def resources(): Map[String, ResourceInformation] diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 8f1af0a5623d0..ccfefc9b27014 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -319,7 +319,7 @@ private[spark] object TestUtils { conf: SparkConf, resourceName: String, resourceCount: Int): SparkConf = { - val key = s"$SPARK_TASK_RESOURCE_PREFIX$resourceName$SPARK_RESOURCE_COUNT_POSTFIX" + val key = s"$SPARK_TASK_RESOURCE_PREFIX$resourceName$SPARK_RESOURCE_COUNT_SUFFIX" conf.set(key, resourceCount.toString) } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3c715cc868161..da2bddfc49515 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -74,7 +74,6 @@ private[spark] class CoarseGrainedExecutorBackend( rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, extractAttributes, resources)) }(ThreadUtils.sameThread).onComplete { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala index 697c66ace99db..fed97294710a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer -import org.apache.spark.ResourceInformation.UNKNOWN import org.apache.spark.annotation.Evolving /** @@ -56,8 +55,3 @@ private[spark] class SchedulerResourceInformation( } } } - -private[spark] object SchedulerResourceInformation { - def empty: SchedulerResourceInformation = - new SchedulerResourceInformation(UNKNOWN, ArrayBuffer.empty[String]) -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7e39aade530c8..c22e53f7c0156 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -340,7 +340,7 @@ private[spark] class TaskSchedulerImpl( val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host if (availableCpus(i) >= CPUS_PER_TASK && - resourceMeetTaskRequirements(availableResources(i))) { + resourcesMeetTaskRequirements(availableResources(i))) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality, availableResources(i))) { tasks(i) += task @@ -372,7 +372,7 @@ private[spark] class TaskSchedulerImpl( /** * Check whether the resources from the WorkerOffer are enough to run at least one task. */ - private def resourceMeetTaskRequirements( + private def resourcesMeetTaskRequirements( resources: Map[String, SchedulerResourceInformation]): Boolean = { RESOURCES_PER_TASK.forall { case (rName, rNum) => resources.contains(rName) && resources(rName).getAvailableAddresses().size >= rNum 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 257b6a52f2e66..a3632b923670e 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 @@ -330,7 +330,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case e: Exception => logError("Exception in error callback", e) } } - } else { + } + else { val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 2fd2f18198b99..1da07653954e0 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -855,8 +855,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val discoveryScript = resourceFile.getPath() val conf = new SparkConf() - .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_COUNT_POSTFIX}", "3") - .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX}", + .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_COUNT_SUFFIX}", "3") + .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_DISCOVERY_SCRIPT_SUFFIX}", discoveryScript) .setMaster("local-cluster[3, 3, 1024]") .setAppName("test-cluster") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 4c6452dc84e1d..a9e3732bc286e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar import org.apache.spark._ +import org.apache.spark.ResourceInformation.GPU import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.util.ManualClock @@ -1245,18 +1246,18 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val executorGpus = 4 val executorCpus = 4 val taskScheduler = setupScheduler(config.CPUS_PER_TASK.key -> taskCpus.toString, - s"${config.SPARK_TASK_RESOURCE_PREFIX}gpu${config.SPARK_RESOURCE_COUNT_POSTFIX}" -> + s"${config.SPARK_TASK_RESOURCE_PREFIX}$GPU${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> taskGpus.toString, - s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${config.SPARK_RESOURCE_COUNT_POSTFIX}" -> + s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}$GPU${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> executorGpus.toString, config.EXECUTOR_CORES.key -> executorCpus.toString) val taskSet = FakeTask.createTaskSet(3) val numFreeCores = 2 - val gpuresources = Map("gpu" -> - new SchedulerResourceInformation("gpu", ArrayBuffer("0", "1", "2", "3"))) + val resources = Map(GPU -> + new SchedulerResourceInformation(GPU, ArrayBuffer("0", "1", "2", "3"))) val singleCoreWorkerOffers = - IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, gpuresources)) + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, resources)) val zeroGpuWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, Map.empty)) taskScheduler.submitTasks(taskSet) @@ -1264,11 +1265,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B var taskDescriptions = taskScheduler.resourceOffers(zeroGpuWorkerOffers).flatten assert(0 === taskDescriptions.length) assert(!failedTaskSet) - // Launch tasks on executor that satisfies GPU resource requirements. + // Launch tasks on executor that satisfies resource requirements. taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten assert(2 === taskDescriptions.length) assert(!failedTaskSet) - assert(ArrayBuffer("0") === taskDescriptions(0).resources.get("gpu").get.addresses) - assert(ArrayBuffer("1") === taskDescriptions(1).resources.get("gpu").get.addresses) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.addresses) + assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.addresses) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 72b2b13a7da14..8d7e9352ffd40 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.apache.spark._ +import org.apache.spark.ResourceInformation.GPU import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.serializer.SerializerInstance @@ -1634,4 +1635,24 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // by that point. assert(FakeRackUtil.numBatchInvocation === 1) } + + test("TaskSetManager allocate resource addresses from available resources") { + import TestUtils._ + + sc = new SparkContext("local", "test") + setTaskResourceRequirement(sc.conf, GPU, 2) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = FakeTask.createTaskSet(1) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + + val availableResources = Map(GPU -> + new SchedulerResourceInformation(GPU, ArrayBuffer("0", "1", "2", "3"))) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, availableResources) + assert(taskOption.isDefined) + val allocatedResources = taskOption.get.resources + assert(allocatedResources.size == 1) + assert(allocatedResources(GPU).addresses sameElements Array("0", "1")) + // Allocated resource addresses should no longer be available in `availableResources`. + assert(availableResources(GPU).getAvailableAddresses() sameElements Array("2", "3")) + } } From 5e36aa09d653fe361a25049d1b907eddd621992d Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 28 May 2019 17:15:23 -0700 Subject: [PATCH 07/18] Refactor the internal Executor Resource information. --- .../apache/spark/ResourceInformation.scala | 5 --- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 + .../spark/internal/config/package.scala | 7 +-- ...ation.scala => ExecutorResourceInfo.scala} | 44 ++++++++++++------- .../spark/scheduler/TaskSchedulerImpl.scala | 10 ++--- .../spark/scheduler/TaskSetManager.scala | 6 +-- .../apache/spark/scheduler/WorkerOffer.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 12 +++-- .../scheduler/cluster/ExecutorData.scala | 4 +- .../scala/org/apache/spark/ResourceName.scala | 23 ++++++++++ .../org/apache/spark/SparkConfSuite.scala | 23 ++++++++++ .../org/apache/spark/SparkContextSuite.scala | 7 +-- .../CoarseGrainedSchedulerBackendSuite.scala | 2 +- .../scheduler/TaskDescriptionSuite.scala | 2 +- .../scheduler/TaskSchedulerImplSuite.scala | 8 ++-- .../spark/scheduler/TaskSetManagerSuite.scala | 6 +-- 17 files changed, 111 insertions(+), 54 deletions(-) rename core/src/main/scala/org/apache/spark/scheduler/{SchedulerResourceInformation.scala => ExecutorResourceInfo.scala} (50%) create mode 100644 core/src/test/scala/org/apache/spark/ResourceName.scala diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index bc969518d285a..6a5b725ac21d7 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -35,8 +35,3 @@ class ResourceInformation( override def toString: String = s"[name: ${name}, addresses: ${addresses.mkString(",")}]" } - -object ResourceInformation { - // known types of resources - final val GPU: String = "gpu" -} diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index ccfefc9b27014..d306eed757b63 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -319,7 +319,7 @@ private[spark] object TestUtils { conf: SparkConf, resourceName: String, resourceCount: Int): SparkConf = { - val key = s"$SPARK_TASK_RESOURCE_PREFIX$resourceName$SPARK_RESOURCE_COUNT_SUFFIX" + val key = s"${SPARK_TASK_RESOURCE_PREFIX}${resourceName}${SPARK_RESOURCE_COUNT_SUFFIX}" conf.set(key, resourceCount.toString) } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index da2bddfc49515..76295b984820c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -66,6 +66,8 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + // Map each taskId to the resource informations allocated to it, the resource information + // includes resource name and resource addresses. private[this] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] override def onStart() { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index d320272c7d3c4..8ea88878c40ea 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -188,7 +188,6 @@ package object config { private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) .intConf - .checkValue(_ > 0, "Each executor must contain at least 1 cpu core.") .createWithDefault(1) private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) @@ -334,11 +333,7 @@ package object config { private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) - private[spark] val CPUS_PER_TASK = - ConfigBuilder("spark.task.cpus") - .intConf - .checkValue(_ > 0, "Each task must require at least 1 cpu core.") - .createWithDefault(1) + private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) private[spark] val DYN_ALLOCATION_ENABLED = ConfigBuilder("spark.dynamicAllocation.enabled").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala similarity index 50% rename from core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala rename to core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index fed97294710a9..ee1e1bcc09b54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -19,30 +19,35 @@ package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer -import org.apache.spark.annotation.Evolving - /** - * Class to hold information about a type of Resource used by the scheduler. This is a separate - * class from the ResourceInformation because here its mutable because the scheduler has to update - * the addresses based on what its assigned and what is available. + * Class to hold information about a type of Resource on an Executor. This information is managed + * by SchedulerBackend, and TaskScheduler shall schedule tasks on idle Executors based on the + * information. */ -@Evolving -private[spark] class SchedulerResourceInformation( +private[spark] class ExecutorResourceInfo( private val name: String, - private val availableAddresses: ArrayBuffer[String] = ArrayBuffer.empty) extends Serializable { + private val addresses: Seq[String] = Seq.empty) extends Serializable { + + // Addresses of resources that has not been assigned or reserved. + // Exposed for testing only. + private[scheduler] val idleAddresses: ArrayBuffer[String] = addresses.to[ArrayBuffer] + // Addresses of resources that has been assigned to running tasks. private val allocatedAddresses: ArrayBuffer[String] = ArrayBuffer.empty + // Addresses of resources that has been reserved but not assigned out yet. + private val reservedAddresses: ArrayBuffer[String] = ArrayBuffer.empty + def getName(): String = name - def getAvailableAddresses(): ArrayBuffer[String] = availableAddresses + def getNumOfIdleResources(): Int = idleAddresses.size def acquireAddresses(num: Int): Seq[String] = { - assert(num <= availableAddresses.size, s"Required to take $num $name addresses but only " + - s"${availableAddresses.size} available.") - val addrs = availableAddresses.take(num) - availableAddresses --= addrs - allocatedAddresses ++= addrs + assert(num <= idleAddresses.size, s"Required to take $num $name addresses but only " + + s"${idleAddresses.size} available.") + val addrs = idleAddresses.take(num) + idleAddresses --= addrs + reservedAddresses ++= addrs addrs } @@ -50,8 +55,17 @@ private[spark] class SchedulerResourceInformation( addrs.foreach { address => assert(allocatedAddresses.contains(address), s"Try to release $name address $address, but " + "it is not allocated.") - availableAddresses += address + idleAddresses += address allocatedAddresses -= address } } + + def assignAddresses(addrs: Array[String]): Unit = { + addrs.foreach { address => + assert(reservedAddresses.contains(address), s"Try to assign $name address $address, but " + + s"it is not reserved.") + allocatedAddresses += address + reservedAddresses -= address + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index c22e53f7c0156..83b6cb0b4b273 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -93,7 +93,7 @@ private[spark] class TaskSchedulerImpl( val CPUS_PER_TASK = conf.get(config.CPUS_PER_TASK) // Resources to request per task - val RESOURCES_PER_TASK = conf.getTaskResourceRequirements() + val resourcesPerTask = conf.getTaskResourceRequirements() // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. Protected by `this` @@ -330,7 +330,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], - availableResources: Array[Map[String, SchedulerResourceInformation]], + availableResources: Array[Map[String, ExecutorResourceInfo]], tasks: IndexedSeq[ArrayBuffer[TaskDescription]], addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = { var launchedTask = false @@ -373,9 +373,9 @@ private[spark] class TaskSchedulerImpl( * Check whether the resources from the WorkerOffer are enough to run at least one task. */ private def resourcesMeetTaskRequirements( - resources: Map[String, SchedulerResourceInformation]): Boolean = { - RESOURCES_PER_TASK.forall { case (rName, rNum) => - resources.contains(rName) && resources(rName).getAvailableAddresses().size >= rNum + resources: Map[String, ExecutorResourceInfo]): Boolean = { + resourcesPerTask.forall { case (rName, rNum) => + resources.contains(rName) && resources(rName).getNumOfIdleResources() >= rNum } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a84e988a51b18..4a5607011fb2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -469,7 +469,7 @@ private[spark] class TaskSetManager( execId: String, host: String, maxLocality: TaskLocality.TaskLocality, - availableResources: Map[String, SchedulerResourceInformation]) + availableResources: Map[String, ExecutorResourceInfo]) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -534,8 +534,8 @@ private[spark] class TaskSetManager( logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") - val extraResources = sched.RESOURCES_PER_TASK.map { case (rName, rNum) => - val allocatedAddresses = availableResources.get(rName).get.acquireAddresses(rNum) + val extraResources = sched.resourcesPerTask.map { case (rName, rNum) => + val allocatedAddresses = availableResources(rName).acquireAddresses(rNum) (rName, new ResourceInformation(rName, allocatedAddresses.toArray)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala index 46e7d88780473..ae64f42e04180 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -28,4 +28,4 @@ case class WorkerOffer( // `address` is an optional hostPort string, it provide more useful information than `host` // when multiple executors are launched on the same host. address: Option[String] = None, - resources: Map[String, SchedulerResourceInformation] = Map.empty) + resources: Map[String, ExecutorResourceInfo] = Map.empty) 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 a3632b923670e..ae02e0e9c044e 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 @@ -217,8 +217,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp totalRegisteredExecutors.addAndGet(1) val data = new ExecutorData(executorRef, executorAddress, hostname, cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, - mutable.Map.empty[String, SchedulerResourceInformation] ++= resources.mapValues(v => - new SchedulerResourceInformation(v.name, v.addresses.to[ArrayBuffer]))) + mutable.Map.empty[String, ExecutorResourceInfo] ++= resources.mapValues(v => + new ExecutorResourceInfo(v.name, v.addresses))) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -334,6 +334,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp else { val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK + task.resources.foreach { case (rName, rInfo) => + assert(executorData.availableResources.contains(rName)) + executorData.availableResources(rName).assignAddresses(rInfo.addresses) + } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + s"${executorData.executorHost}.") @@ -536,9 +540,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // this function is for testing only def getExecutorAvailableResources(executorId: String): - mutable.Map[String, SchedulerResourceInformation] = { + mutable.Map[String, ExecutorResourceInfo] = { executorDataMap.get(executorId).map(_.availableResources). - getOrElse(mutable.Map.empty[String, SchedulerResourceInformation]) + getOrElse(mutable.Map.empty[String, ExecutorResourceInfo]) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 3964d94713730..94904d32c3d0f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.apache.spark.ResourceInformation import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} -import org.apache.spark.scheduler.SchedulerResourceInformation +import org.apache.spark.scheduler.ExecutorResourceInfo /** * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. @@ -43,5 +43,5 @@ private[cluster] class ExecutorData( override val logUrlMap: Map[String, String], override val attributes: Map[String, String], override val totalResources: Map[String, ResourceInformation], - val availableResources: mutable.Map[String, SchedulerResourceInformation] + val availableResources: mutable.Map[String, ExecutorResourceInfo] ) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, totalResources) diff --git a/core/src/test/scala/org/apache/spark/ResourceName.scala b/core/src/test/scala/org/apache/spark/ResourceName.scala new file mode 100644 index 0000000000000..c97eaae00bbd1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ResourceName.scala @@ -0,0 +1,23 @@ +/* + * 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 + +private[spark] object ResourceName { + // known types of resources + final val GPU: String = "gpu" +} diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 965c7df69cfe3..30fe18b53e8ab 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -446,6 +446,29 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(thrown.getMessage.contains(key)) } } + + test("get task resource requirement from config") { + val conf = new SparkConf() + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_SUFFIX, "2") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_SUFFIX, "1") + var taskResourceRequirement = conf.getTaskResourceRequirements() + assert(taskResourceRequirement.size == 2) + assert(taskResourceRequirement("gpu") == 2) + assert(taskResourceRequirement("fpga") == 1) + + conf.remove(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_SUFFIX) + // Ignore invalid prefix + conf.set("spark.invalid.prefix" + "fpga" + SPARK_RESOURCE_COUNT_SUFFIX, "1") + taskResourceRequirement = conf.getTaskResourceRequirements() + assert(taskResourceRequirement.size == 1) + assert(taskResourceRequirement.get("fpga").isEmpty) + + // Ignore invalid suffix + conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + "invalid.suffix", "1") + taskResourceRequirement = conf.getTaskResourceRequirements() + assert(taskResourceRequirement.size == 1) + assert(taskResourceRequirement.get("fpga").isEmpty) + } } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 1da07653954e0..a3a2ceeb48ee6 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -39,6 +39,7 @@ import org.json4s.jackson.JsonMethods.{compact, render} import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually +import org.apache.spark.ResourceName.GPU import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} @@ -855,8 +856,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val discoveryScript = resourceFile.getPath() val conf = new SparkConf() - .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_COUNT_SUFFIX}", "3") - .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}gpu${SPARK_RESOURCE_DISCOVERY_SCRIPT_SUFFIX}", + .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}${GPU}${SPARK_RESOURCE_COUNT_SUFFIX}", "3") + .set(s"${SPARK_EXECUTOR_RESOURCE_PREFIX}${GPU}${SPARK_RESOURCE_DISCOVERY_SCRIPT_SUFFIX}", discoveryScript) .setMaster("local-cluster[3, 3, 1024]") .setAppName("test-cluster") @@ -870,7 +871,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => val context = TaskContext.get() - context.resources().get(ResourceInformation.GPU).get.addresses.iterator + context.resources().get(GPU).get.addresses.iterator } val gpus = rdd.collect() assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 63daeaa8e38c6..f0a6863f09825 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar._ import org.apache.spark._ -import org.apache.spark.ResourceInformation.GPU +import org.apache.spark.ResourceName.GPU import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala index 0677205e42895..233bc73aa7419 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -24,7 +24,7 @@ import java.util.Properties import scala.collection.mutable.HashMap import org.apache.spark.ResourceInformation -import org.apache.spark.ResourceInformation.GPU +import org.apache.spark.ResourceName.GPU import org.apache.spark.SparkFunSuite class TaskDescriptionSuite extends SparkFunSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index a9e3732bc286e..2db261f108676 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -29,7 +29,7 @@ import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar import org.apache.spark._ -import org.apache.spark.ResourceInformation.GPU +import org.apache.spark.ResourceName.GPU import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.util.ManualClock @@ -1246,16 +1246,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val executorGpus = 4 val executorCpus = 4 val taskScheduler = setupScheduler(config.CPUS_PER_TASK.key -> taskCpus.toString, - s"${config.SPARK_TASK_RESOURCE_PREFIX}$GPU${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> + s"${config.SPARK_TASK_RESOURCE_PREFIX}${GPU}${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> taskGpus.toString, - s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}$GPU${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> + s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}${GPU}${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> executorGpus.toString, config.EXECUTOR_CORES.key -> executorCpus.toString) val taskSet = FakeTask.createTaskSet(3) val numFreeCores = 2 val resources = Map(GPU -> - new SchedulerResourceInformation(GPU, ArrayBuffer("0", "1", "2", "3"))) + new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"))) val singleCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, resources)) val zeroGpuWorkerOffers = diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 8d7e9352ffd40..a22e4b87b587d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -27,7 +27,7 @@ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.apache.spark._ -import org.apache.spark.ResourceInformation.GPU +import org.apache.spark.ResourceName.GPU import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.serializer.SerializerInstance @@ -1646,13 +1646,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) val availableResources = Map(GPU -> - new SchedulerResourceInformation(GPU, ArrayBuffer("0", "1", "2", "3"))) + new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"))) val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, availableResources) assert(taskOption.isDefined) val allocatedResources = taskOption.get.resources assert(allocatedResources.size == 1) assert(allocatedResources(GPU).addresses sameElements Array("0", "1")) // Allocated resource addresses should no longer be available in `availableResources`. - assert(availableResources(GPU).getAvailableAddresses() sameElements Array("2", "3")) + assert(availableResources(GPU).idleAddresses sameElements Array("2", "3")) } } From 41e94402f4181d9c6fb4acbdeb405dafff26ec50 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 28 May 2019 18:10:25 -0700 Subject: [PATCH 08/18] add ExecutorResourceInfoSuite --- .../scheduler/ExecutorResourceInfo.scala | 18 ++-- .../scheduler/ExecutorResourceInfoSuite.scala | 92 +++++++++++++++++++ 2 files changed, 102 insertions(+), 8 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index ee1e1bcc09b54..fde3f925abe98 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -33,18 +33,20 @@ private[spark] class ExecutorResourceInfo( private[scheduler] val idleAddresses: ArrayBuffer[String] = addresses.to[ArrayBuffer] // Addresses of resources that has been assigned to running tasks. - private val allocatedAddresses: ArrayBuffer[String] = ArrayBuffer.empty + // Exposed for testing only. + private[scheduler] val allocatedAddresses: ArrayBuffer[String] = ArrayBuffer.empty // Addresses of resources that has been reserved but not assigned out yet. - private val reservedAddresses: ArrayBuffer[String] = ArrayBuffer.empty + // Exposed for testing only. + private[scheduler] val reservedAddresses: ArrayBuffer[String] = ArrayBuffer.empty def getName(): String = name def getNumOfIdleResources(): Int = idleAddresses.size def acquireAddresses(num: Int): Seq[String] = { - assert(num <= idleAddresses.size, s"Required to take $num $name addresses but only " + - s"${idleAddresses.size} available.") + assert(num <= idleAddresses.size, "Required to take more addresses than available. " + + s"Required $num $name addresses, but only ${idleAddresses.size} available.") val addrs = idleAddresses.take(num) idleAddresses --= addrs reservedAddresses ++= addrs @@ -53,8 +55,8 @@ private[spark] class ExecutorResourceInfo( def releaseAddresses(addrs: Array[String]): Unit = { addrs.foreach { address => - assert(allocatedAddresses.contains(address), s"Try to release $name address $address, but " + - "it is not allocated.") + assert(allocatedAddresses.contains(address), "Try to release address that is not " + + s"allocated. $name address $address is not allocated.") idleAddresses += address allocatedAddresses -= address } @@ -62,8 +64,8 @@ private[spark] class ExecutorResourceInfo( def assignAddresses(addrs: Array[String]): Unit = { addrs.foreach { address => - assert(reservedAddresses.contains(address), s"Try to assign $name address $address, but " + - s"it is not reserved.") + assert(reservedAddresses.contains(address), "Try to assign address that is not reserved. " + + s"$name address $address is not reserved.") allocatedAddresses += address reservedAddresses -= address } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala new file mode 100644 index 0000000000000..a258dfe393d85 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -0,0 +1,92 @@ +/* + * 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 + +import org.apache.spark.ResourceName.GPU +import org.apache.spark.SparkFunSuite + +class ExecutorResourceInfoSuite extends SparkFunSuite { + + test("Track Executor Resource information") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) + assert(info.idleAddresses sameElements Seq("0", "1", "2", "3")) + assert(info.allocatedAddresses.isEmpty) + assert(info.reservedAddresses.isEmpty) + assert(info.getNumOfIdleResources() == 4) + + // Acquire addresses + info.acquireAddresses(2) + assert(info.idleAddresses sameElements Seq("2", "3")) + assert(info.allocatedAddresses.isEmpty) + assert(info.reservedAddresses sameElements Seq("0", "1")) + assert(info.getNumOfIdleResources() == 2) + + // Assign addresses + info.assignAddresses(Array("0", "1")) + assert(info.idleAddresses sameElements Seq("2", "3")) + assert(info.allocatedAddresses sameElements Seq("0", "1")) + assert(info.reservedAddresses.isEmpty) + assert(info.getNumOfIdleResources() == 2) + + // release addresses + info.releaseAddresses(Array("0", "1")) + assert(info.idleAddresses sameElements Seq("2", "3", "0", "1")) + assert(info.allocatedAddresses.isEmpty) + assert(info.reservedAddresses.isEmpty) + assert(info.getNumOfIdleResources() == 4) + } + + test("Don't allow acquire more addresses than available") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) + val e = intercept[AssertionError] { + info.acquireAddresses(5) + } + assert(e.getMessage.contains("Required to take more addresses than available.")) + } + + test("Don't allow assign address that is not reserved") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) + // Acquire addresses. + info.acquireAddresses(2) + assert(!info.reservedAddresses.contains("2")) + // Assign an address that is not reserved + val e = intercept[AssertionError] { + info.assignAddresses(Array("2")) + } + assert(e.getMessage.contains("Try to assign address that is not reserved.")) + } + + test("Don't allow release address that is not allocated") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) + // Acquire addresses. + info.acquireAddresses(2) + assert(info.reservedAddresses sameElements Seq("0", "1")) + // Assign addresses + info.assignAddresses(Array("0", "1")) + assert(!info.allocatedAddresses.contains("2")) + // Release an address that is not allocated + val e = intercept[AssertionError] { + info.releaseAddresses(Array("2")) + } + assert(e.getMessage.contains("Try to release address that is not allocated.")) + } +} From d3f4a03d202b8e40b337331df2661c9ea6cd47a1 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 29 May 2019 15:40:02 -0700 Subject: [PATCH 09/18] add test for CoarseGrainedExecutorBackend and CoarseGrainedSchedulerBackend --- .../CoarseGrainedExecutorBackend.scala | 3 +- .../CoarseGrainedExecutorBackendSuite.scala | 63 ++++++++++++- .../CoarseGrainedSchedulerBackendSuite.scala | 89 ++++++++++++++++++- 3 files changed, 149 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 76295b984820c..8cf2760de259c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -68,7 +68,8 @@ private[spark] class CoarseGrainedExecutorBackend( // Map each taskId to the resource informations allocated to it, the resource information // includes resource name and resource addresses. - private[this] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] + // Exposed for testing only. + private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] override def onStart() { logInfo("Connecting to driver: " + driverUrl) diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index b3d16d179cb42..c66feeb58fa96 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -20,23 +20,32 @@ package org.apache.spark.executor import java.io.File import java.net.URL +import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.nio.file.{Files => JavaFiles} import java.nio.file.attribute.PosixFilePermission.{OWNER_EXECUTE, OWNER_READ, OWNER_WRITE} -import java.util.EnumSet +import java.util.{EnumSet, Properties} + +import scala.collection.mutable +import scala.concurrent.duration._ import com.google.common.io.Files import org.json4s.JsonAST.{JArray, JObject} import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, render} import org.mockito.Mockito.when +import org.scalatest.concurrent.Eventually.{eventually, timeout} import org.scalatest.mockito.MockitoSugar import org.apache.spark._ +import org.apache.spark.ResourceInformation +import org.apache.spark.ResourceName.GPU import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEnv +import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.LaunchTask import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableBuffer, Utils} class CoarseGrainedExecutorBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { @@ -224,13 +233,59 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } - private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { + test("track allocated resources by taskId") { + val conf = new SparkConf + val securityMgr = new SecurityManager(conf) + val serializer = new JavaSerializer(conf) + var backend: CoarseGrainedExecutorBackend = null + + try { + val rpcEnv = RpcEnv.create("1", "localhost", 0, conf, securityMgr) + val env = createMockEnv(conf, serializer, Some(rpcEnv)) + backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", + "host1", 4, Seq.empty[URL], env, None) + assert(backend.taskResources.isEmpty) + + val taskId = 1000000 + // We don't really verify the data, just pass it around. + val data = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) + val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", 19, 1, + mutable.Map.empty, mutable.Map.empty, new Properties, + Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) + val serializedTaskDescription = TaskDescription.encode(taskDescription) + backend.executor = mock[Executor] + backend.rpcEnv.setupEndpoint("Executor 1", backend) + + // Launch a new task shall add an entry to `taskResources` map. + backend.self.send(LaunchTask(new SerializableBuffer(serializedTaskDescription))) + eventually(timeout(10.seconds)) { + assert(backend.taskResources.size == 1) + assert(backend.taskResources(taskId)(GPU).addresses sameElements Array("0", "1")) + } + + // Update the status of a running task shall not affect `taskResources` map. + backend.statusUpdate(taskId, TaskState.RUNNING, data) + assert(backend.taskResources.size == 1) + assert(backend.taskResources(taskId)(GPU).addresses sameElements Array("0", "1")) + + // Update the status of a finished task shall remove the entry from `taskResources` map. + backend.statusUpdate(taskId, TaskState.FINISHED, data) + assert(backend.taskResources.isEmpty) + } finally { + if (backend != null) { + backend.rpcEnv.shutdown() + } + } + } + + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer, + rpcEnv: Option[RpcEnv] = None): SparkEnv = { val mockEnv = mock[SparkEnv] val mockRpcEnv = mock[RpcEnv] when(mockEnv.conf).thenReturn(conf) when(mockEnv.serializer).thenReturn(serializer) when(mockEnv.closureSerializer).thenReturn(serializer) - when(mockEnv.rpcEnv).thenReturn(mockRpcEnv) + when(mockEnv.rpcEnv).thenReturn(rpcEnv.getOrElse(mockRpcEnv)) SparkEnv.set(mockEnv) mockEnv } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index f0a6863f09825..fa89c0ddb317c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -27,6 +27,7 @@ import scala.language.postfixOps import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar._ @@ -181,6 +182,92 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(executorAddedCount === 3) } + test("extra resources from executor") { + import TestUtils._ + + val conf = new SparkConf() + .set(SPARK_EXECUTOR_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "3") + .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test + .setMaster( + "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") + .setAppName("test") + setTaskResourceRequirement(conf, GPU, 3) + + sc = new SparkContext(conf) + val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] + val mockEndpointRef = mock[RpcEndpointRef] + val mockAddress = mock[RpcAddress] + + val logUrls = Map( + "stdout" -> "http://oldhost:8888/logs/dummy/stdout", + "stderr" -> "http://oldhost:8888/logs/dummy/stderr") + val attributes = Map( + "CLUSTER_ID" -> "cl1", + "USER" -> "dummy", + "CONTAINER_ID" -> "container1", + "LOG_FILES" -> "stdout,stderr") + val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + + s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" + val resources = Map(GPU -> new ResourceInformation(GPU, Array("0", "1", "3"))) + + var executorAddedCount: Int = 0 + val listener = new SparkListener() { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + executorAddedCount += 1 + assert(executorAdded.executorInfo.totalResources.get(GPU).nonEmpty) + val totalResources = executorAdded.executorInfo.totalResources(GPU) + assert(totalResources.addresses === Array("0", "1", "3")) + assert(totalResources.name == GPU) + } + } + + sc.addSparkListener(listener) + + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + + val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) + val bytebuffer = java.nio.ByteBuffer.allocate(frameSize/2) + val buffer = new SerializableBuffer(bytebuffer) + + var execResources = backend.getExecutorAvailableResources("1") + + assert(execResources(GPU).idleAddresses === Array("0", "1", "3")) + + var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", + "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], + new Properties(), immutable.Map(GPU -> new ResourceInformation(GPU, Array("0"))), + bytebuffer))) + val ts = backend.getTaskSchedulerImpl() + // resource offer such that gpu address 0 gets removed + when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { + backend.getExecutorAvailableResources("1")(GPU).acquireAddresses(1) + taskDescs + }) + + backend.driverEndpoint.send(ReviveOffers) + + eventually(timeout(5 seconds)) { + execResources = backend.getExecutorAvailableResources("1") + assert(execResources(GPU).idleAddresses === Array("1", "3")) + } + + var finishedTaskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) + backend.driverEndpoint.send( + StatusUpdate("1", 1, TaskState.FINISHED, buffer, finishedTaskResources)) + + eventually(timeout(5 seconds)) { + execResources = backend.getExecutorAvailableResources("1") + assert(execResources(GPU).idleAddresses === Array("1", "3", "0")) + } + sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) + assert(executorAddedCount === 3) + } + private def testSubmitJob(sc: SparkContext, rdd: RDD[Int]): Unit = { sc.submitJob( rdd, @@ -192,7 +279,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo } } -/** Simple cluster manager that wires up our mock backend for the gpu resource tests. */ +/** Simple cluster manager that wires up our mock backend for the resource tests. */ private class CSMockExternalClusterManager extends ExternalClusterManager { private var ts: TaskSchedulerImpl = _ From 26126fc69339899a4ca291ea6e6f50245f29cb50 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 29 May 2019 17:17:40 -0700 Subject: [PATCH 10/18] update failed test cases --- .../scheduler/ExecutorResourceInfo.scala | 19 +++- .../CoarseGrainedSchedulerBackend.scala | 13 ++- .../CoarseGrainedSchedulerBackendSuite.scala | 106 ++++++++++++++---- .../scheduler/ExecutorResourceInfoSuite.scala | 4 +- 4 files changed, 111 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index fde3f925abe98..62a43e7542e3e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer */ private[spark] class ExecutorResourceInfo( private val name: String, - private val addresses: Seq[String] = Seq.empty) extends Serializable { + private val addresses: Seq[String]) extends Serializable { // Addresses of resources that has not been assigned or reserved. // Exposed for testing only. @@ -44,6 +44,8 @@ private[spark] class ExecutorResourceInfo( def getNumOfIdleResources(): Int = idleAddresses.size + // Reserve given number of resource addresses, these addresses can be assigned to a future + // launched task. def acquireAddresses(num: Int): Seq[String] = { assert(num <= idleAddresses.size, "Required to take more addresses than available. " + s"Required $num $name addresses, but only ${idleAddresses.size} available.") @@ -53,15 +55,24 @@ private[spark] class ExecutorResourceInfo( addrs } + // Give back a sequence of resource addresses, these addresses must have been reserved or + // assigned. Resource addresses are released when a task has finished, or the task launch is + // skipped. def releaseAddresses(addrs: Array[String]): Unit = { addrs.foreach { address => - assert(allocatedAddresses.contains(address), "Try to release address that is not " + - s"allocated. $name address $address is not allocated.") + assert((allocatedAddresses ++ reservedAddresses).contains(address), "Try to release " + + s"address that is not reserved or allocated. $name address $address is not allocated.") idleAddresses += address - allocatedAddresses -= address + if (allocatedAddresses.contains(address)) { + allocatedAddresses -= address + } else if (reservedAddresses.contains(address)) { + reservedAddresses -= address + } } } + // Assign a sequence of resource addresses (to a launched task), these addresses must have been + // reserved. def assignAddresses(addrs: Array[String]): Unit = { addrs.foreach { address => assert(reservedAddresses.contains(address), "Try to assign address that is not reserved. " + 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 ae02e0e9c044e..df7c10f907841 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 @@ -317,8 +317,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Launch tasks returned by a set of resource offers private def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { + val executorData = executorDataMap(task.executorId) val serializedTask = TaskDescription.encode(task) if (serializedTask.limit() >= maxRpcMessageSize) { + // We skip launch the task, should release all the reserved resources. + task.resources.foreach { case (rName, rInfo) => + assert(executorData.availableResources.contains(rName)) + executorData.availableResources(rName).releaseAddresses(rInfo.addresses) + } Option(scheduler.taskIdToTaskSetManager.get(task.taskId)).foreach { taskSetMgr => try { var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " + @@ -332,11 +338,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } else { - val executorData = executorDataMap(task.executorId) + // Do resources allocation here. The allocated resources will get released after the task + // finishes. executorData.freeCores -= scheduler.CPUS_PER_TASK task.resources.foreach { case (rName, rInfo) => - assert(executorData.availableResources.contains(rName)) - executorData.availableResources(rName).assignAddresses(rInfo.addresses) + assert(executorData.availableResources.contains(rName)) + executorData.availableResources(rName).assignAddresses(rInfo.addresses) } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index fa89c0ddb317c..399ddde9d85b1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -191,23 +191,14 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo .setMaster( "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") .setAppName("test") - setTaskResourceRequirement(conf, GPU, 3) + setTaskResourceRequirement(conf, GPU, 1) sc = new SparkContext(conf) val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] + when(mockEndpointRef.send(LaunchTask)).thenAnswer((_: InvocationOnMock) => {}) - val logUrls = Map( - "stdout" -> "http://oldhost:8888/logs/dummy/stdout", - "stderr" -> "http://oldhost:8888/logs/dummy/stderr") - val attributes = Map( - "CLUSTER_ID" -> "cl1", - "USER" -> "dummy", - "CONTAINER_ID" -> "container1", - "LOG_FILES" -> "stdout,stderr") - val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + - s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" val resources = Map(GPU -> new ResourceInformation(GPU, Array("0", "1", "3"))) var executorAddedCount: Int = 0 @@ -224,29 +215,32 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo sc.addSparkListener(listener) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, resources)) + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) - val bytebuffer = java.nio.ByteBuffer.allocate(frameSize/2) + val bytebuffer = java.nio.ByteBuffer.allocate(frameSize - 100) val buffer = new SerializableBuffer(bytebuffer) var execResources = backend.getExecutorAvailableResources("1") assert(execResources(GPU).idleAddresses === Array("0", "1", "3")) + val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], - new Properties(), immutable.Map(GPU -> new ResourceInformation(GPU, Array("0"))), - bytebuffer))) + new Properties(), taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() - // resource offer such that gpu address 0 gets removed + var numPendingTasks = 1 when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { - backend.getExecutorAvailableResources("1")(GPU).acquireAddresses(1) - taskDescs + if (numPendingTasks > 0) { + numPendingTasks -= 1 + backend.getExecutorAvailableResources("1")(GPU).acquireAddresses(1) + taskDescs + } }) backend.driverEndpoint.send(ReviveOffers) @@ -254,11 +248,12 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") assert(execResources(GPU).idleAddresses === Array("1", "3")) + assert(execResources(GPU).allocatedAddresses === Array("0")) + assert(execResources(GPU).reservedAddresses.isEmpty) } - var finishedTaskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, finishedTaskResources)) + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskResources)) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") @@ -268,6 +263,73 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(executorAddedCount === 3) } + test("release resources when launch task fails") { + import TestUtils._ + + val conf = new SparkConf() + .set(SPARK_EXECUTOR_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "3") + .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test + .setMaster( + "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") + .setAppName("test") + setTaskResourceRequirement(conf, GPU, 1) + + sc = new SparkContext(conf) + val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] + val mockEndpointRef = mock[RpcEndpointRef] + val mockAddress = mock[RpcAddress] + val resources = Map(GPU -> new ResourceInformation(GPU, Array("0", "1", "3"))) + + var executorAddedCount: Int = 0 + val listener = new SparkListener() { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + executorAddedCount += 1 + assert(executorAdded.executorInfo.totalResources.get(GPU).nonEmpty) + val totalResources = executorAdded.executorInfo.totalResources(GPU) + assert(totalResources.addresses === Array("0", "1", "3")) + assert(totalResources.name == GPU) + } + } + + sc.addSparkListener(listener) + + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + + val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) + val bytebuffer = java.nio.ByteBuffer.allocate(frameSize * 2) + val buffer = new SerializableBuffer(bytebuffer) + + var execResources = backend.getExecutorAvailableResources("1") + + assert(execResources(GPU).idleAddresses === Array("0", "1", "3")) + + val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) + var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", + "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], + new Properties(), taskResources, bytebuffer))) + val ts = backend.getTaskSchedulerImpl() + when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { + backend.getExecutorAvailableResources("1")(GPU).acquireAddresses(1) + taskDescs + }) + + backend.driverEndpoint.send(ReviveOffers) + + // The task fail to launch, should release reserved resources. + eventually(timeout(5 seconds)) { + execResources = backend.getExecutorAvailableResources("1") + assert(execResources(GPU).idleAddresses === Array("1", "3", "0")) + } + + sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) + assert(executorAddedCount === 3) + } + private def testSubmitJob(sc: SparkContext, rdd: RDD[Int]): Unit = { sc.submitJob( rdd, diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala index a258dfe393d85..1bcbd202522b8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -74,7 +74,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { assert(e.getMessage.contains("Try to assign address that is not reserved.")) } - test("Don't allow release address that is not allocated") { + test("Don't allow release address that is not reserved or allocated") { // Init Executor Resource. val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) // Acquire addresses. @@ -87,6 +87,6 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val e = intercept[AssertionError] { info.releaseAddresses(Array("2")) } - assert(e.getMessage.contains("Try to release address that is not allocated.")) + assert(e.getMessage.contains("Try to release address that is not reserved or allocated.")) } } From bbac8932b4d08230f00c1e018f9297cefbd1d766 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 30 May 2019 13:34:39 -0700 Subject: [PATCH 11/18] update config check --- .../scala/org/apache/spark/SparkConf.scala | 3 +- .../scala/org/apache/spark/SparkContext.scala | 56 ++++++++++++------- .../org/apache/spark/SparkContextSuite.scala | 18 +++++- .../CoarseGrainedSchedulerBackendSuite.scala | 6 +- 4 files changed, 57 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index eab89ccf9e372..33d4aeb0863aa 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -615,8 +615,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria // Make sure the executor resources were specified and are large enough if // any task resources were specified. - val taskResourcesAndCount = - getAllWithPrefixAndSuffix(SPARK_TASK_RESOURCE_PREFIX, SPARK_RESOURCE_COUNT_SUFFIX).toMap + val taskResourcesAndCount = getTaskResourceRequirements() val executorResourcesAndCounts = getAllWithPrefixAndSuffix(SPARK_EXECUTOR_RESOURCE_PREFIX, SPARK_RESOURCE_COUNT_SUFFIX).toMap diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6266ce62669e3..c53c815f4d3df 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2707,27 +2707,43 @@ object SparkContext extends Logging { // When running locally, don't try to re-execute tasks on failure. val MAX_LOCAL_TASK_FAILURES = 1 - // SPARK-26340: Ensure that executor's core num meets at least one task requirement. - def checkCpusPerTask( - clusterMode: Boolean, - maxCoresPerExecutor: Option[Int]): Unit = { - val cpusPerTask = sc.conf.get(CPUS_PER_TASK) - if (clusterMode && sc.conf.contains(EXECUTOR_CORES)) { - if (sc.conf.get(EXECUTOR_CORES) < cpusPerTask) { - throw new SparkException(s"${CPUS_PER_TASK.key}" + - s" must be <= ${EXECUTOR_CORES.key} when run on $master.") - } - } else if (maxCoresPerExecutor.isDefined) { - if (maxCoresPerExecutor.get < cpusPerTask) { - throw new SparkException(s"Only ${maxCoresPerExecutor.get} cores available per executor" + - s" when run on $master, and ${CPUS_PER_TASK.key} must be <= it.") + // Ensure that executor's resources satisfies one or more tasks requirement. + def checkResourcesPerTask(clusterMode: Boolean, executorCores: Option[Int]): Unit = { + val taskCores = sc.conf.get(CPUS_PER_TASK) + val execCores = if (clusterMode) { + executorCores.getOrElse(sc.conf.get(EXECUTOR_CORES)) + } else { + executorCores.get + } + + // Number of cores per executor must meet at least one task requirement. + if (execCores < taskCores) { + throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + + s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.") + } + + // Resources per executor must be the same multiple numbers of the task requirements. + val numSlots = execCores / taskCores + val taskResourcesAndCount = sc.conf.getTaskResourceRequirements() + val executorResourcesAndCounts = sc.conf.getAllWithPrefixAndSuffix( + SPARK_EXECUTOR_RESOURCE_PREFIX, SPARK_RESOURCE_COUNT_SUFFIX).toMap + // There have been checks inside SparkConf to make sure the executor resources were specified + // and are large enough if any task resources were specified. + taskResourcesAndCount.foreach { case (rName, taskCount) => + val execCount = executorResourcesAndCounts(rName) + if (execCount.toInt / taskCount.toInt != numSlots) { + throw new SparkException( + s"The value of executor resource config: " + + s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + + s"= $execCount has to be $numSlots times the number of the task config: " + + s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount") } } } master match { case "local" => - checkCpusPerTask(clusterMode = false, Some(1)) + checkResourcesPerTask(clusterMode = false, Some(1)) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) @@ -2740,7 +2756,7 @@ object SparkContext extends Logging { if (threadCount <= 0) { throw new SparkException(s"Asked to run locally with $threadCount threads") } - checkCpusPerTask(clusterMode = false, Some(threadCount)) + checkResourcesPerTask(clusterMode = false, Some(threadCount)) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) @@ -2751,14 +2767,14 @@ object SparkContext extends Logging { // local[*, M] means the number of cores on the computer with M failures // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt - checkCpusPerTask(clusterMode = false, Some(threadCount)) + checkResourcesPerTask(clusterMode = false, Some(threadCount)) val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case SPARK_REGEX(sparkUrl) => - checkCpusPerTask(clusterMode = true, None) + checkResourcesPerTask(clusterMode = true, None) val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) @@ -2766,7 +2782,7 @@ object SparkContext extends Logging { (backend, scheduler) case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - checkCpusPerTask(clusterMode = true, Some(coresPerSlave.toInt)) + checkResourcesPerTask(clusterMode = true, Some(coresPerSlave.toInt)) // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt if (sc.executorMemory > memoryPerSlaveInt) { @@ -2787,7 +2803,7 @@ object SparkContext extends Logging { (backend, scheduler) case masterUrl => - checkCpusPerTask(clusterMode = true, None) + checkResourcesPerTask(clusterMode = true, None) val cm = getClusterManager(masterUrl) match { case Some(clusterMgr) => clusterMgr case None => throw new SparkException("Could not parse Master URL: '" + master + "'") diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a3a2ceeb48ee6..20634969f2483 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -719,7 +719,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test(s"Avoid setting ${CPUS_PER_TASK.key} unreasonably (SPARK-27192)") { - val FAIL_REASON = s"${CPUS_PER_TASK.key} must be <=" + val FAIL_REASON = s"has to be >= the task config: ${CPUS_PER_TASK.key}" Seq( ("local", 2, None), ("local[2]", 3, None), @@ -837,6 +837,22 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu "spark.task.resource.gpu.count = 2")) } + test("Parse resources executor config not the same multiple numbers of the task requirements") { + val conf = new SparkConf() + .set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_SUFFIX, "2") + .set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_SUFFIX, "4") + .setMaster("local-cluster[1, 1, 1024]") + .setAppName("test-cluster") + + var error = intercept[SparkException] { + sc = new SparkContext(conf) + }.getMessage() + + assert(error.contains("The value of executor resource config: " + + "spark.executor.resource.gpu.count = 4 has to be 1 times the number of the task config: " + + "spark.task.resource.gpu.count = 2")) + } + def mockDiscoveryScript(file: File, result: String): String = { Files.write(s"echo $result", file, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file.toPath(), diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 399ddde9d85b1..7d4139d127523 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -238,7 +238,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { if (numPendingTasks > 0) { numPendingTasks -= 1 - backend.getExecutorAvailableResources("1")(GPU).acquireAddresses(1) + backend.getExecutorAvailableResources("1").toMap.get(GPU).get.acquireAddresses(1) taskDescs } }) @@ -314,8 +314,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo new Properties(), taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { - backend.getExecutorAvailableResources("1")(GPU).acquireAddresses(1) - taskDescs + backend.getExecutorAvailableResources("1").toMap.get(GPU).get.acquireAddresses(1) + taskDescs }) backend.driverEndpoint.send(ReviveOffers) From 10d5fab26d376c15d8a07007e5839dc2bc8a7ca7 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 30 May 2019 18:20:47 -0700 Subject: [PATCH 12/18] Only modify ExecutorResourceInfo inside SchedulerBackend. --- .../scheduler/ExecutorResourceInfo.scala | 40 ++-- .../InternalExecutorResourcesInfo.scala | 39 ++++ .../spark/scheduler/TaskSchedulerImpl.scala | 7 +- .../spark/scheduler/TaskSetManager.scala | 5 +- .../apache/spark/scheduler/WorkerOffer.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 15 +- .../local/LocalSchedulerBackend.scala | 2 +- .../CoarseGrainedSchedulerBackendSuite.scala | 77 +------ .../scheduler/ExecutorResourceInfoSuite.scala | 42 ++-- .../scheduler/TaskSchedulerImplSuite.scala | 14 +- .../spark/scheduler/TaskSetManagerSuite.scala | 214 +++++++++--------- 11 files changed, 213 insertions(+), 244 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index 62a43e7542e3e..193aace0166d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -26,22 +26,19 @@ import scala.collection.mutable.ArrayBuffer */ private[spark] class ExecutorResourceInfo( private val name: String, - private val addresses: Seq[String]) extends Serializable { + private[scheduler] val addresses: ArrayBuffer[String]) extends Serializable { - // Addresses of resources that has not been assigned or reserved. + // Addresses of resource that has not been acquired. // Exposed for testing only. - private[scheduler] val idleAddresses: ArrayBuffer[String] = addresses.to[ArrayBuffer] + private[scheduler] val idleAddresses: ArrayBuffer[String] = addresses.clone() - // Addresses of resources that has been assigned to running tasks. + // Addresses of resource that has been assigned to running tasks. // Exposed for testing only. private[scheduler] val allocatedAddresses: ArrayBuffer[String] = ArrayBuffer.empty - // Addresses of resources that has been reserved but not assigned out yet. - // Exposed for testing only. - private[scheduler] val reservedAddresses: ArrayBuffer[String] = ArrayBuffer.empty - def getName(): String = name + // Number of resource addresses that can be acquired. def getNumOfIdleResources(): Int = idleAddresses.size // Reserve given number of resource addresses, these addresses can be assigned to a future @@ -51,34 +48,29 @@ private[spark] class ExecutorResourceInfo( s"Required $num $name addresses, but only ${idleAddresses.size} available.") val addrs = idleAddresses.take(num) idleAddresses --= addrs - reservedAddresses ++= addrs addrs } - // Give back a sequence of resource addresses, these addresses must have been reserved or - // assigned. Resource addresses are released when a task has finished, or the task launch is - // skipped. + // Give back a sequence of resource addresses, these addresses must have been assigned. Resource + // addresses are released when a task has finished. def releaseAddresses(addrs: Array[String]): Unit = { addrs.foreach { address => - assert((allocatedAddresses ++ reservedAddresses).contains(address), "Try to release " + - s"address that is not reserved or allocated. $name address $address is not allocated.") + assert(allocatedAddresses.contains(address), "Try to release address that is not " + + s"allocated. $name address $address is not allocated.") + addresses += address idleAddresses += address - if (allocatedAddresses.contains(address)) { - allocatedAddresses -= address - } else if (reservedAddresses.contains(address)) { - reservedAddresses -= address - } + allocatedAddresses -= address } } - // Assign a sequence of resource addresses (to a launched task), these addresses must have been - // reserved. + // Assign a sequence of resource addresses (to a launched task), these addresses must be + // available. def assignAddresses(addrs: Array[String]): Unit = { addrs.foreach { address => - assert(reservedAddresses.contains(address), "Try to assign address that is not reserved. " + - s"$name address $address is not reserved.") + assert(addresses.contains(address), "Try to assign address that is not available. " + + s"$name address $address is not available.") allocatedAddresses += address - reservedAddresses -= address + addresses -= address } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala new file mode 100644 index 0000000000000..192eda0a11eb8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala @@ -0,0 +1,39 @@ +/* + * 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 + + +/** + * Class to hold information about resources on an Executor. + */ +private[scheduler] class InternalExecutorResourcesInfo( + private val resources: Map[String, ExecutorResourceInfo]) extends Serializable { + + def getNumOfIdleResources(resourceName: String): Int = + resources.get(resourceName).map(_.getNumOfIdleResources()).getOrElse(0) + + // Reserve given number of resource addresses, these addresses can be assigned to a future + // launched task. + def acquireAddresses(resourceName: String, num: Int): Seq[String] = { + resources.get(resourceName).map(_.acquireAddresses(num)).getOrElse(Seq.empty) + } +} + +private[scheduler] object InternalExecutorResourcesInfo { + final val EMPTY_RESOURCES_INFO = new InternalExecutorResourcesInfo(Map.empty) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 83b6cb0b4b273..8b886c293c49c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -330,7 +330,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], - availableResources: Array[Map[String, ExecutorResourceInfo]], + availableResources: Array[InternalExecutorResourcesInfo], tasks: IndexedSeq[ArrayBuffer[TaskDescription]], addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = { var launchedTask = false @@ -372,10 +372,9 @@ private[spark] class TaskSchedulerImpl( /** * Check whether the resources from the WorkerOffer are enough to run at least one task. */ - private def resourcesMeetTaskRequirements( - resources: Map[String, ExecutorResourceInfo]): Boolean = { + private def resourcesMeetTaskRequirements(resources: InternalExecutorResourcesInfo): Boolean = { resourcesPerTask.forall { case (rName, rNum) => - resources.contains(rName) && resources(rName).getNumOfIdleResources() >= rNum + resources.getNumOfIdleResources(rName) >= rNum } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 4a5607011fb2d..4a43a91af9e17 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -30,6 +30,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.util.{AccumulatorV2, Clock, LongAccumulator, SystemClock, Utils} import org.apache.spark.util.collection.MedianHeap @@ -469,7 +470,7 @@ private[spark] class TaskSetManager( execId: String, host: String, maxLocality: TaskLocality.TaskLocality, - availableResources: Map[String, ExecutorResourceInfo]) + availableResources: InternalExecutorResourcesInfo = EMPTY_RESOURCES_INFO) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -535,7 +536,7 @@ private[spark] class TaskSetManager( s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") val extraResources = sched.resourcesPerTask.map { case (rName, rNum) => - val allocatedAddresses = availableResources(rName).acquireAddresses(rNum) + val allocatedAddresses = availableResources.acquireAddresses(rName, rNum) (rName, new ResourceInformation(rName, allocatedAddresses.toArray)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala index ae64f42e04180..3d39b9e9ea458 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -28,4 +28,4 @@ case class WorkerOffer( // `address` is an optional hostPort string, it provide more useful information than `host` // when multiple executors are launched on the same host. address: Option[String] = None, - resources: Map[String, ExecutorResourceInfo] = Map.empty) + resources: InternalExecutorResourcesInfo = InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO) 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 df7c10f907841..9cc5130f1c3ce 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 @@ -218,7 +218,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val data = new ExecutorData(executorRef, executorAddress, hostname, cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, mutable.Map.empty[String, ExecutorResourceInfo] ++= resources.mapValues(v => - new ExecutorResourceInfo(v.name, v.addresses))) + new ExecutorResourceInfo(v.name, v.addresses.to[ArrayBuffer]))) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -272,7 +272,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val workOffers = activeExecutors.map { case (id, executorData) => new WorkerOffer(id, executorData.executorHost, executorData.freeCores, - Some(executorData.executorAddress.hostPort), executorData.availableResources.toMap) + Some(executorData.executorAddress.hostPort), + new InternalExecutorResourcesInfo(executorData.availableResources.toMap)) }.toIndexedSeq scheduler.resourceOffers(workOffers) } @@ -298,7 +299,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val executorData = executorDataMap(executorId) val workOffers = IndexedSeq( new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores, - Some(executorData.executorAddress.hostPort), executorData.availableResources.toMap)) + Some(executorData.executorAddress.hostPort), + new InternalExecutorResourcesInfo(executorData.availableResources.toMap))) scheduler.resourceOffers(workOffers) } else { Seq.empty @@ -317,14 +319,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Launch tasks returned by a set of resource offers private def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - val executorData = executorDataMap(task.executorId) val serializedTask = TaskDescription.encode(task) if (serializedTask.limit() >= maxRpcMessageSize) { - // We skip launch the task, should release all the reserved resources. - task.resources.foreach { case (rName, rInfo) => - assert(executorData.availableResources.contains(rName)) - executorData.availableResources(rName).releaseAddresses(rInfo.addresses) - } Option(scheduler.taskIdToTaskSetManager.get(task.taskId)).foreach { taskSetMgr => try { var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " + @@ -338,6 +334,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } else { + val executorData = executorDataMap(task.executorId) // Do resources allocation here. The allocated resources will get released after the task // finishes. executorData.freeCores -= scheduler.CPUS_PER_TASK diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index fae322a7f2be2..8112d3a3a62cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -83,7 +83,7 @@ private[spark] class LocalEndpoint( def reviveOffers() { // local mode doesn't support extra resources like GPUs right now val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores, - Some(rpcEnv.address.hostPort), Map.empty)) + Some(rpcEnv.address.hostPort), InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO)) for (task <- scheduler.resourceOffers(offers).flatten) { freeCores -= scheduler.CPUS_PER_TASK executor.launchTask(executorBackend, task) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 7d4139d127523..5febff6479f9e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -186,6 +186,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo import TestUtils._ val conf = new SparkConf() + .set(EXECUTOR_CORES, 3) .set(SPARK_EXECUTOR_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "3") .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test .setMaster( @@ -238,7 +239,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { if (numPendingTasks > 0) { numPendingTasks -= 1 - backend.getExecutorAvailableResources("1").toMap.get(GPU).get.acquireAddresses(1) + new InternalExecutorResourcesInfo(backend.getExecutorAvailableResources("1").toMap) + .acquireAddresses(GPU, 1) taskDescs } }) @@ -247,9 +249,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).idleAddresses === Array("1", "3")) + assert(execResources(GPU).addresses === Array("1", "3")) assert(execResources(GPU).allocatedAddresses === Array("0")) - assert(execResources(GPU).reservedAddresses.isEmpty) } backend.driverEndpoint.send( @@ -257,79 +258,13 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).idleAddresses === Array("1", "3", "0")) + assert(execResources(GPU).addresses === Array("1", "3", "0")) + assert(execResources(GPU).allocatedAddresses.isEmpty) } sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) assert(executorAddedCount === 3) } - test("release resources when launch task fails") { - import TestUtils._ - - val conf = new SparkConf() - .set(SPARK_EXECUTOR_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "3") - .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test - .setMaster( - "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") - .setAppName("test") - setTaskResourceRequirement(conf, GPU, 1) - - sc = new SparkContext(conf) - val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] - val mockEndpointRef = mock[RpcEndpointRef] - val mockAddress = mock[RpcAddress] - val resources = Map(GPU -> new ResourceInformation(GPU, Array("0", "1", "3"))) - - var executorAddedCount: Int = 0 - val listener = new SparkListener() { - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { - executorAddedCount += 1 - assert(executorAdded.executorInfo.totalResources.get(GPU).nonEmpty) - val totalResources = executorAdded.executorInfo.totalResources(GPU) - assert(totalResources.addresses === Array("0", "1", "3")) - assert(totalResources.name == GPU) - } - } - - sc.addSparkListener(listener) - - backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) - backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) - backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) - - val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) - val bytebuffer = java.nio.ByteBuffer.allocate(frameSize * 2) - val buffer = new SerializableBuffer(bytebuffer) - - var execResources = backend.getExecutorAvailableResources("1") - - assert(execResources(GPU).idleAddresses === Array("0", "1", "3")) - - val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) - var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", - "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], - new Properties(), taskResources, bytebuffer))) - val ts = backend.getTaskSchedulerImpl() - when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { - backend.getExecutorAvailableResources("1").toMap.get(GPU).get.acquireAddresses(1) - taskDescs - }) - - backend.driverEndpoint.send(ReviveOffers) - - // The task fail to launch, should release reserved resources. - eventually(timeout(5 seconds)) { - execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).idleAddresses === Array("1", "3", "0")) - } - - sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) - assert(executorAddedCount === 3) - } - private def testSubmitJob(sc: SparkContext, rdd: RDD[Int]): Unit = { sc.submitJob( rdd, diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala index 1bcbd202522b8..7b0be813026b0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -17,69 +17,69 @@ package org.apache.spark.scheduler +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.ResourceName.GPU import org.apache.spark.SparkFunSuite + class ExecutorResourceInfoSuite extends SparkFunSuite { test("Track Executor Resource information") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + assert(info.addresses sameElements Seq("0", "1", "2", "3")) assert(info.idleAddresses sameElements Seq("0", "1", "2", "3")) assert(info.allocatedAddresses.isEmpty) - assert(info.reservedAddresses.isEmpty) assert(info.getNumOfIdleResources() == 4) // Acquire addresses info.acquireAddresses(2) + assert(info.addresses sameElements Seq("0", "1", "2", "3")) assert(info.idleAddresses sameElements Seq("2", "3")) - assert(info.allocatedAddresses.isEmpty) - assert(info.reservedAddresses sameElements Seq("0", "1")) + assert(info.allocatedAddresses sameElements Seq.empty) assert(info.getNumOfIdleResources() == 2) // Assign addresses info.assignAddresses(Array("0", "1")) + assert(info.addresses sameElements Seq("2", "3")) assert(info.idleAddresses sameElements Seq("2", "3")) assert(info.allocatedAddresses sameElements Seq("0", "1")) - assert(info.reservedAddresses.isEmpty) assert(info.getNumOfIdleResources() == 2) // release addresses info.releaseAddresses(Array("0", "1")) + assert(info.addresses sameElements Seq("2", "3", "0", "1")) assert(info.idleAddresses sameElements Seq("2", "3", "0", "1")) assert(info.allocatedAddresses.isEmpty) - assert(info.reservedAddresses.isEmpty) assert(info.getNumOfIdleResources() == 4) } test("Don't allow acquire more addresses than available") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) val e = intercept[AssertionError] { info.acquireAddresses(5) } assert(e.getMessage.contains("Required to take more addresses than available.")) } - test("Don't allow assign address that is not reserved") { + test("Don't allow assign address that is not available") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) - // Acquire addresses. - info.acquireAddresses(2) - assert(!info.reservedAddresses.contains("2")) - // Assign an address that is not reserved + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + // Assign addresses that are available + info.assignAddresses(Array("0", "1")) + assert(!info.addresses.contains("1")) + // Assign an address that is not available val e = intercept[AssertionError] { - info.assignAddresses(Array("2")) + info.assignAddresses(Array("1")) } - assert(e.getMessage.contains("Try to assign address that is not reserved.")) + assert(e.getMessage.contains("Try to assign address that is not available.")) } - test("Don't allow release address that is not reserved or allocated") { + test("Don't allow release address that is not allocated") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) - // Acquire addresses. - info.acquireAddresses(2) - assert(info.reservedAddresses sameElements Seq("0", "1")) + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) // Assign addresses info.assignAddresses(Array("0", "1")) assert(!info.allocatedAddresses.contains("2")) @@ -87,6 +87,6 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val e = intercept[AssertionError] { info.releaseAddresses(Array("2")) } - assert(e.getMessage.contains("Try to release address that is not reserved or allocated.")) + assert(e.getMessage.contains("Try to release address that is not allocated.")) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 2db261f108676..cee1a4f4d49c9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -81,6 +81,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B setupSchedulerWithMaster("local", confs: _*) } + def setupScheduler(numCores: Int, confs: (String, String)*): TaskSchedulerImpl = { + setupSchedulerWithMaster(s"local[$numCores]", confs: _*) + } + def setupSchedulerWithMaster(master: String, confs: (String, String)*): TaskSchedulerImpl = { val conf = new SparkConf().setMaster(master).setAppName("TaskSchedulerImplSuite") confs.foreach { case (k, v) => conf.set(k, v) } @@ -1245,7 +1249,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val taskGpus = 1 val executorGpus = 4 val executorCpus = 4 - val taskScheduler = setupScheduler(config.CPUS_PER_TASK.key -> taskCpus.toString, + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, s"${config.SPARK_TASK_RESOURCE_PREFIX}${GPU}${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> taskGpus.toString, s"${config.SPARK_EXECUTOR_RESOURCE_PREFIX}${GPU}${config.SPARK_RESOURCE_COUNT_SUFFIX}" -> @@ -1254,12 +1259,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val taskSet = FakeTask.createTaskSet(3) val numFreeCores = 2 - val resources = Map(GPU -> - new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"))) + val resources = new InternalExecutorResourcesInfo(Map(GPU -> + new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")))) val singleCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, resources)) val zeroGpuWorkerOffers = - IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, Map.empty)) + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, + InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO)) taskScheduler.submitTasks(taskSet) // WorkerOffer doesn't contain GPU resource, don't launch any task. var taskDescriptions = taskScheduler.resourceOffers(zeroGpuWorkerOffers).flatten diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index a22e4b87b587d..141ec3b9c4b1b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -211,7 +211,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) clock.advance(1) @@ -232,7 +232,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -240,7 +240,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) === None) + assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) @@ -263,12 +263,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. - assert(manager.resourceOffer("execC", "host2", ANY, Map.empty) === None) + assert(manager.resourceOffer("execC", "host2", ANY) === None) // Because there are no alive PROCESS_LOCAL executors, the base locality level should be // NODE_LOCAL. So, we should schedule the task on this offered NODE_LOCAL executor before // any of the locality wait timers expire. - assert(manager.resourceOffer("execA", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY).get.index === 0) } test("basic delay scheduling") { @@ -283,22 +283,22 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 3) should // get chosen before the noPref task - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, Map.empty).get.index == 2) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) // Offer host2, exec2, at NODE_LOCAL level: we should choose task 2 - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, Map.empty).get.index == 1) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).get.index == 1) // Offer host2, exec2 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) clock.advance(LOCALITY_WAIT_MS) - assert(manager.resourceOffer("exec2", "host2", NO_PREF, Map.empty).get.index == 3) + assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) } test("we do not need to delay scheduling when we only have noPref tasks in the queue") { @@ -312,10 +312,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty).get.index === 0) - assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL, Map.empty).get.index === 1) - assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL, Map.empty) == None) - assert(manager.resourceOffer("exec3", "host2", NO_PREF, Map.empty).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) + assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) + assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("exec3", "host2", NO_PREF).get.index === 2) } test("delay scheduling with fallback") { @@ -333,29 +333,29 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // Offer host1 again: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) // Offer host1 again: third task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) // Offer host2: fifth task (also on host2) should get chosen - assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty).get.index === 4) + assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 4) // Now that we've launched a local task, we should no longer launch the task for host3 - assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty) === None) + assert(manager.resourceOffer("exec2", "host2", ANY) === None) clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally - assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty).get.index === 3) + assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) } test("delay scheduling with failed hosts") { @@ -371,28 +371,28 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // After this, nothing should get chosen, because we have separated tasks with unavailable // preference from the noPrefPendingTasks - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2", SlaveLost()) // nothing should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 1) - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) // all finished - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty) === None) - assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec2", "host2", ANY) === None) } test("task result lost") { @@ -403,14 +403,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // Tell it the task has finished but the result was lost. manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost) assert(sched.endedTasks(0) === TaskResultLost) // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) } test("repeated failures lead to task set abortion") { @@ -424,7 +424,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. (1 to manager.maxTaskFailures).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", ANY, Map.empty) + val offerResult = manager.resourceOffer("exec1", "host1", ANY) assert(offerResult.isDefined, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) @@ -460,7 +460,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -471,15 +471,15 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty).isEmpty) - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, Map.empty).isEmpty) - assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL, Map.empty).isEmpty) - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, Map.empty) + val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -491,12 +491,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, Map.empty).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL).isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", ANY, Map.empty) + val offerResult = manager.resourceOffer("exec2", "host2", ANY) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -507,7 +507,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", ANY, Map.empty).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) } // Despite advancing beyond the time for expiring executors from within the blacklist, @@ -515,17 +515,17 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, Map.empty) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) assert(offerResult.isEmpty) } { - val offerResult = manager.resourceOffer("exec3", "host3", ANY, Map.empty) + val offerResult = manager.resourceOffer("exec3", "host3", ANY) assert(offerResult.isDefined) assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec3", "host3", ANY, Map.empty).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) @@ -584,14 +584,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg manager.executorAdded() sched.addExecutor("execC", "host2") manager.executorAdded() - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).isDefined) + assert(manager.resourceOffer("exec1", "host1", ANY).isDefined) sched.removeExecutor("execA") manager.executorLost( "execA", "host1", ExecutorExited(143, false, "Terminated for reason unrelated to running tasks")) assert(!sched.taskSetsFailed.contains(taskSet.id)) - assert(manager.resourceOffer("execC", "host2", ANY, Map.empty).isDefined) + assert(manager.resourceOffer("execC", "host2", ANY).isDefined) sched.removeExecutor("execC") manager.executorLost( "execC", "host2", ExecutorExited(1, true, "Terminated due to issue with running tasks")) @@ -619,12 +619,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL - assert(manager.resourceOffer("execC", "host3", RACK_LOCAL, Map.empty) === None) + assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) // Task 0 can be scheduled with ANY - assert(manager.resourceOffer("execC", "host3", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("execC", "host3", ANY).get.index === 0) // Offer host2 // Task 1 can be scheduled with RACK_LOCAL - assert(manager.resourceOffer("execB", "host2", RACK_LOCAL, Map.empty).get.index === 1) + assert(manager.resourceOffer("execB", "host2", RACK_LOCAL).get.index === 1) } test("do not emit warning when serialized task is small") { @@ -635,7 +635,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(!manager.emittedTaskSizeWarning) } @@ -649,7 +649,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY, Map.empty).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.emittedTaskSizeWarning) } @@ -663,7 +663,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { - manager.resourceOffer("exec1", "host1", ANY, Map.empty) + manager.resourceOffer("exec1", "host1", ANY) } assert(manager.isZombie) } @@ -734,12 +734,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer host1, which should be accepted as a PROCESS_LOCAL location // by the one task in the task set - val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL, Map.empty).get + val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL).get // Mark the task as available for speculation, and then offer another resource, // which should be used to launch a speculative copy of the task. manager.speculatableTasks += singleTask.partitionId - val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY, Map.empty).get + val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY).get assert(manager.runningTasks === 2) assert(manager.isZombie === false) @@ -824,7 +824,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF, Map.empty) + val taskOption = manager.resourceOffer(exec, host, NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -850,7 +850,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY, Map.empty) + val taskOption = manager.resourceOffer("exec2", "host2", ANY) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -879,19 +879,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, Map.empty).get.index === 0) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF, Map.empty).get.index == 1) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) manager.speculatableTasks += 1 clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task - assert(manager.resourceOffer("execA", "host1", NO_PREF, Map.empty).get.index === 2) + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) // schedule the speculative task - assert(manager.resourceOffer("execB", "host2", NO_PREF, Map.empty).get.index === 1) + assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks - assert(manager.resourceOffer("execB", "host2", ANY, Map.empty).get.index === 3) + assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } test("node-local tasks should be scheduled right away " + @@ -908,13 +908,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty).get.index === 0) - assert(manager.resourceOffer("execA", "host2", NODE_LOCAL, Map.empty).get.index === 1) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, Map.empty).get.index === 3) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, Map.empty) === None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) + assert(manager.resourceOffer("execA", "host2", NODE_LOCAL).get.index === 1) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL).get.index === 3) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL) === None) // schedule no-preference after node local ones - assert(manager.resourceOffer("execA", "host3", NO_PREF, Map.empty).get.index === 2) + assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) } test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") @@ -930,13 +930,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty).get.index === 2) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, Map.empty).get.index === 3) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty).get.index === 0) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, Map.empty).get.index === 1) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty) == None) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, Map.empty) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None) } test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { @@ -950,13 +950,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, Map.empty).get.index === 1) - assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL, Map.empty).get.index === 2) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2) // no-pref tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, Map.empty) == None) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, Map.empty) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF, Map.empty).get.index === 0) - assert(manager.resourceOffer("execA", "host1", ANY, Map.empty) == None) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY) == None) } test("Ensure TaskSetManager is usable after addition of levels") { @@ -977,9 +977,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.pendingTasksWithNoPrefs.size === 0) // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) - assert(manager.resourceOffer("execA", "host1", ANY, Map.empty) !== None) + assert(manager.resourceOffer("execA", "host1", ANY) !== None) clock.advance(LOCALITY_WAIT_MS * 4) - assert(manager.resourceOffer("execB.2", "host2", ANY, Map.empty) !== None) + assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1", SlaveLost()) @@ -988,7 +988,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: - assert(manager.resourceOffer("execC", "host3", ANY, Map.empty) !== None) + assert(manager.resourceOffer("execC", "host3", ANY) !== None) } test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { @@ -1040,7 +1040,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF, Map.empty) + val taskOption = manager.resourceOffer(k, v, NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1061,7 +1061,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1100,7 +1100,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF, Map.empty) + val taskOption = manager.resourceOffer(k, v, NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1133,7 +1133,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) sched.endedTasks(task.taskId) = endReason assert(!manager.isZombie) - val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF, Map.empty) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") tasks += nextTask.get } @@ -1149,7 +1149,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(3, 4)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption5.isDefined) val speculativeTask = taskOption5.get assert(speculativeTask.index === 3 || speculativeTask.index === 4) @@ -1174,7 +1174,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.isZombie) // now run another speculative task - val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOpt6.isDefined) val speculativeTask2 = taskOpt6.get assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) @@ -1205,7 +1205,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any())).thenAnswer( (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -1250,7 +1250,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec2" -> "host1" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY, Map.empty)} + (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY)} } assert(taskDescs.size === 4) @@ -1287,7 +1287,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec2" -> "host2" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY, Map.empty)} + (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY)} } assert(taskDescs.size === 4) @@ -1323,7 +1323,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) val taskSetManagerSpy = spy(taskSetManager) - val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY, Map.empty) + val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) // Assert the task has been black listed on the executor it was last executed on. when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( @@ -1351,9 +1351,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val manager1 = new TaskSetManager(sched, taskSet1, MAX_TASK_FAILURES, clock = new ManualClock) // all tasks from the first taskset have the same jars - val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption1.get.addedJars === addedJarsPreTaskSet) - val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption2.get.addedJars === addedJarsPreTaskSet) // even with a jar added mid-TaskSet @@ -1361,7 +1361,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sc.addJar(jarPath.toString) val addedJarsMidTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) assert(addedJarsPreTaskSet !== addedJarsMidTaskSet) - val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF) // which should have the old version of the jars list assert(taskOption3.get.addedJars === addedJarsPreTaskSet) @@ -1369,7 +1369,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet2 = FakeTask.createTaskSet(1) val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) - val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } @@ -1466,7 +1466,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec3" -> "host3", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF, Map.empty) + val taskOption = manager.resourceOffer(exec, host, NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -1492,7 +1492,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY, Map.empty) + val taskOption = manager.resourceOffer("exec2", "host2", ANY) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -1537,7 +1537,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF, Map.empty) + val taskOption = manager.resourceOffer(k, v, NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1557,7 +1557,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, Map.empty) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1616,18 +1616,17 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(FakeRackUtil.numBatchInvocation === 1) assert(FakeRackUtil.numSingleHostInvocation === 0) // with rack locality, reject an offer on a host with an unknown rack - assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL, - Map.empty).isEmpty) + assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL).isEmpty) (0 until 20).foreach { rackIdx => (0 until 5).foreach { offerIdx => // if we offer hosts which are not in preferred locations, // we'll reject them at NODE_LOCAL level, // but accept them at RACK_LOCAL level if they're on OK racks val hostIdx = 100 + rackIdx - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL, - Map.empty).isEmpty) - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL, - Map.empty).isDefined) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, + TaskLocality.NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, + TaskLocality.RACK_LOCAL).isDefined) } } // check no more expensive calls to the rack resolution. manager.resourceOffer() will call @@ -1647,7 +1646,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val availableResources = Map(GPU -> new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"))) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, availableResources) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, + new InternalExecutorResourcesInfo(availableResources)) assert(taskOption.isDefined) val allocatedResources = taskOption.get.resources assert(allocatedResources.size == 1) From 7844e5c9c3b02183d9ccd8b1f419eb8d2b584fe3 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 30 May 2019 18:55:34 -0700 Subject: [PATCH 13/18] add comments --- .../spark/scheduler/ExecutorResourceInfo.scala | 7 ++++++- .../scheduler/InternalExecutorResourcesInfo.scala | 12 ++++++++++-- .../apache/spark/scheduler/TaskSetManagerSuite.scala | 8 ++++---- 3 files changed, 20 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index 193aace0166d8..3c45efe46e4d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -30,7 +30,7 @@ private[spark] class ExecutorResourceInfo( // Addresses of resource that has not been acquired. // Exposed for testing only. - private[scheduler] val idleAddresses: ArrayBuffer[String] = addresses.clone() + private[scheduler] var idleAddresses: ArrayBuffer[String] = addresses.clone() // Addresses of resource that has been assigned to running tasks. // Exposed for testing only. @@ -73,4 +73,9 @@ private[spark] class ExecutorResourceInfo( addresses -= address } } + + // Reset the resource addresses that has not been acquired. + def resetIdleAddresses(): Unit = { + idleAddresses = addresses.clone() + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala index 192eda0a11eb8..a722c54fa1c41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala @@ -19,15 +19,23 @@ package org.apache.spark.scheduler /** - * Class to hold information about resources on an Executor. + * Internal class to expose executor resources information to TaskScheduler from SchedulerBackend, + * while also ensure the TaskScheduler will not modify the executor resources information. + * Idealy we shall keep a copy of the map from resource name to resource addresses inside this + * class, but the deep copy can be time consuming when you have many executors, so we just keep + * the copy inside class ExecutorResourceInfo. */ private[scheduler] class InternalExecutorResourcesInfo( private val resources: Map[String, ExecutorResourceInfo]) extends Serializable { + // Reset all the buffered addresses. + resources.values.foreach(_.resetIdleAddresses()) + + // Number of resource addresses that can be acquired. def getNumOfIdleResources(resourceName: String): Int = resources.get(resourceName).map(_.getNumOfIdleResources()).getOrElse(0) - // Reserve given number of resource addresses, these addresses can be assigned to a future + // Acquire given number of resource addresses, these addresses can be assigned to a future // launched task. def acquireAddresses(resourceName: String, num: Int): Seq[String] = { resources.get(resourceName).map(_.acquireAddresses(num)).getOrElse(Seq.empty) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 141ec3b9c4b1b..d07de8662dc1f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1623,10 +1623,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // we'll reject them at NODE_LOCAL level, // but accept them at RACK_LOCAL level if they're on OK racks val hostIdx = 100 + rackIdx - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, - TaskLocality.NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, - TaskLocality.RACK_LOCAL).isDefined) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL) + .isEmpty) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL) + .isDefined) } } // check no more expensive calls to the rack resolution. manager.resourceOffer() will call From 04fa3808020203a1e5818b33d445599bbef76d95 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Fri, 31 May 2019 15:56:05 -0700 Subject: [PATCH 14/18] update ExecutorResourceInfo --- .../scheduler/ExecutorResourceInfo.scala | 76 ++++++++----------- .../InternalExecutorResourcesInfo.scala | 47 ------------ .../spark/scheduler/TaskSchedulerImpl.scala | 13 +++- .../spark/scheduler/TaskSetManager.scala | 11 ++- .../apache/spark/scheduler/WorkerOffer.scala | 4 +- .../CoarseGrainedSchedulerBackend.scala | 16 ++-- .../local/LocalSchedulerBackend.scala | 2 +- .../CoarseGrainedSchedulerBackendSuite.scala | 20 ++--- .../scheduler/ExecutorResourceInfoSuite.scala | 73 ++++++------------ .../scheduler/TaskSchedulerImplSuite.scala | 6 +- .../spark/scheduler/TaskSetManagerSuite.scala | 11 ++- 11 files changed, 101 insertions(+), 178 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index 3c45efe46e4d4..734bbeb8656ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -17,7 +17,9 @@ package org.apache.spark.scheduler -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap + +import org.apache.spark.SparkException /** * Class to hold information about a type of Resource on an Executor. This information is managed @@ -25,57 +27,45 @@ import scala.collection.mutable.ArrayBuffer * information. */ private[spark] class ExecutorResourceInfo( - private val name: String, - private[scheduler] val addresses: ArrayBuffer[String]) extends Serializable { - - // Addresses of resource that has not been acquired. - // Exposed for testing only. - private[scheduler] var idleAddresses: ArrayBuffer[String] = addresses.clone() + val name: String, + private val addresses: Seq[String]) extends Serializable { - // Addresses of resource that has been assigned to running tasks. - // Exposed for testing only. - private[scheduler] val allocatedAddresses: ArrayBuffer[String] = ArrayBuffer.empty - - def getName(): String = name + private val addressesMap = new HashMap[String, Boolean]() + addresses.foreach(addressesMap.put(_, true)) - // Number of resource addresses that can be acquired. - def getNumOfIdleResources(): Int = idleAddresses.size + // Sequence of currently available resource addresses. + def availableAddrs: Seq[String] = addressesMap.toList.filter(_._2 == true).map(_._1) - // Reserve given number of resource addresses, these addresses can be assigned to a future - // launched task. - def acquireAddresses(num: Int): Seq[String] = { - assert(num <= idleAddresses.size, "Required to take more addresses than available. " + - s"Required $num $name addresses, but only ${idleAddresses.size} available.") - val addrs = idleAddresses.take(num) - idleAddresses --= addrs - addrs - } + // Sequence of currently assigned resource addresses. + // Exposed for testing only. + private[scheduler] def assignedAddrs: Seq[String] = + addressesMap.toList.filter(_._2 == false).map(_._1) - // Give back a sequence of resource addresses, these addresses must have been assigned. Resource - // addresses are released when a task has finished. - def releaseAddresses(addrs: Array[String]): Unit = { + // Acquire a sequence of resource addresses (to a launched task), these addresses must be + // available. When the task finishes, it will return the acquired resource addresses. + def acquire(addrs: Seq[String]): Unit = { addrs.foreach { address => - assert(allocatedAddresses.contains(address), "Try to release address that is not " + - s"allocated. $name address $address is not allocated.") - addresses += address - idleAddresses += address - allocatedAddresses -= address + val isAvailable = addressesMap.getOrElse(address, false) + if (isAvailable) { + addressesMap(address) = false + } else { + throw new SparkException(s"Try to acquire address that is not available. $name address " + + s"$address is not available.") + } } } - // Assign a sequence of resource addresses (to a launched task), these addresses must be - // available. - def assignAddresses(addrs: Array[String]): Unit = { + // Release a sequence of resource addresses, these addresses must have been assigned. Resource + // addresses are released when a task has finished. + def release(addrs: Seq[String]): Unit = { addrs.foreach { address => - assert(addresses.contains(address), "Try to assign address that is not available. " + - s"$name address $address is not available.") - allocatedAddresses += address - addresses -= address + val isAssigned = addressesMap.getOrElse(address, true) + if (!isAssigned) { + addressesMap(address) = true + } else { + throw new SparkException(s"Try to release address that is not assigned. $name address " + + s"$address is not assigned.") + } } } - - // Reset the resource addresses that has not been acquired. - def resetIdleAddresses(): Unit = { - idleAddresses = addresses.clone() - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala deleted file mode 100644 index a722c54fa1c41..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/InternalExecutorResourcesInfo.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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 - - -/** - * Internal class to expose executor resources information to TaskScheduler from SchedulerBackend, - * while also ensure the TaskScheduler will not modify the executor resources information. - * Idealy we shall keep a copy of the map from resource name to resource addresses inside this - * class, but the deep copy can be time consuming when you have many executors, so we just keep - * the copy inside class ExecutorResourceInfo. - */ -private[scheduler] class InternalExecutorResourcesInfo( - private val resources: Map[String, ExecutorResourceInfo]) extends Serializable { - - // Reset all the buffered addresses. - resources.values.foreach(_.resetIdleAddresses()) - - // Number of resource addresses that can be acquired. - def getNumOfIdleResources(resourceName: String): Int = - resources.get(resourceName).map(_.getNumOfIdleResources()).getOrElse(0) - - // Acquire given number of resource addresses, these addresses can be assigned to a future - // launched task. - def acquireAddresses(resourceName: String, num: Int): Seq[String] = { - resources.get(resourceName).map(_.acquireAddresses(num)).getOrElse(Seq.empty) - } -} - -private[scheduler] object InternalExecutorResourcesInfo { - final val EMPTY_RESOURCES_INFO = new InternalExecutorResourcesInfo(Map.empty) -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8b886c293c49c..4f10b6ae89135 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,7 +22,7 @@ import java.util.{Locale, Timer, TimerTask} import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap, HashSet} import scala.util.Random import org.apache.spark._ @@ -330,7 +330,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], - availableResources: Array[InternalExecutorResourcesInfo], + availableResources: Array[Map[String, Buffer[String]]], tasks: IndexedSeq[ArrayBuffer[TaskDescription]], addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = { var launchedTask = false @@ -350,6 +350,11 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds(execId).add(tid) availableCpus(i) -= CPUS_PER_TASK assert(availableCpus(i) >= 0) + task.resources.foreach { case (rName, rInfo) => + availableResources(i).getOrElse(rName, + throw new SparkException(s"Try to acquire resource $rName that doesn't exist.")) + .remove(0, rInfo.addresses.size) + } // Only update hosts for a barrier task. if (taskSet.isBarrier) { // The executor address is expected to be non empty. @@ -372,9 +377,9 @@ private[spark] class TaskSchedulerImpl( /** * Check whether the resources from the WorkerOffer are enough to run at least one task. */ - private def resourcesMeetTaskRequirements(resources: InternalExecutorResourcesInfo): Boolean = { + private def resourcesMeetTaskRequirements(resources: Map[String, Buffer[String]]): Boolean = { resourcesPerTask.forall { case (rName, rNum) => - resources.getNumOfIdleResources(rName) >= rNum + resources.contains(rName) && resources(rName).size >= rNum } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 4a43a91af9e17..c6ddcf44d5c8b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.immutable.Map -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap, HashSet} import scala.math.max import scala.util.control.NonFatal @@ -30,7 +30,6 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ -import org.apache.spark.scheduler.InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.util.{AccumulatorV2, Clock, LongAccumulator, SystemClock, Utils} import org.apache.spark.util.collection.MedianHeap @@ -470,7 +469,7 @@ private[spark] class TaskSetManager( execId: String, host: String, maxLocality: TaskLocality.TaskLocality, - availableResources: InternalExecutorResourcesInfo = EMPTY_RESOURCES_INFO) + availableResources: Map[String, Buffer[String]] = Map.empty) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -536,7 +535,11 @@ private[spark] class TaskSetManager( s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") val extraResources = sched.resourcesPerTask.map { case (rName, rNum) => - val allocatedAddresses = availableResources.acquireAddresses(rName, rNum) + val rAddresses = availableResources.getOrElse(rName, Buffer.empty) + assert(rAddresses.size >= rNum, s"Required $rNum $rName addresses, but only " + + s"${rAddresses.size} available.") + // We'll drop the allocated addresses later inside TaskSchedulerImpl. + val allocatedAddresses = rAddresses.take(rNum) (rName, new ResourceInformation(rName, allocatedAddresses.toArray)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala index 3d39b9e9ea458..522dbfa9457b9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.Buffer + /** * Represents free resources available on an executor. */ @@ -28,4 +30,4 @@ case class WorkerOffer( // `address` is an optional hostPort string, it provide more useful information than `host` // when multiple executors are launched on the same host. address: Option[String] = None, - resources: InternalExecutorResourcesInfo = InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO) + resources: Map[String, Buffer[String]] = Map.empty) 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 9cc5130f1c3ce..107a822aeaee1 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 @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.Future import org.apache.hadoop.security.UserGroupInformation @@ -148,7 +148,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorInfo.freeCores += scheduler.CPUS_PER_TASK for ((k, v) <- resources) { executorInfo.availableResources.get(k).foreach { r => - r.releaseAddresses(v.addresses) + r.release(v.addresses) } } makeOffers(executorId) @@ -218,7 +218,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val data = new ExecutorData(executorRef, executorAddress, hostname, cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, mutable.Map.empty[String, ExecutorResourceInfo] ++= resources.mapValues(v => - new ExecutorResourceInfo(v.name, v.addresses.to[ArrayBuffer]))) + new ExecutorResourceInfo(v.name, v.addresses))) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -273,7 +273,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case (id, executorData) => new WorkerOffer(id, executorData.executorHost, executorData.freeCores, Some(executorData.executorAddress.hostPort), - new InternalExecutorResourcesInfo(executorData.availableResources.toMap)) + executorData.availableResources.map { case (rName, rInfo) => + (rName, rInfo.availableAddrs.toBuffer) + }.toMap) }.toIndexedSeq scheduler.resourceOffers(workOffers) } @@ -300,7 +302,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val workOffers = IndexedSeq( new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores, Some(executorData.executorAddress.hostPort), - new InternalExecutorResourcesInfo(executorData.availableResources.toMap))) + executorData.availableResources.map { case (rName, rInfo) => + (rName, rInfo.availableAddrs.toBuffer) + }.toMap)) scheduler.resourceOffers(workOffers) } else { Seq.empty @@ -340,7 +344,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorData.freeCores -= scheduler.CPUS_PER_TASK task.resources.foreach { case (rName, rInfo) => assert(executorData.availableResources.contains(rName)) - executorData.availableResources(rName).assignAddresses(rInfo.addresses) + executorData.availableResources(rName).acquire(rInfo.addresses) } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index 8112d3a3a62cf..a0e2254e86f96 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -83,7 +83,7 @@ private[spark] class LocalEndpoint( def reviveOffers() { // local mode doesn't support extra resources like GPUs right now val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores, - Some(rpcEnv.address.hostPort), InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO)) + Some(rpcEnv.address.hostPort))) for (task <- scheduler.resourceOffers(offers).flatten) { freeCores -= scheduler.CPUS_PER_TASK executor.launchTask(executorBackend, task) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 5febff6479f9e..eb23fb1608f18 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -228,29 +228,21 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo var execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).idleAddresses === Array("0", "1", "3")) + assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], new Properties(), taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() - var numPendingTasks = 1 - when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenAnswer((_: InvocationOnMock) => { - if (numPendingTasks > 0) { - numPendingTasks -= 1 - new InternalExecutorResourcesInfo(backend.getExecutorAvailableResources("1").toMap) - .acquireAddresses(GPU, 1) - taskDescs - } - }) + when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(taskDescs) backend.driverEndpoint.send(ReviveOffers) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).addresses === Array("1", "3")) - assert(execResources(GPU).allocatedAddresses === Array("0")) + assert(execResources(GPU).availableAddrs.sorted === Array("1", "3")) + assert(execResources(GPU).assignedAddrs === Array("0")) } backend.driverEndpoint.send( @@ -258,8 +250,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).addresses === Array("1", "3", "0")) - assert(execResources(GPU).allocatedAddresses.isEmpty) + assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) + assert(execResources(GPU).assignedAddrs.isEmpty) } sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) assert(executorAddedCount === 3) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala index 7b0be813026b0..953fa04f33bf9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -19,74 +19,51 @@ package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ResourceName.GPU -import org.apache.spark.SparkFunSuite - class ExecutorResourceInfoSuite extends SparkFunSuite { test("Track Executor Resource information") { // Init Executor Resource. val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) - assert(info.addresses sameElements Seq("0", "1", "2", "3")) - assert(info.idleAddresses sameElements Seq("0", "1", "2", "3")) - assert(info.allocatedAddresses.isEmpty) - assert(info.getNumOfIdleResources() == 4) + assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) + assert(info.assignedAddrs.isEmpty) // Acquire addresses - info.acquireAddresses(2) - assert(info.addresses sameElements Seq("0", "1", "2", "3")) - assert(info.idleAddresses sameElements Seq("2", "3")) - assert(info.allocatedAddresses sameElements Seq.empty) - assert(info.getNumOfIdleResources() == 2) - - // Assign addresses - info.assignAddresses(Array("0", "1")) - assert(info.addresses sameElements Seq("2", "3")) - assert(info.idleAddresses sameElements Seq("2", "3")) - assert(info.allocatedAddresses sameElements Seq("0", "1")) - assert(info.getNumOfIdleResources() == 2) + info.acquire(Seq("0", "1")) + assert(info.availableAddrs.sorted sameElements Seq("2", "3")) + assert(info.assignedAddrs.sorted sameElements Seq("0", "1")) // release addresses - info.releaseAddresses(Array("0", "1")) - assert(info.addresses sameElements Seq("2", "3", "0", "1")) - assert(info.idleAddresses sameElements Seq("2", "3", "0", "1")) - assert(info.allocatedAddresses.isEmpty) - assert(info.getNumOfIdleResources() == 4) - } - - test("Don't allow acquire more addresses than available") { - // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) - val e = intercept[AssertionError] { - info.acquireAddresses(5) - } - assert(e.getMessage.contains("Required to take more addresses than available.")) + info.release(Array("0", "1")) + assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) + assert(info.assignedAddrs.isEmpty) } - test("Don't allow assign address that is not available") { + test("Don't allow acquire address that is not available") { // Init Executor Resource. val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) - // Assign addresses that are available - info.assignAddresses(Array("0", "1")) - assert(!info.addresses.contains("1")) - // Assign an address that is not available - val e = intercept[AssertionError] { - info.assignAddresses(Array("1")) + // Acquire some addresses. + info.acquire(Seq("0", "1")) + assert(!info.availableAddrs.contains("1")) + // Acquire an address that is not available + val e = intercept[SparkException] { + info.acquire(Array("1")) } - assert(e.getMessage.contains("Try to assign address that is not available.")) + assert(e.getMessage.contains("Try to acquire address that is not available.")) } - test("Don't allow release address that is not allocated") { + test("Don't allow release address that is not assigned") { // Init Executor Resource. val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) - // Assign addresses - info.assignAddresses(Array("0", "1")) - assert(!info.allocatedAddresses.contains("2")) - // Release an address that is not allocated - val e = intercept[AssertionError] { - info.releaseAddresses(Array("2")) + // Acquire addresses + info.acquire(Array("0", "1")) + assert(!info.assignedAddrs.contains("2")) + // Release an address that is not assigned + val e = intercept[SparkException] { + info.release(Array("2")) } - assert(e.getMessage.contains("Try to release address that is not allocated.")) + assert(e.getMessage.contains("Try to release address that is not assigned.")) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index cee1a4f4d49c9..456996c8eb3f2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1259,13 +1259,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val taskSet = FakeTask.createTaskSet(3) val numFreeCores = 2 - val resources = new InternalExecutorResourcesInfo(Map(GPU -> - new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")))) + val resources = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) val singleCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, resources)) val zeroGpuWorkerOffers = - IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, - InternalExecutorResourcesInfo.EMPTY_RESOURCES_INFO)) + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, Map.empty)) taskScheduler.submitTasks(taskSet) // WorkerOffer doesn't contain GPU resource, don't launch any task. var taskDescriptions = taskScheduler.resourceOffers(zeroGpuWorkerOffers).flatten diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index d07de8662dc1f..3d09b10a6dc8d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1644,15 +1644,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val availableResources = Map(GPU -> - new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3"))) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, - new InternalExecutorResourcesInfo(availableResources)) + val availableResources = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, availableResources) assert(taskOption.isDefined) val allocatedResources = taskOption.get.resources assert(allocatedResources.size == 1) assert(allocatedResources(GPU).addresses sameElements Array("0", "1")) - // Allocated resource addresses should no longer be available in `availableResources`. - assert(availableResources(GPU).idleAddresses sameElements Array("2", "3")) + // Allocated resource addresses should still present in `availableResources`, they will only + // get removed inside TaskSchedulerImpl later. + assert(availableResources(GPU) sameElements Array("0", "1", "2", "3")) } } From dcc147ee3682a2762e20d75c6d86450930fbe0aa Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Fri, 31 May 2019 17:31:59 -0700 Subject: [PATCH 15/18] update config check and comments --- .../scala/org/apache/spark/SparkConf.scala | 23 ----------- .../scala/org/apache/spark/SparkContext.scala | 38 +++++++++++++++---- .../scala/org/apache/spark/TaskContext.scala | 4 +- .../CoarseGrainedExecutorBackend.scala | 8 ++-- .../scheduler/ExecutorResourceInfo.scala | 24 ++++++++---- .../CoarseGrainedSchedulerBackend.scala | 11 +++--- .../scheduler/cluster/ExecutorData.scala | 4 +- .../scala/org/apache/spark/ResourceName.scala | 1 + .../org/apache/spark/SparkConfSuite.scala | 19 +++++----- .../org/apache/spark/SparkContextSuite.scala | 33 ++++++++-------- 10 files changed, 89 insertions(+), 76 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 33d4aeb0863aa..227f4a5bb3a2d 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -612,29 +612,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be no less than the value of " + s"${EXECUTOR_HEARTBEAT_INTERVAL.key}=${executorHeartbeatIntervalMs}ms.") - - // Make sure the executor resources were specified and are large enough if - // any task resources were specified. - val taskResourcesAndCount = getTaskResourceRequirements() - val executorResourcesAndCounts = - getAllWithPrefixAndSuffix(SPARK_EXECUTOR_RESOURCE_PREFIX, SPARK_RESOURCE_COUNT_SUFFIX).toMap - - taskResourcesAndCount.foreach { case (rName, taskCount) => - val execCount = executorResourcesAndCounts.get(rName).getOrElse( - throw new SparkException( - s"The executor resource config: " + - s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + - "needs to be specified since a task requirement config: " + - s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} was specified") - ) - if (execCount.toLong < taskCount.toLong) { - throw new SparkException( - s"The executor resource config: " + - s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + - s"= $execCount has to be >= the task config: " + - s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount") - } - } } /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c53c815f4d3df..74c0592a7d397 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2722,21 +2722,45 @@ object SparkContext extends Logging { s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.") } - // Resources per executor must be the same multiple numbers of the task requirements. - val numSlots = execCores / taskCores + // Calculate the max slots each executor can provide based on resources available on each + // executor and resources required by each task. val taskResourcesAndCount = sc.conf.getTaskResourceRequirements() val executorResourcesAndCounts = sc.conf.getAllWithPrefixAndSuffix( SPARK_EXECUTOR_RESOURCE_PREFIX, SPARK_RESOURCE_COUNT_SUFFIX).toMap + val numSlots = (taskResourcesAndCount.map { case (rName, taskCount) => + // Make sure the executor resources were specified through config. + val execCount = executorResourcesAndCounts.getOrElse(rName, + throw new SparkException( + s"The executor resource config: " + + s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + + "needs to be specified since a task requirement config: " + + s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} was specified") + ) + // Make sure the executor resources are large enough to launch at least one task. + if (execCount.toLong < taskCount.toLong) { + throw new SparkException( + s"The executor resource config: " + + s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + + s"= $execCount has to be >= the task config: " + + s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount") + } + execCount.toInt / taskCount + }.toList ++ Seq(execCores / taskCores)).min // There have been checks inside SparkConf to make sure the executor resources were specified // and are large enough if any task resources were specified. taskResourcesAndCount.foreach { case (rName, taskCount) => val execCount = executorResourcesAndCounts(rName) if (execCount.toInt / taskCount.toInt != numSlots) { - throw new SparkException( - s"The value of executor resource config: " + - s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + - s"= $execCount has to be $numSlots times the number of the task config: " + - s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount") + val message = s"The value of executor resource config: " + + s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + + s"= $execCount is more than that tasks can take: $numSlots * " + + s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount. " + + s"The resources may be wasted." + if (Utils.isTesting) { + throw new SparkException(message) + } else { + logWarning(message) + } } } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 04248dfa2f735..803167ee95aa0 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -177,8 +177,8 @@ abstract class TaskContext extends Serializable { def getLocalProperty(key: String): String /** - * Resources allocated to the task. A task may retrieve the resource name and available resource - * addresses from the map values. + * Resources allocated to the task. The key is the resource name and the value is information + * about the resource. Please refer to [[ResourceInformation]] for specifics. */ @Evolving def resources(): Map[String, ResourceInformation] diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 8cf2760de259c..13bf8a97ef2c5 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -66,9 +66,11 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() - // Map each taskId to the resource informations allocated to it, the resource information - // includes resource name and resource addresses. - // Exposed for testing only. + /** + * Map each taskId to the information about the resource allocated to it, Please refer to + * [[ResourceInformation]] for specifics. + * Exposed for testing only. + */ private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] override def onStart() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index 734bbeb8656ed..68f8627206bdd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -25,6 +25,8 @@ import org.apache.spark.SparkException * Class to hold information about a type of Resource on an Executor. This information is managed * by SchedulerBackend, and TaskScheduler shall schedule tasks on idle Executors based on the * information. + * @param name Resource name + * @param addresses Resource addresses provided by the executor */ private[spark] class ExecutorResourceInfo( val name: String, @@ -33,16 +35,22 @@ private[spark] class ExecutorResourceInfo( private val addressesMap = new HashMap[String, Boolean]() addresses.foreach(addressesMap.put(_, true)) - // Sequence of currently available resource addresses. + /** + * Sequence of currently available resource addresses. + */ def availableAddrs: Seq[String] = addressesMap.toList.filter(_._2 == true).map(_._1) - // Sequence of currently assigned resource addresses. - // Exposed for testing only. + /** + * Sequence of currently assigned resource addresses. + * Exposed for testing only. + */ private[scheduler] def assignedAddrs: Seq[String] = addressesMap.toList.filter(_._2 == false).map(_._1) - // Acquire a sequence of resource addresses (to a launched task), these addresses must be - // available. When the task finishes, it will return the acquired resource addresses. + /** + * Acquire a sequence of resource addresses (to a launched task), these addresses must be + * available. When the task finishes, it will return the acquired resource addresses. + */ def acquire(addrs: Seq[String]): Unit = { addrs.foreach { address => val isAvailable = addressesMap.getOrElse(address, false) @@ -55,8 +63,10 @@ private[spark] class ExecutorResourceInfo( } } - // Release a sequence of resource addresses, these addresses must have been assigned. Resource - // addresses are released when a task has finished. + /** + * Release a sequence of resource addresses, these addresses must have been assigned. Resource + * addresses are released when a task has finished. + */ def release(addrs: Seq[String]): Unit = { addrs.foreach { address => val isAssigned = addressesMap.getOrElse(address, true) 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 107a822aeaee1..e151e1eaf0b68 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 @@ -215,10 +215,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) + val availableResources = resources.map{ case (k, v) => + (v.name, new ExecutorResourceInfo(v.name, v.addresses))} val data = new ExecutorData(executorRef, executorAddress, hostname, cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, - mutable.Map.empty[String, ExecutorResourceInfo] ++= resources.mapValues(v => - new ExecutorResourceInfo(v.name, v.addresses))) + availableResources) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -547,10 +548,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } // this function is for testing only - def getExecutorAvailableResources(executorId: String): - mutable.Map[String, ExecutorResourceInfo] = { - executorDataMap.get(executorId).map(_.availableResources). - getOrElse(mutable.Map.empty[String, ExecutorResourceInfo]) + def getExecutorAvailableResources(executorId: String): Map[String, ExecutorResourceInfo] = { + executorDataMap.get(executorId).map(_.availableResources).getOrElse(Map.empty) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 94904d32c3d0f..9e372e9fac46a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler.cluster -import scala.collection.mutable - import org.apache.spark.ResourceInformation import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler.ExecutorResourceInfo @@ -43,5 +41,5 @@ private[cluster] class ExecutorData( override val logUrlMap: Map[String, String], override val attributes: Map[String, String], override val totalResources: Map[String, ResourceInformation], - val availableResources: mutable.Map[String, ExecutorResourceInfo] + val availableResources: Map[String, ExecutorResourceInfo] ) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, totalResources) diff --git a/core/src/test/scala/org/apache/spark/ResourceName.scala b/core/src/test/scala/org/apache/spark/ResourceName.scala index c97eaae00bbd1..6efe064a7773f 100644 --- a/core/src/test/scala/org/apache/spark/ResourceName.scala +++ b/core/src/test/scala/org/apache/spark/ResourceName.scala @@ -20,4 +20,5 @@ package org.apache.spark private[spark] object ResourceName { // known types of resources final val GPU: String = "gpu" + final val FPGA: String = "fpga" } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 30fe18b53e8ab..6978f303b3760 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -25,6 +25,7 @@ import scala.util.{Random, Try} import com.esotericsoftware.kryo.Kryo +import org.apache.spark.ResourceName._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.Kryo._ @@ -449,25 +450,25 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst test("get task resource requirement from config") { val conf = new SparkConf() - conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_SUFFIX, "2") - conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_SUFFIX, "1") + conf.set(SPARK_TASK_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "2") + conf.set(SPARK_TASK_RESOURCE_PREFIX + FPGA + SPARK_RESOURCE_COUNT_SUFFIX, "1") var taskResourceRequirement = conf.getTaskResourceRequirements() assert(taskResourceRequirement.size == 2) - assert(taskResourceRequirement("gpu") == 2) - assert(taskResourceRequirement("fpga") == 1) + assert(taskResourceRequirement(GPU) == 2) + assert(taskResourceRequirement(FPGA) == 1) - conf.remove(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_SUFFIX) + conf.remove(SPARK_TASK_RESOURCE_PREFIX + FPGA + SPARK_RESOURCE_COUNT_SUFFIX) // Ignore invalid prefix - conf.set("spark.invalid.prefix" + "fpga" + SPARK_RESOURCE_COUNT_SUFFIX, "1") + conf.set("spark.invalid.prefix" + FPGA + SPARK_RESOURCE_COUNT_SUFFIX, "1") taskResourceRequirement = conf.getTaskResourceRequirements() assert(taskResourceRequirement.size == 1) - assert(taskResourceRequirement.get("fpga").isEmpty) + assert(taskResourceRequirement.get(FPGA).isEmpty) // Ignore invalid suffix - conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + "invalid.suffix", "1") + conf.set(SPARK_TASK_RESOURCE_PREFIX + FPGA + "invalid.suffix", "1") taskResourceRequirement = conf.getTaskResourceRequirements() assert(taskResourceRequirement.size == 1) - assert(taskResourceRequirement.get("fpga").isEmpty) + assert(taskResourceRequirement.get(FPGA).isEmpty) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 20634969f2483..1107741cd0596 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -746,9 +746,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu """'{"name": "gpu","addresses":["5", "6"]}'""") val conf = new SparkConf() - .set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + + .set(SPARK_DRIVER_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "1") - .set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + + .set(SPARK_DRIVER_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_DISCOVERY_SCRIPT_SUFFIX, scriptPath) .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") @@ -759,8 +759,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.statusTracker.getExecutorInfos.size == 1) } assert(sc.resources.size === 1) - assert(sc.resources.get("gpu").get.addresses === Array("5", "6")) - assert(sc.resources.get("gpu").get.name === "gpu") + assert(sc.resources.get(GPU).get.addresses === Array("5", "6")) + assert(sc.resources.get(GPU).get.name === "gpu") } } @@ -783,9 +783,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val resourcesFile = writeJsonFile(dir, ja) val conf = new SparkConf() - .set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + + .set(SPARK_DRIVER_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "1") - .set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + + .set(SPARK_DRIVER_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_DISCOVERY_SCRIPT_SUFFIX, scriptPath) .set(DRIVER_RESOURCES_FILE, resourcesFile) .setMaster("local-cluster[1, 1, 1024]") @@ -798,14 +798,14 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } // driver gpu addresses config should take precedence over the script assert(sc.resources.size === 1) - assert(sc.resources.get("gpu").get.addresses === Array("0", "1", "8")) - assert(sc.resources.get("gpu").get.name === "gpu") + assert(sc.resources.get(GPU).get.addresses === Array("0", "1", "8")) + assert(sc.resources.get(GPU).get.name === "gpu") } } test("Test parsing resources task configs with missing executor config") { val conf = new SparkConf() - .set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + + .set(SPARK_TASK_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "1") .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") @@ -821,9 +821,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("Test parsing resources executor config < task requirements") { val conf = new SparkConf() - .set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + + .set(SPARK_TASK_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "2") - .set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + .set(SPARK_EXECUTOR_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "1") .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") @@ -839,8 +839,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("Parse resources executor config not the same multiple numbers of the task requirements") { val conf = new SparkConf() - .set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_SUFFIX, "2") - .set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_SUFFIX, "4") + .set(SPARK_TASK_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "2") + .set(SPARK_EXECUTOR_RESOURCE_PREFIX + GPU + SPARK_RESOURCE_COUNT_SUFFIX, "4") .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") @@ -849,8 +849,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu }.getMessage() assert(error.contains("The value of executor resource config: " + - "spark.executor.resource.gpu.count = 4 has to be 1 times the number of the task config: " + - "spark.task.resource.gpu.count = 2")) + "spark.executor.resource.gpu.count = 4 is more than that tasks can take: " + + "1 * spark.task.resource.gpu.count = 2. The resources may be wasted.")) } def mockDiscoveryScript(file: File, result: String): String = { @@ -863,6 +863,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("test resource scheduling under local-cluster mode") { import org.apache.spark.TestUtils._ + assume(!(Utils.isWindows)) withTempDir { dir => val resourceFile = new File(dir, "resourceDiscoverScript") val resources = """'{"name": "gpu", "addresses": ["0", "1", "2"]}'""" @@ -877,7 +878,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu discoveryScript) .setMaster("local-cluster[3, 3, 1024]") .setAppName("test-cluster") - setTaskResourceRequirement(conf, "gpu", 1) + setTaskResourceRequirement(conf, GPU, 1) sc = new SparkContext(conf) // Ensure all executors has started From cd01caec2aa03376348c50746303a41991f93f93 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Mon, 3 Jun 2019 23:16:05 -0700 Subject: [PATCH 16/18] update config check logic and update comments --- .../scala/org/apache/spark/SparkContext.scala | 26 ++++++++++++------- .../scheduler/ExecutorResourceInfo.scala | 26 ++++++++++--------- .../spark/scheduler/TaskSchedulerImpl.scala | 4 +++ .../org/apache/spark/SparkContextSuite.scala | 6 ++--- .../scheduler/ExecutorResourceInfoSuite.scala | 6 +++-- 5 files changed, 41 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 74c0592a7d397..c076ab562cb76 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2727,7 +2727,9 @@ object SparkContext extends Logging { val taskResourcesAndCount = sc.conf.getTaskResourceRequirements() val executorResourcesAndCounts = sc.conf.getAllWithPrefixAndSuffix( SPARK_EXECUTOR_RESOURCE_PREFIX, SPARK_RESOURCE_COUNT_SUFFIX).toMap - val numSlots = (taskResourcesAndCount.map { case (rName, taskCount) => + var numSlots = execCores / taskCores + var limitingResourceName = "CPU" + taskResourcesAndCount.foreach { case (rName, taskCount) => // Make sure the executor resources were specified through config. val execCount = executorResourcesAndCounts.getOrElse(rName, throw new SparkException( @@ -2744,18 +2746,22 @@ object SparkContext extends Logging { s"= $execCount has to be >= the task config: " + s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount") } - execCount.toInt / taskCount - }.toList ++ Seq(execCores / taskCores)).min - // There have been checks inside SparkConf to make sure the executor resources were specified - // and are large enough if any task resources were specified. + // Compare and update the max slots each executor can provide. + val resourceNumSlots = execCount.toInt / taskCount + if (resourceNumSlots < numSlots) { + numSlots = resourceNumSlots + limitingResourceName = rName + } + } + // There have been checks above to make sure the executor resources were specified and are + // large enough if any task resources were specified. taskResourcesAndCount.foreach { case (rName, taskCount) => val execCount = executorResourcesAndCounts(rName) if (execCount.toInt / taskCount.toInt != numSlots) { - val message = s"The value of executor resource config: " + - s"${SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} " + - s"= $execCount is more than that tasks can take: $numSlots * " + - s"${SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_SUFFIX} = $taskCount. " + - s"The resources may be wasted." + val message = s"The configuration of resource: $rName (exec = ${execCount.toInt}, " + + s"task = ${taskCount}) will result in wasted resources due to resource " + + s"${limitingResourceName} limiting the number of runnable tasks per executor to: " + + s"${numSlots}. Please adjust your configuration." if (Utils.isTesting) { throw new SparkException(message) } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index 68f8627206bdd..1c8d5966f3b89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -32,33 +32,34 @@ private[spark] class ExecutorResourceInfo( val name: String, private val addresses: Seq[String]) extends Serializable { - private val addressesMap = new HashMap[String, Boolean]() - addresses.foreach(addressesMap.put(_, true)) + private val addressesAllocatedMap = new HashMap[String, Boolean]() + addresses.foreach(addressesAllocatedMap.put(_, true)) /** * Sequence of currently available resource addresses. */ - def availableAddrs: Seq[String] = addressesMap.toList.filter(_._2 == true).map(_._1) + def availableAddrs: Seq[String] = addressesAllocatedMap.toList.filter(_._2 == true).map(_._1) /** * Sequence of currently assigned resource addresses. * Exposed for testing only. */ private[scheduler] def assignedAddrs: Seq[String] = - addressesMap.toList.filter(_._2 == false).map(_._1) + addressesAllocatedMap.toList.filter(_._2 == false).map(_._1) /** * Acquire a sequence of resource addresses (to a launched task), these addresses must be * available. When the task finishes, it will return the acquired resource addresses. + * Throw an Exception if an address is not available or doesn't exist. */ def acquire(addrs: Seq[String]): Unit = { addrs.foreach { address => - val isAvailable = addressesMap.getOrElse(address, false) + val isAvailable = addressesAllocatedMap.getOrElse(address, false) if (isAvailable) { - addressesMap(address) = false + addressesAllocatedMap(address) = false } else { - throw new SparkException(s"Try to acquire address that is not available. $name address " + - s"$address is not available.") + throw new SparkException("Try to acquire an address that is not available or doesn't " + + s"exist. $name address $address is not available or doesn't exist.") } } } @@ -66,15 +67,16 @@ private[spark] class ExecutorResourceInfo( /** * Release a sequence of resource addresses, these addresses must have been assigned. Resource * addresses are released when a task has finished. + * Throw an Exception if an address is not assigned or doesn't exist. */ def release(addrs: Seq[String]): Unit = { addrs.foreach { address => - val isAssigned = addressesMap.getOrElse(address, true) + val isAssigned = addressesAllocatedMap.getOrElse(address, true) if (!isAssigned) { - addressesMap(address) = true + addressesAllocatedMap(address) = true } else { - throw new SparkException(s"Try to release address that is not assigned. $name address " + - s"$address is not assigned.") + throw new SparkException("Try to release an address that is not assigned or doesn't " + + s"exist. $name address $address is not assigned or doesn't exist.") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 4f10b6ae89135..cf07847190f94 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -351,6 +351,10 @@ private[spark] class TaskSchedulerImpl( availableCpus(i) -= CPUS_PER_TASK assert(availableCpus(i) >= 0) task.resources.foreach { case (rName, rInfo) => + // Remove the first n elements from availableResources addresses, these removed + // addresses are the same as that we allocated in taskSet.resourceOffer() since it's + // synchronized. We don't remove the exact addresses allocated because the current + // approach produces the identical result with less time complexity. availableResources(i).getOrElse(rName, throw new SparkException(s"Try to acquire resource $rName that doesn't exist.")) .remove(0, rInfo.addresses.size) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 1107741cd0596..40ec1b2194fed 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -848,9 +848,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The value of executor resource config: " + - "spark.executor.resource.gpu.count = 4 is more than that tasks can take: " + - "1 * spark.task.resource.gpu.count = 2. The resources may be wasted.")) + assert(error.contains("The configuration of resource: gpu (exec = 4, task = 2) will result " + + "in wasted resources due to resource CPU limiting the number of runnable tasks per " + + "executor to: 1. Please adjust your configuration.")) } def mockDiscoveryScript(file: File, result: String): String = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala index 953fa04f33bf9..f9a7c6935705d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -51,7 +51,8 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val e = intercept[SparkException] { info.acquire(Array("1")) } - assert(e.getMessage.contains("Try to acquire address that is not available.")) + assert(e.getMessage.contains( + "Try to acquire an address that is not available or doesn't exist.")) } test("Don't allow release address that is not assigned") { @@ -64,6 +65,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val e = intercept[SparkException] { info.release(Array("2")) } - assert(e.getMessage.contains("Try to release address that is not assigned.")) + assert(e.getMessage.contains( + "Try to release an address that is not assigned or doesn't exist.")) } } From e539097e714b45d012c394ffa6a32cc65fa8e47d Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 4 Jun 2019 13:35:26 -0700 Subject: [PATCH 17/18] code cleanup and update test cases --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scheduler/ExecutorResourceInfo.scala | 50 +++++++++++++------ .../spark/scheduler/TaskDescription.scala | 20 +++++--- .../spark/scheduler/TaskSetManager.scala | 6 +-- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../scheduler/ExecutorResourceInfoSuite.scala | 28 +++++++++-- 6 files changed, 75 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c076ab562cb76..b00bb9add5c8c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2757,7 +2757,7 @@ object SparkContext extends Logging { // large enough if any task resources were specified. taskResourcesAndCount.foreach { case (rName, taskCount) => val execCount = executorResourcesAndCounts(rName) - if (execCount.toInt / taskCount.toInt != numSlots) { + if (taskCount.toInt * numSlots < execCount.toInt) { val message = s"The configuration of resource: $rName (exec = ${execCount.toInt}, " + s"task = ${taskCount}) will result in wasted resources due to resource " + s"${limitingResourceName} limiting the number of runnable tasks per executor to: " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala index 1c8d5966f3b89..c75931d53b4be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -17,35 +17,45 @@ package org.apache.spark.scheduler -import scala.collection.mutable.HashMap +import scala.collection.mutable import org.apache.spark.SparkException +import org.apache.spark.util.collection.OpenHashMap /** * Class to hold information about a type of Resource on an Executor. This information is managed * by SchedulerBackend, and TaskScheduler shall schedule tasks on idle Executors based on the * information. + * Please note that this class is intended to be used in a single thread. * @param name Resource name * @param addresses Resource addresses provided by the executor */ private[spark] class ExecutorResourceInfo( val name: String, - private val addresses: Seq[String]) extends Serializable { + addresses: Seq[String]) extends Serializable { - private val addressesAllocatedMap = new HashMap[String, Boolean]() - addresses.foreach(addressesAllocatedMap.put(_, true)) + /** + * Map from an address to its availability, the value `true` means the address is available, + * while value `false` means the address is assigned. + * TODO Use [[OpenHashMap]] instead to gain better performance. + */ + private val addressAvailabilityMap = mutable.HashMap(addresses.map(_ -> true): _*) /** * Sequence of currently available resource addresses. */ - def availableAddrs: Seq[String] = addressesAllocatedMap.toList.filter(_._2 == true).map(_._1) + def availableAddrs: Seq[String] = addressAvailabilityMap.flatMap { case (addr, available) => + if (available) Some(addr) else None + }.toSeq /** * Sequence of currently assigned resource addresses. * Exposed for testing only. */ - private[scheduler] def assignedAddrs: Seq[String] = - addressesAllocatedMap.toList.filter(_._2 == false).map(_._1) + private[scheduler] def assignedAddrs: Seq[String] = addressAvailabilityMap + .flatMap { case (addr, available) => + if (!available) Some(addr) else None + }.toSeq /** * Acquire a sequence of resource addresses (to a launched task), these addresses must be @@ -54,12 +64,16 @@ private[spark] class ExecutorResourceInfo( */ def acquire(addrs: Seq[String]): Unit = { addrs.foreach { address => - val isAvailable = addressesAllocatedMap.getOrElse(address, false) + if (!addressAvailabilityMap.contains(address)) { + throw new SparkException(s"Try to acquire an address that doesn't exist. $name address " + + s"$address doesn't exist.") + } + val isAvailable = addressAvailabilityMap(address) if (isAvailable) { - addressesAllocatedMap(address) = false + addressAvailabilityMap(address) = false } else { - throw new SparkException("Try to acquire an address that is not available or doesn't " + - s"exist. $name address $address is not available or doesn't exist.") + throw new SparkException(s"Try to acquire an address that is not available. $name " + + s"address $address is not available.") } } } @@ -71,12 +85,16 @@ private[spark] class ExecutorResourceInfo( */ def release(addrs: Seq[String]): Unit = { addrs.foreach { address => - val isAssigned = addressesAllocatedMap.getOrElse(address, true) - if (!isAssigned) { - addressesAllocatedMap(address) = true + if (!addressAvailabilityMap.contains(address)) { + throw new SparkException(s"Try to release an address that doesn't exist. $name address " + + s"$address doesn't exist.") + } + val isAvailable = addressAvailabilityMap(address) + if (!isAvailable) { + addressAvailabilityMap(address) = true } else { - throw new SparkException("Try to release an address that is not assigned or doesn't " + - s"exist. $name address $address is not assigned or doesn't exist.") + throw new SparkException(s"Try to release an address that is not assigned. $name " + + s"address $address is not assigned.") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index fd0d9f7e43b6e..c29ee0619e5ec 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -65,7 +65,7 @@ private[spark] class TaskDescription( private[spark] object TaskDescription { private def serializeStringLongMap(map: Map[String, Long], dataOut: DataOutputStream): Unit = { dataOut.writeInt(map.size) - for ((key, value) <- map) { + map.foreach { case (key, value) => dataOut.writeUTF(key) dataOut.writeLong(value) } @@ -74,13 +74,11 @@ private[spark] object TaskDescription { private def serializeResources(map: immutable.Map[String, ResourceInformation], dataOut: DataOutputStream): Unit = { dataOut.writeInt(map.size) - for ((key, value) <- map) { + map.foreach { case (key, value) => dataOut.writeUTF(key) dataOut.writeUTF(value.name) dataOut.writeInt(value.addresses.size) - for (identifier <- value.addresses) { - dataOut.writeUTF(identifier) - } + value.addresses.foreach(dataOut.writeUTF(_)) } } @@ -125,8 +123,10 @@ private[spark] object TaskDescription { private def deserializeStringLongMap(dataIn: DataInputStream): HashMap[String, Long] = { val map = new HashMap[String, Long]() val mapSize = dataIn.readInt() - for (i <- 0 until mapSize) { + var i = 0 + while (i < mapSize) { map(dataIn.readUTF()) = dataIn.readLong() + i += 1 } map } @@ -135,15 +135,19 @@ private[spark] object TaskDescription { immutable.Map[String, ResourceInformation] = { val map = new HashMap[String, ResourceInformation]() val mapSize = dataIn.readInt() - for (i <- 0 until mapSize) { + var i = 0 + while (i < mapSize) { val resType = dataIn.readUTF() val name = dataIn.readUTF() val numIdentifier = dataIn.readInt() val identifiers = new ArrayBuffer[String](numIdentifier) - for (j <- 0 until numIdentifier) { + var j = 0 + while (j < numIdentifier) { identifiers += dataIn.readUTF() + j += 1 } map(resType) = new ResourceInformation(name, identifiers.toArray) + i += 1 } map.toMap } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c6ddcf44d5c8b..6f2b982d17936 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.immutable.Map -import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.math.max import scala.util.control.NonFatal @@ -469,7 +469,7 @@ private[spark] class TaskSetManager( execId: String, host: String, maxLocality: TaskLocality.TaskLocality, - availableResources: Map[String, Buffer[String]] = Map.empty) + availableResources: Map[String, Seq[String]] = Map.empty) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -535,7 +535,7 @@ private[spark] class TaskSetManager( s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") val extraResources = sched.resourcesPerTask.map { case (rName, rNum) => - val rAddresses = availableResources.getOrElse(rName, Buffer.empty) + val rAddresses = availableResources.getOrElse(rName, Seq.empty) assert(rAddresses.size >= rNum, s"Required $rNum $rName addresses, but only " + s"${rAddresses.size} available.") // We'll drop the allocated addresses later inside TaskSchedulerImpl. 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 e151e1eaf0b68..1fc321e7e2bd1 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 @@ -146,7 +146,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId) match { case Some(executorInfo) => executorInfo.freeCores += scheduler.CPUS_PER_TASK - for ((k, v) <- resources) { + resources.foreach { case (k, v) => executorInfo.availableResources.get(k).foreach { r => r.release(v.addresses) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala index f9a7c6935705d..b4ff73083a754 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -51,8 +51,18 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val e = intercept[SparkException] { info.acquire(Array("1")) } - assert(e.getMessage.contains( - "Try to acquire an address that is not available or doesn't exist.")) + assert(e.getMessage.contains("Try to acquire an address that is not available.")) + } + + test("Don't allow acquire address that doesn't exist") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + assert(!info.availableAddrs.contains("4")) + // Acquire an address that doesn't exist + val e = intercept[SparkException] { + info.acquire(Array("4")) + } + assert(e.getMessage.contains("Try to acquire an address that doesn't exist.")) } test("Don't allow release address that is not assigned") { @@ -65,7 +75,17 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val e = intercept[SparkException] { info.release(Array("2")) } - assert(e.getMessage.contains( - "Try to release an address that is not assigned or doesn't exist.")) + assert(e.getMessage.contains("Try to release an address that is not assigned.")) + } + + test("Don't allow release address that doesn't exist") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + assert(!info.assignedAddrs.contains("4")) + // Release an address that doesn't exist + val e = intercept[SparkException] { + info.release(Array("4")) + } + assert(e.getMessage.contains("Try to release an address that doesn't exist.")) } } From 82cd1e35bdc7caaddb4122060d8fd6b98893cbb6 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Tue, 4 Jun 2019 14:38:10 -0700 Subject: [PATCH 18/18] remove totalResources from ExecutorInfo --- .../CoarseGrainedSchedulerBackend.scala | 23 +++++++++---------- .../scheduler/cluster/ExecutorData.scala | 9 +++----- .../scheduler/cluster/ExecutorInfo.scala | 11 ++++----- .../local/LocalSchedulerBackend.scala | 2 +- .../CoarseGrainedSchedulerBackendSuite.scala | 4 ---- 5 files changed, 19 insertions(+), 30 deletions(-) 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 1fc321e7e2bd1..9f535889193f1 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 @@ -147,7 +147,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case Some(executorInfo) => executorInfo.freeCores += scheduler.CPUS_PER_TASK resources.foreach { case (k, v) => - executorInfo.availableResources.get(k).foreach { r => + executorInfo.resourcesInfo.get(k).foreach { r => r.release(v.addresses) } } @@ -215,12 +215,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) - val availableResources = resources.map{ case (k, v) => + val resourcesInfo = resources.map{ case (k, v) => (v.name, new ExecutorResourceInfo(v.name, v.addresses))} val data = new ExecutorData(executorRef, executorAddress, hostname, - cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resources, - availableResources) - + cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, + resourcesInfo) // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { @@ -274,9 +273,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case (id, executorData) => new WorkerOffer(id, executorData.executorHost, executorData.freeCores, Some(executorData.executorAddress.hostPort), - executorData.availableResources.map { case (rName, rInfo) => + executorData.resourcesInfo.map { case (rName, rInfo) => (rName, rInfo.availableAddrs.toBuffer) - }.toMap) + }) }.toIndexedSeq scheduler.resourceOffers(workOffers) } @@ -303,9 +302,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val workOffers = IndexedSeq( new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores, Some(executorData.executorAddress.hostPort), - executorData.availableResources.map { case (rName, rInfo) => + executorData.resourcesInfo.map { case (rName, rInfo) => (rName, rInfo.availableAddrs.toBuffer) - }.toMap)) + })) scheduler.resourceOffers(workOffers) } else { Seq.empty @@ -344,8 +343,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // finishes. executorData.freeCores -= scheduler.CPUS_PER_TASK task.resources.foreach { case (rName, rInfo) => - assert(executorData.availableResources.contains(rName)) - executorData.availableResources(rName).acquire(rInfo.addresses) + assert(executorData.resourcesInfo.contains(rName)) + executorData.resourcesInfo(rName).acquire(rInfo.addresses) } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + @@ -549,7 +548,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // this function is for testing only def getExecutorAvailableResources(executorId: String): Map[String, ExecutorResourceInfo] = { - executorDataMap.get(executorId).map(_.availableResources).getOrElse(Map.empty) + executorDataMap.get(executorId).map(_.resourcesInfo).getOrElse(Map.empty) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 9e372e9fac46a..435365d5b6e00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.ResourceInformation import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler.ExecutorResourceInfo @@ -29,8 +28,7 @@ import org.apache.spark.scheduler.ExecutorResourceInfo * @param executorHost The hostname that this executor is running on * @param freeCores The current number of cores available for work on the executor * @param totalCores The total number of cores available to the executor - * @param totalResources The information of all resources on the executor - * @param availableResources The information of the currently available resources on the executor + * @param resourcesInfo The information of the currently available resources on the executor */ private[cluster] class ExecutorData( val executorEndpoint: RpcEndpointRef, @@ -40,6 +38,5 @@ private[cluster] class ExecutorData( override val totalCores: Int, override val logUrlMap: Map[String, String], override val attributes: Map[String, String], - override val totalResources: Map[String, ResourceInformation], - val availableResources: Map[String, ExecutorResourceInfo] -) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, totalResources) + val resourcesInfo: Map[String, ExecutorResourceInfo] +) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 1f6d572613047..3197e06fcd13a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.scheduler.cluster -import org.apache.spark.ResourceInformation import org.apache.spark.annotation.DeveloperApi /** @@ -28,11 +27,10 @@ class ExecutorInfo( val executorHost: String, val totalCores: Int, val logUrlMap: Map[String, String], - val attributes: Map[String, String], - val totalResources: Map[String, ResourceInformation] = Map.empty) { + val attributes: Map[String, String]) { def this(executorHost: String, totalCores: Int, logUrlMap: Map[String, String]) = { - this(executorHost, totalCores, logUrlMap, Map.empty, Map.empty) + this(executorHost, totalCores, logUrlMap, Map.empty) } def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -43,13 +41,12 @@ class ExecutorInfo( executorHost == that.executorHost && totalCores == that.totalCores && logUrlMap == that.logUrlMap && - attributes == that.attributes && - totalResources == that.totalResources + attributes == that.attributes case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores, logUrlMap, attributes, totalResources) + val state = Seq(executorHost, totalCores, logUrlMap, attributes) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index a0e2254e86f96..cbcc5310a59f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -131,7 +131,7 @@ private[spark] class LocalSchedulerBackend( System.currentTimeMillis, executorEndpoint.localExecutorId, new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty, - Map.empty, Map.empty))) + Map.empty))) launcherBackend.setAppId(appId) launcherBackend.setState(SparkAppHandle.State.RUNNING) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index eb23fb1608f18..6b3916b1f5e63 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -206,10 +206,6 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val listener = new SparkListener() { override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { executorAddedCount += 1 - assert(executorAdded.executorInfo.totalResources.get(GPU).nonEmpty) - val totalResources = executorAdded.executorInfo.totalResources(GPU) - assert(totalResources.addresses === Array("0", "1", "3")) - assert(totalResources.name == GPU) } }