From ee71e466f966ff1ac0ee5166f8219bd0d032da7b Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Sun, 15 Oct 2023 09:35:30 +0800 Subject: [PATCH 01/18] Use fraction to do the resource calculation --- .../spark/deploy/master/WorkerInfo.scala | 25 +- .../CoarseGrainedExecutorBackend.scala | 3 +- .../spark/resource/ResourceAllocator.scala | 105 ++-- .../apache/spark/resource/ResourceUtils.scala | 6 +- .../spark/resource/TaskResourceRequest.scala | 6 +- .../scheduler/ExecutorResourceInfo.scala | 32 +- .../scheduler/ExecutorResourcesAmounts.scala | 199 +++++++ .../spark/scheduler/TaskDescription.scala | 45 +- .../spark/scheduler/TaskSchedulerImpl.scala | 64 +- .../spark/scheduler/TaskSetManager.scala | 17 +- .../apache/spark/scheduler/WorkerOffer.scala | 4 +- .../cluster/CoarseGrainedClusterMessage.scala | 10 +- .../CoarseGrainedSchedulerBackend.scala | 32 +- .../org/apache/spark/SparkConfSuite.scala | 2 +- .../CoarseGrainedExecutorBackendSuite.scala | 15 +- .../apache/spark/executor/ExecutorSuite.scala | 1 + .../spark/resource/ResourceProfileSuite.scala | 9 +- .../CoarseGrainedSchedulerBackendSuite.scala | 18 +- .../scheduler/ExecutorResourceInfoSuite.scala | 123 +++- .../ExecutorResourcesAmountsSuite.scala | 545 ++++++++++++++++++ .../scheduler/TaskDescriptionSuite.scala | 4 + .../scheduler/TaskSchedulerImplSuite.scala | 469 +++++++++++++-- 22 files changed, 1511 insertions(+), 223 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 0137e2be7472..7408e227746d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import scala.collection.mutable import org.apache.spark.resource.{ResourceAllocator, ResourceInformation, ResourceRequirement} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -28,12 +29,24 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String override protected def resourceName = this.name override protected def resourceAddresses = this.addresses - override protected def slotsPerAddress: Int = 1 + /** + * Acquire the resources. + * @param amount How many addresses are requesting. + * @return ResourceInformation + */ def acquire(amount: Int): ResourceInformation = { - val allocated = availableAddrs.take(amount) - acquire(allocated) - new ResourceInformation(resourceName, allocated.toArray) + + var count = amount + val allocated: mutable.HashMap[String, Double] = mutable.HashMap.empty + for (address <- availableAddrs if count > 0) { + if (addressAvailabilityMap(address) == RESOURCE_TOTAL_AMOUNT) { + allocated(address) = 1.0 + count -= 1 + } + } + acquire(allocated.toMap) + new ResourceInformation(resourceName, allocated.keys.toArray) } } @@ -162,7 +175,7 @@ private[spark] class WorkerInfo( */ def recoverResources(expected: Map[String, ResourceInformation]): Unit = { expected.foreach { case (rName, rInfo) => - resources(rName).acquire(rInfo.addresses) + resources(rName).acquire(rInfo.addresses.map(addr => addr -> 1.0).toMap) } } @@ -172,7 +185,7 @@ private[spark] class WorkerInfo( */ private def releaseResources(allocated: Map[String, ResourceInformation]): Unit = { allocated.foreach { case (rName, rInfo) => - resources(rName).release(rInfo.addresses) + resources(rName).release(rInfo.addresses.map(addrs => addrs -> 1.0).toMap) } } } 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 f964e2b50b57..44cce1c28ce9 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -273,7 +273,8 @@ private[spark] class CoarseGrainedExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = { val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation]) val cpus = executor.runningTasks.get(taskId).taskDescription.cpus - val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources) + val resourcesAmounts = executor.runningTasks.get(taskId).taskDescription.resourcesAmounts + val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources, resourcesAmounts) if (TaskState.isFinished(state)) { taskResources.remove(taskId) } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 7b97d9704282..454e6a057095 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -20,6 +20,42 @@ package org.apache.spark.resource import scala.collection.mutable import org.apache.spark.SparkException +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT + +private[spark] object ResourceAmountUtils { + /** + * Using "double" to do the resource calculation may encounter a problem of precision loss. Eg + * + * scala> val taskAmount = 1.0 / 9 + * taskAmount: Double = 0.1111111111111111 + * + * scala> var total = 1.0 + * total: Double = 1.0 + * + * scala> for (i <- 1 to 9 ) { + * | if (total >= taskAmount) { + * | total -= taskAmount + * | println(s"assign $taskAmount for task $i, total left: $total") + * | } else { + * | println(s"ERROR Can't assign $taskAmount for task $i, total left: $total") + * | } + * | } + * assign 0.1111111111111111 for task 1, total left: 0.8888888888888888 + * assign 0.1111111111111111 for task 2, total left: 0.7777777777777777 + * assign 0.1111111111111111 for task 3, total left: 0.6666666666666665 + * assign 0.1111111111111111 for task 4, total left: 0.5555555555555554 + * assign 0.1111111111111111 for task 5, total left: 0.44444444444444425 + * assign 0.1111111111111111 for task 6, total left: 0.33333333333333315 + * assign 0.1111111111111111 for task 7, total left: 0.22222222222222204 + * assign 0.1111111111111111 for task 8, total left: 0.11111111111111094 + * ERROR Can't assign 0.1111111111111111 for task 9, total left: 0.11111111111111094 + * + * So we multiply RESOURCE_TOTAL_AMOUNT to convert the double to long to avoid this limitation. + * Double can display up to 16 decimal places, so we set the factor to + * 10, 000, 000, 000, 000, 000L. + */ + final val RESOURCE_TOTAL_AMOUNT: Long = 10000000000000000L +} /** * Trait used to help executor/worker allocate resources. @@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator { protected def resourceName: String protected def resourceAddresses: Seq[String] - protected def slotsPerAddress: Int /** - * Map from an address to its availability, a value > 0 means the address is available, - * while value of 0 means the address is fully assigned. - * - * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value - * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]] - * times. + * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means + * the address is available, while value of 0 means the address is fully assigned. */ - private lazy val addressAvailabilityMap = { - mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*) + protected lazy val addressAvailabilityMap = { + mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*) } /** - * Sequence of currently available resource addresses. - * - * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses - * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like - * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one. + * Sequence of currently available resource addresses which is not fully assigned. */ def availableAddrs: Seq[String] = addressAvailabilityMap - .flatMap { case (addr, available) => - (0 until available).map(_ => addr) - }.toSeq.sorted + .filter(addresses => addresses._2 > 0).keys.toSeq.sorted /** * Sequence of currently assigned resource addresses. - * - * With [[slotsPerAddress]] greater than 1, [[assignedAddrs]] can contain duplicate addresses - * e.g. with [[slotsPerAddress]] == 2, assignedAddrs for addresses 0 and 1 can look like - * Seq("0", "1", "1"), where address 0 was assigned once, and 1 was assigned twice. */ private[spark] def assignedAddrs: Seq[String] = addressAvailabilityMap - .flatMap { case (addr, available) => - (0 until slotsPerAddress - available).map(_ => addr) - }.toSeq.sorted + .filter(addresses => addresses._2 < RESOURCE_TOTAL_AMOUNT).keys.toSeq.sorted /** * 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 = addressAvailabilityMap.getOrElse(address, + def acquire(addressesAmounts: Map[String, Double]): Unit = { + addressesAmounts.foreach { case (address, amount) => + val prevAmount = addressAvailabilityMap.getOrElse(address, throw new SparkException(s"Try to acquire an address that doesn't exist. $resourceName " + - s"address $address doesn't exist.")) - if (isAvailable > 0) { - addressAvailabilityMap(address) -= 1 + s"address $address doesn't exist.")) + + val internalLeft = addressAvailabilityMap(address) - (amount * RESOURCE_TOTAL_AMOUNT).toLong + + if (internalLeft < 0) { + throw new SparkException(s"Try to acquire $resourceName address $address " + + s"amount: $amount, but only ${prevAmount.toDouble / RESOURCE_TOTAL_AMOUNT} left.") } else { - throw new SparkException("Try to acquire an address that is not available. " + - s"$resourceName address $address is not available.") + addressAvailabilityMap(address) = internalLeft } } } @@ -91,16 +113,21 @@ private[spark] trait ResourceAllocator { * 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 isAvailable = addressAvailabilityMap.getOrElse(address, + def release (addressesAmounts: Map[String, Double]): Unit = { + addressesAmounts.foreach { case (address, amount) => + val prevAmount = addressAvailabilityMap.getOrElse(address, throw new SparkException(s"Try to release an address that doesn't exist. $resourceName " + s"address $address doesn't exist.")) - if (isAvailable < slotsPerAddress) { - addressAvailabilityMap(address) += 1 + + val internalTotal = prevAmount + (amount * RESOURCE_TOTAL_AMOUNT).toLong + + if (internalTotal > RESOURCE_TOTAL_AMOUNT) { + throw new SparkException(s"Try to release $resourceName address $address " + + s"amount: $amount. But the total amount: " + + s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " + + s"after release should be <= 1") } else { - throw new SparkException(s"Try to release an address that is not assigned. $resourceName " + - s"address $address is not assigned.") + addressAvailabilityMap(address) = internalTotal } } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index d19f413598b5..04acf94c75cb 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -170,16 +170,16 @@ private[spark] object ResourceUtils extends Logging { // integer amount and the number of slots per address. For instance, if the amount is 0.5, // the we get (1, 2) back out. This indicates that for each 1 address, it has 2 slots per // address, which allows you to put 2 tasks on that address. Note if amount is greater - // than 1, then the number of slots per address has to be 1. This would indicate that a + // than 1, then the number of parts per address has to be 1. This would indicate that a // would have multiple addresses assigned per task. This can be used for calculating // the number of tasks per executor -> (executorAmount * numParts) / (integer amount). // Returns tuple of (integer amount, numParts) def calculateAmountAndPartsForFraction(doubleAmount: Double): (Int, Int) = { - val parts = if (doubleAmount <= 0.5) { + val parts = if (doubleAmount <= 1.0) { Math.floor(1.0 / doubleAmount).toInt } else if (doubleAmount % 1 != 0) { throw new SparkException( - s"The resource amount ${doubleAmount} must be either <= 0.5, or a whole number.") + s"The resource amount ${doubleAmount} must be either <= 1.0, or a whole number.") } else { 1 } diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala index cbd57808213a..9fc0d93373b5 100644 --- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.{Evolving, Since} * * @param resourceName Resource name * @param amount Amount requesting as a Double to support fractional resource requests. - * Valid values are less than or equal to 0.5 or whole numbers. This essentially + * Valid values are less than or equal to 1.0 or whole numbers. This essentially * lets you configure X number of tasks to run on a single resource, * ie amount equals 0.5 translates into 2 tasks per resource address. */ @@ -37,8 +37,8 @@ import org.apache.spark.annotation.{Evolving, Since} class TaskResourceRequest(val resourceName: String, val amount: Double) extends Serializable { - assert(amount <= 0.5 || amount % 1 == 0, - s"The resource amount ${amount} must be either <= 0.5, or a whole number.") + assert(amount <= 1.0 || amount % 1 == 0, + s"The resource amount ${amount} must be either <= 1.0, or a whole number.") override def equals(obj: Any): Boolean = { obj match { 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 508c6cebd9fe..dde701daf6b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.resource.{ResourceAllocator, ResourceInformation} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT /** * Class to hold information about a type of Resource on an Executor. This information is managed @@ -25,16 +26,37 @@ import org.apache.spark.resource.{ResourceAllocator, ResourceInformation} * information. * @param name Resource name * @param addresses Resource addresses provided by the executor - * @param numParts Number of ways each resource is subdivided when scheduling tasks */ private[spark] class ExecutorResourceInfo( name: String, - addresses: Seq[String], - numParts: Int) + addresses: Seq[String]) extends ResourceInformation(name, addresses.toArray) with ResourceAllocator { override protected def resourceName = this.name + override protected def resourceAddresses = this.addresses - override protected def slotsPerAddress: Int = numParts - def totalAddressAmount: Int = resourceAddresses.length * slotsPerAddress + + /** + * Calculate how many parts the executor can offer according to the task resource amount + * @param taskAmount how many resource amount the task required + * @return the total parts + */ + def totalParts(taskAmount: Double): Int = { + assert(taskAmount > 0.0) + if (taskAmount >= 1.0) { + addresses.length / taskAmount.ceil.toInt + } else { + addresses.length * Math.floor(1.0 / taskAmount).toInt + } + } + + /** + * Convert the internal address availability to the public resource format + * @return the resources amounts + */ + def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map { + case (address, internalAmount) => + address -> (internalAmount.toDouble / RESOURCE_TOTAL_AMOUNT) + }.toMap + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala new file mode 100644 index 000000000000..bbc9d19334b6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -0,0 +1,199 @@ +/* + * 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.HashMap + +import org.apache.spark.SparkException +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT + +/** + * Class to hold information about a series of resources belonging to an executor. + * A resource could be a GPU, FPGA, etc. And it is used as a temporary + * class to calculate the resources amounts when offering resources to + * the tasks in the [[TaskSchedulerImpl]] + * + * One example is GPUs, where the addresses would be the indices of the GPUs + * + * @param resources The executor available resources and amount. eg, + * Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0), + * "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9) + * ) + */ +private[spark] class ExecutorResourcesAmounts( + private val resources: Map[String, Map[String, Double]]) extends Serializable { + + resources.foreach { case (_, addressMount) => + addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}} + + // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly. + // and convert the addressesAmounts to be mutable.HashMap + private val internalResources: Map[String, HashMap[String, Long]] = { + resources.map { case (rName, addressAmounts) => + rName -> HashMap(addressAmounts.map { case (address, amount) => + address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong + }.toSeq: _*) + } + } + + // It maps from the resource name to its amount. + lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) => + rName -> addressMap.size + } + + // convert internal resources back to the public. + def availableResources: Map[String, Map[String, Double]] = { + internalResources.map { case (rName, addressMap) => + rName -> addressMap.map { case (address, amount) => + address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT + }.toMap + } + } + + // Acquire the resource and update the resource + def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = { + assignedResource.foreach { case (rName, taskResAmounts) => + val availableResourceAmounts = internalResources.getOrElse(rName, + throw new SparkException(s"Try to acquire an address from $rName that doesn't exist")) + taskResAmounts.foreach { case (address, amount) => + val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address, + throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " + + s"address $address doesn't exist.")) + + val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong + val internalLeft = prevInternalTotalAmount - internalTaskAmount + val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT + if (realLeft < 0) { + throw new SparkException(s"The total amount ${realLeft} " + + s"after acquiring $rName address $address should be >= 0") + } + internalResources(rName)(address) = internalLeft + // scalastyle:off println + println(s"Acquired. left ${realLeft}") + // scalastyle:on println + } + } + } + + // release the resources and update the values + def release(assignedResource: Map[String, Map[String, Double]]): Unit = { + assignedResource.foreach { case (rName, taskResAmounts) => + val availableResourceAmounts = internalResources.getOrElse(rName, + throw new SparkException(s"Try to release an address from $rName that doesn't exist")) + taskResAmounts.foreach { case (address, amount) => + val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address, + throw new SparkException(s"Try to release an address that is not assigned. $rName " + + s"address $address is not assigned.")) + val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong + val internalTotal = prevInternalTotalAmount + internalTaskAmount + if (internalTotal > RESOURCE_TOTAL_AMOUNT) { + throw new SparkException(s"The total amount " + + s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " + + s"after releasing $rName address $address should be <= 1.0") + } + internalResources(rName)(address) = internalTotal + // scalastyle:off println + println(s"Released. amount ${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT}") + // scalastyle:on println + } + } + } + + // Try to assign the address according to the task requirement. + // Please note that this function will not update the values. + def assignResources(taskSetProf: ResourceProfile): + Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = { + + // only look at the resource other than cpus + val tsResources = taskSetProf.getCustomTaskResources() + if (tsResources.isEmpty) { + return Some(Map.empty, Map.empty) + } + + val localTaskReqAssign = HashMap[String, ResourceInformation]() + val allocatedAddresses = HashMap[String, Map[String, Double]]() + + // we go through all resources here so that we can make sure they match and also get what the + // assignments are for the next task + for ((rName, taskReqs) <- tsResources) { + // if taskAmount = 1.5, we assign 2.0 gpu for user or + // just throw an exception in a very begging? + // TODO, just remove it, since we enabled the check at the very beginning. + val taskAmount = if (taskReqs.amount < 1.0) taskReqs.amount else Math.ceil(taskReqs.amount) + + internalResources.get(rName) match { + case Some(addressesAmountMap) => + + var internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong + val allocatedAddressesMap = HashMap[String, Double]() + + // always sort the addresses + val addresses = addressesAmountMap.keys.toSeq.sorted + + for (address <- addresses if internalTaskAmount > 0) { + val freeAmount = addressesAmountMap(address) + + // The address is still a whole resource + if (freeAmount == RESOURCE_TOTAL_AMOUNT) { + // Try to assign this whole address first + if (internalTaskAmount >= RESOURCE_TOTAL_AMOUNT) { + internalTaskAmount -= RESOURCE_TOTAL_AMOUNT + // Assign the whole resource of the address + allocatedAddressesMap(address) = 1.0 + } else { + // Assign the part of the address. + allocatedAddressesMap(address) = taskAmount + internalTaskAmount = 0 + } + } else { + // The address is fraction + if (freeAmount - internalTaskAmount >= 0) { + allocatedAddressesMap(address) = taskAmount + internalTaskAmount = 0 + } + } + } + + if (internalTaskAmount == 0 && allocatedAddressesMap.size > 0) { + localTaskReqAssign.put(rName, new ResourceInformation(rName, + allocatedAddressesMap.keys.toArray)) + allocatedAddresses.put(rName, allocatedAddressesMap.toMap) + } else return None + + case None => return None + } + } + Some(localTaskReqAssign.toMap, allocatedAddresses.toMap) + } + +} + +private[spark] object ExecutorResourcesAmounts { + + // Create an empty ExecutorResourcesAmounts + def empty: ExecutorResourcesAmounts = new ExecutorResourcesAmounts(Map.empty) + + // Converts executor infos to ExecutorResourcesAmounts + def apply(executorInfos: Map[String, ExecutorResourceInfo]): ExecutorResourcesAmounts = { + new ExecutorResourcesAmounts( + executorInfos.map { case (rName, rInfo) => rName -> rInfo.resourcesAmounts } + ) + } + +} 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 6e6507782a49..f0cd1e32062b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -58,6 +58,7 @@ private[spark] class TaskDescription( val properties: Properties, val cpus: Int, val resources: immutable.Map[String, ResourceInformation], + val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]], val serializedTask: ByteBuffer) { assert(cpus > 0, "CPUs per task should be > 0") @@ -85,6 +86,20 @@ private[spark] object TaskDescription { } } + + private def serializeResourcesAmounts(map: immutable.Map[String, immutable.Map[String, Double]], + dataOut: DataOutputStream): Unit = { + dataOut.writeInt(map.size) + map.foreach { case (rName, addressAmountMap) => + dataOut.writeUTF(rName) + dataOut.writeInt(addressAmountMap.size) + addressAmountMap.foreach { case (address, amount) => + dataOut.writeUTF(address) + dataOut.writeDouble(amount) + } + } + } + def encode(taskDescription: TaskDescription): ByteBuffer = { val bytesOut = new ByteBufferOutputStream(4096) val dataOut = new DataOutputStream(bytesOut) @@ -115,6 +130,9 @@ private[spark] object TaskDescription { // Write resources. serializeResources(taskDescription.resources, dataOut) + // Write resourcesAmounts. + serializeResourcesAmounts(taskDescription.resourcesAmounts, dataOut) + // Write the task. The task is already serialized, so write it directly to the byte buffer. Utils.writeByteBuffer(taskDescription.serializedTask, bytesOut) @@ -192,6 +210,28 @@ private[spark] object TaskDescription { map.toMap } + private def deserializeResourcesAmounts(dataIn: DataInputStream): + immutable.Map[String, immutable.Map[String, Double]] = { + val map = new HashMap[String, immutable.Map[String, Double]]() + val mapSize = dataIn.readInt() + var i = 0 + while (i < mapSize) { + val resType = dataIn.readUTF() + val addressAmountMap = new HashMap[String, Double]() + val addressAmountSize = dataIn.readInt() + var j = 0 + while (j < addressAmountSize) { + val address = dataIn.readUTF() + val amount = dataIn.readDouble() + addressAmountMap(address) = amount + j += 1 + } + map.put(resType, addressAmountMap.toMap) + i += 1 + } + map.toMap + } + def decode(byteBuffer: ByteBuffer): TaskDescription = { val dataIn = new DataInputStream(new ByteBufferInputStream(byteBuffer)) val taskId = dataIn.readLong() @@ -221,10 +261,13 @@ private[spark] object TaskDescription { // Read resources. val resources = deserializeResources(dataIn) + // Read resources. + val resourcesAmounts = deserializeResourcesAmounts(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, artifacts, - properties, cpus, resources, serializedTask) + properties, cpus, resources, resourcesAmounts, 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 41f6b3ad64bf..8bbb687e27b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -23,7 +23,7 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random import com.google.common.cache.CacheBuilder @@ -389,7 +389,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], - availableResources: Array[Map[String, Buffer[String]]], + availableResources: Array[ExecutorResourcesAmounts], tasks: IndexedSeq[ArrayBuffer[TaskDescription]]) : (Boolean, Option[TaskLocality]) = { var noDelayScheduleRejects = true @@ -406,36 +406,31 @@ private[spark] class TaskSchedulerImpl( .canBeScheduled(taskSetRpID, shuffledOffers(i).resourceProfileId)) { val taskResAssignmentsOpt = resourcesMeetTaskRequirements(taskSet, availableCpus(i), availableResources(i)) - taskResAssignmentsOpt.foreach { taskResAssignments => + taskResAssignmentsOpt.foreach { case (taskResAssignments, taskResAmounts) => try { val prof = sc.resourceProfileManager.resourceProfileFromId(taskSetRpID) val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(prof, conf) val (taskDescOption, didReject, index) = - taskSet.resourceOffer(execId, host, maxLocality, taskCpus, taskResAssignments) + taskSet.resourceOffer(execId, host, maxLocality, taskCpus, taskResAssignments, + taskResAmounts) noDelayScheduleRejects &= !didReject for (task <- taskDescOption) { - val (locality, resources) = if (task != null) { + val (locality, resourcesAmounts) = if (task != null) { tasks(i) += task addRunningTask(task.taskId, execId, taskSet) - (taskSet.taskInfos(task.taskId).taskLocality, task.resources) + (taskSet.taskInfos(task.taskId).taskLocality, task.resourcesAmounts) } else { assert(taskSet.isBarrier, "TaskDescription can only be null for barrier task") val barrierTask = taskSet.barrierPendingLaunchTasks(index) barrierTask.assignedOfferIndex = i barrierTask.assignedCores = taskCpus - (barrierTask.taskLocality, barrierTask.assignedResources) + (barrierTask.taskLocality, barrierTask.assignedResourcesAmount) } minLaunchedLocality = minTaskLocality(minLaunchedLocality, Some(locality)) availableCpus(i) -= taskCpus assert(availableCpus(i) >= 0) - resources.foreach { case (rName, rInfo) => - // Remove the first n elements from availableResources addresses, these removed - // addresses are the same as that we allocated in taskResourceAssignments 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)(rName).remove(0, rInfo.addresses.size) - } + availableResources(i).acquire(resourcesAmounts) } } catch { case e: TaskNotSerializableException => @@ -468,33 +463,15 @@ private[spark] class TaskSchedulerImpl( private def resourcesMeetTaskRequirements( taskSet: TaskSetManager, availCpus: Int, - availWorkerResources: Map[String, Buffer[String]] - ): Option[Map[String, ResourceInformation]] = { + availWorkerResources: ExecutorResourcesAmounts): Option[(Map[String, ResourceInformation], + Map[String, Map[String, Double]])] = { val rpId = taskSet.taskSet.resourceProfileId val taskSetProf = sc.resourceProfileManager.resourceProfileFromId(rpId) val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(taskSetProf, conf) // check if the ResourceProfile has cpus first since that is common case if (availCpus < taskCpus) return None // only look at the resource other than cpus - val tsResources = taskSetProf.getCustomTaskResources() - if (tsResources.isEmpty) return Some(Map.empty) - val localTaskReqAssign = HashMap[String, ResourceInformation]() - // we go through all resources here so that we can make sure they match and also get what the - // assignments are for the next task - for ((rName, taskReqs) <- tsResources) { - val taskAmount = taskSetProf.getSchedulerTaskResourceAmount(rName) - availWorkerResources.get(rName) match { - case Some(workerRes) => - if (workerRes.size >= taskAmount) { - localTaskReqAssign.put(rName, new ResourceInformation(rName, - workerRes.take(taskAmount).toArray)) - } else { - return None - } - case None => return None - } - } - Some(localTaskReqAssign.toMap) + availWorkerResources.assignResources(taskSetProf) } private def minTaskLocality( @@ -576,13 +553,8 @@ private[spark] class TaskSchedulerImpl( // value is -1 val numBarrierSlotsAvailable = if (taskSet.isBarrier) { val rpId = taskSet.taskSet.resourceProfileId - val availableResourcesAmount = availableResources.map { resourceMap => - // available addresses already takes into account if there are fractional - // task resource requests - resourceMap.map { case (name, addresses) => (name, addresses.length) } - } - calculateAvailableSlots(this, conf, rpId, resourceProfileIds, availableCpus, - availableResourcesAmount) + val resAmounts = availableResources.map(_.resourceAmount) + calculateAvailableSlots(this, conf, rpId, resourceProfileIds, availableCpus, resAmounts) } else { -1 } @@ -715,9 +687,8 @@ private[spark] class TaskSchedulerImpl( barrierPendingLaunchTasks.foreach { task => // revert all assigned resources availableCpus(task.assignedOfferIndex) += task.assignedCores - task.assignedResources.foreach { case (rName, rInfo) => - availableResources(task.assignedOfferIndex)(rName).appendAll(rInfo.addresses) - } + availableResources(task.assignedOfferIndex).release( + task.assignedResourcesAmount) // re-add the task to the schedule pending list taskSet.addPendingTask(task.index) } @@ -735,7 +706,8 @@ private[spark] class TaskSchedulerImpl( false, task.assignedCores, task.assignedResources, - launchTime) + launchTime, + task.assignedResourcesAmount) addRunningTask(taskDesc.taskId, taskDesc.executorId, taskSet) tasks(task.assignedOfferIndex) += taskDesc shuffledOffers(task.assignedOfferIndex).address.get -> taskDesc 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 6157a3e46c87..31bd10759717 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -21,7 +21,6 @@ import java.io.NotSerializableException import java.nio.ByteBuffer import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, TimeUnit} -import scala.collection.immutable.Map import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.math.max import scala.util.control.NonFatal @@ -444,7 +443,8 @@ private[spark] class TaskSetManager( host: String, maxLocality: TaskLocality.TaskLocality, taskCpus: Int = sched.CPUS_PER_TASK, - taskResourceAssignments: Map[String, ResourceInformation] = Map.empty) + taskResourceAssignments: Map[String, ResourceInformation] = Map.empty, + taskResourceAmounts: Map[String, Map[String, Double]] = Map.empty) : (Option[TaskDescription], Boolean, Int) = { val offerExcluded = taskSetExcludelistHelperOpt.exists { excludeList => @@ -479,7 +479,8 @@ private[spark] class TaskSetManager( host, index, taskLocality, - taskResourceAssignments) + taskResourceAssignments, + taskResourceAmounts) // return null since the TaskDescription for the barrier task is not ready yet null } else { @@ -491,7 +492,8 @@ private[spark] class TaskSetManager( speculative, taskCpus, taskResourceAssignments, - curTime) + curTime, + taskResourceAmounts) } } val hasPendingTasks = pendingTasks.all.nonEmpty || pendingSpeculatableTasks.all.nonEmpty @@ -513,7 +515,8 @@ private[spark] class TaskSetManager( speculative: Boolean, taskCpus: Int, taskResourceAssignments: Map[String, ResourceInformation], - launchTime: Long): TaskDescription = { + launchTime: Long, + resourcesAmounts: Map[String, Map[String, Double]]): TaskDescription = { // Found a task; do some bookkeeping and return a task description val task = tasks(index) val taskId = sched.newTaskId() @@ -567,6 +570,7 @@ private[spark] class TaskSetManager( task.localProperties, taskCpus, taskResourceAssignments, + resourcesAmounts, serializedTask) } @@ -1380,7 +1384,8 @@ private[scheduler] case class BarrierPendingLaunchTask( host: String, index: Int, taskLocality: TaskLocality.TaskLocality, - assignedResources: Map[String, ResourceInformation]) { + assignedResources: Map[String, ResourceInformation], + assignedResourcesAmount: Map[String, Map[String, Double]]) { // Stored the corresponding index of the WorkerOffer which is responsible to launch the task. // Used to revert the assigned resources (e.g., cores, custome resources) when the barrier // task set doesn't launch successfully in a single resourceOffers round. 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 92a12f13576c..a7d63a8949e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import scala.collection.mutable.Buffer - import org.apache.spark.resource.ResourceProfile /** @@ -32,5 +30,5 @@ 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, Buffer[String]] = Map.empty, + resources: ExecutorResourcesAmounts = ExecutorResourcesAmounts.empty, resourceProfileId: Int = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) 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 b49f5269169d..eeeeba94439c 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 @@ -25,6 +25,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.{ExecutorLossReason, MiscellaneousProcessDetails} import org.apache.spark.util.SerializableBuffer + private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { @@ -80,7 +81,8 @@ private[spark] object CoarseGrainedClusterMessages { state: TaskState, data: SerializableBuffer, taskCpus: Int, - resources: Map[String, ResourceInformation] = Map.empty) + resources: Map[String, ResourceInformation] = Map.empty, + resourcesAmounts: Map[String, Map[String, Double]] = Map.empty) extends CoarseGrainedClusterMessage object StatusUpdate { @@ -91,8 +93,10 @@ private[spark] object CoarseGrainedClusterMessages { state: TaskState, data: ByteBuffer, taskCpus: Int, - resources: Map[String, ResourceInformation]): StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), taskCpus, resources) + resources: Map[String, ResourceInformation], + resourcesAmounts: Map[String, Map[String, Double]]): StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), taskCpus, resources, + resourcesAmounts) } } 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 c49b2411e763..39a7e3cdbfd2 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 @@ -165,15 +165,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } override def receive: PartialFunction[Any, Unit] = { - case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) => + case StatusUpdate(executorId, taskId, state, data, taskCpus, resources, resourcesAmounts) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { executorDataMap.get(executorId) match { case Some(executorInfo) => executorInfo.freeCores += taskCpus - resources.foreach { case (k, v) => - executorInfo.resourcesInfo.get(k).foreach { r => - r.release(v.addresses) + resourcesAmounts.foreach { case (rName, addressAmount) => + addressAmount.foreach { case (address, amount) => + executorInfo.resourcesInfo.get(rName).foreach { r => + r.release(Map(address -> amount)) + } } } makeOffers(executorId) @@ -270,11 +272,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) val resourcesInfo = resources.map { case (rName, info) => - // tell the executor it can schedule resources up to numSlotsPerAddress times, - // as configured by the user, or set to 1 as that is the default (1 task/resource) - val numParts = scheduler.sc.resourceProfileManager - .resourceProfileFromId(resourceProfileId).getNumSlotsPerAddress(rName, conf) - (info.name, new ExecutorResourceInfo(info.name, info.addresses, numParts)) + (info.name, new ExecutorResourceInfo(info.name, info.addresses)) } // If we've requested the executor figure out when we did. val reqTs: Option[Long] = CoarseGrainedSchedulerBackend.this.synchronized { @@ -383,9 +381,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } private def buildWorkerOffer(executorId: String, executorData: ExecutorData) = { - val resources = executorData.resourcesInfo.map { case (rName, rInfo) => - (rName, rInfo.availableAddrs.toBuffer) - } + val resources = ExecutorResourcesAmounts(executorData.resourcesInfo) WorkerOffer( executorId, executorData.executorHost, @@ -444,9 +440,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Do resources allocation here. The allocated resources will get released after the task // finishes. executorData.freeCores -= task.cpus - task.resources.foreach { case (rName, rInfo) => - assert(executorData.resourcesInfo.contains(rName)) - executorData.resourcesInfo(rName).acquire(rInfo.addresses) + task.resourcesAmounts.foreach { case (rName, addressAmounts) => + addressAmounts.foreach { case (address, amount) => + executorData.resourcesInfo(rName).acquire(Map(address -> amount)) + } } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + @@ -763,7 +760,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp ( executor.resourceProfileId, executor.totalCores, - executor.resourcesInfo.map { case (name, rInfo) => (name, rInfo.totalAddressAmount) } + executor.resourcesInfo.map { case (name, rInfo) => + val taskAmount = rp.taskResources.get(name).get.amount + (name, rInfo.totalParts(taskAmount)) + } ) }.unzip3 } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 5f7958167507..8eb8d9c6dc85 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -465,7 +465,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst case (ratio, slots) => val conf = new SparkConf() conf.set(TASK_GPU_ID.amountConf, ratio.toString) - if (ratio > 0.5 && ratio % 1 != 0) { + if (ratio > 1.0 && ratio % 1 != 0) { assertThrows[SparkException] { parseResourceRequirements(conf, SPARK_TASK_PREFIX) } 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 3ef4da6d3d3f..9a22c098baf6 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -307,11 +307,12 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(backend.taskResources.isEmpty) val taskId = 1000000L + val resourcesAmounts = Map(GPU -> Map("0" -> 0.15, "1" -> 0.76)) // 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, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, - Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) + Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), resourcesAmounts, data) val serializedTaskDescription = TaskDescription.encode(taskDescription) backend.rpcEnv.setupEndpoint("Executor 1", backend) backend.executor = mock[Executor](CALLS_REAL_METHODS) @@ -345,6 +346,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(backend.taskResources.size == 1) val resources = backend.taskResources.get(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) + assert(executor.runningTasks.get(taskId).taskDescription.resourcesAmounts + === resourcesAmounts) } // Update the status of a running task shall not affect `taskResources` map. @@ -352,6 +355,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(backend.taskResources.size == 1) val resources = backend.taskResources.get(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) + assert(executor.runningTasks.get(taskId).taskDescription.resourcesAmounts + === resourcesAmounts) // Update the status of a finished task shall remove the entry from `taskResources` map. backend.statusUpdate(taskId, TaskState.FINISHED, data) @@ -423,11 +428,13 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val tasksKilled = new TrieMap[Long, Boolean]() val tasksExecuted = new TrieMap[Long, Boolean]() + val resourcesAmounts = Map(GPU -> Map("0" -> 0.15, "1" -> 0.76)) + // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { taskId => new TaskDescription(taskId, 2, "1", s"TASK $taskId", 19, 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, - Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) + Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), resourcesAmounts, data) } assert(taskDescriptions.length == numTasks) @@ -512,11 +519,13 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val tasksKilled = new TrieMap[Long, Boolean]() val tasksExecuted = new TrieMap[Long, Boolean]() + val resourcesAmounts = Map(GPU -> Map("0" -> 0.15, "1" -> 0.76)) + // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { taskId => new TaskDescription(taskId, 2, "1", s"TASK $taskId", 19, 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, - Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) + Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), resourcesAmounts, data) } assert(taskDescriptions.length == numTasks) 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 49e19dd2a00e..a5bfb679086b 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -643,6 +643,7 @@ class ExecutorSuite extends SparkFunSuite properties = new Properties, cpus = 1, resources = immutable.Map[String, ResourceInformation](), + Map.empty, serializedTask) } diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index fd7018f189e2..f549b5f4dcae 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -357,12 +357,11 @@ class ResourceProfileSuite extends SparkFunSuite with MockitoSugar { var taskError = intercept[AssertionError] { rprof.require(new TaskResourceRequests().resource("gpu", 1.5)) }.getMessage() - assert(taskError.contains("The resource amount 1.5 must be either <= 0.5, or a whole number.")) + assert(taskError.contains("The resource amount 1.5 must be either <= 1.0, or a whole number.")) - taskError = intercept[AssertionError] { - rprof.require(new TaskResourceRequests().resource("gpu", 0.7)) - }.getMessage() - assert(taskError.contains("The resource amount 0.7 must be either <= 0.5, or a whole number.")) + rprof.require(new TaskResourceRequests().resource("gpu", 0.7)) + rprof.require(new TaskResourceRequests().resource("gpu", 1.0)) + rprof.require(new TaskResourceRequests().resource("gpu", 2.0)) } test("ResourceProfile has correct custom executor resources") { 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 bf5e9d96cd80..de115bd45453 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -255,10 +255,13 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(exec3ResourceProfileId === rp.id) val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) + val taskResourcesAmount = Map(GPU -> + taskResources(GPU).addresses.map(address => address -> 1.0).toMap) + val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), - taskCpus, taskResources, bytebuffer))) + taskCpus, taskResources, taskResourcesAmount, bytebuffer))) val ts = backend.getTaskSchedulerImpl() when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) @@ -274,7 +277,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo // make sure that `availableAddrs` below won't change when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(Seq.empty) backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources)) + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources, + taskResourcesAmount)) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") @@ -362,10 +366,13 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(exec3ResourceProfileId === rp.id) val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) + val taskResourcesAmount = Map(GPU -> + taskResources(GPU).addresses.map(address => address -> 1.0).toMap) + val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), - taskCpus, taskResources, bytebuffer))) + taskCpus, taskResources, taskResourcesAmount, bytebuffer))) val ts = backend.getTaskSchedulerImpl() when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) @@ -381,7 +388,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo // make sure that `availableAddrs` below won't change when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(Seq.empty) backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources)) + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources, + taskResourcesAmount)) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") @@ -458,7 +466,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val taskCpus = 2 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), - taskCpus, Map.empty, bytebuffer))) + taskCpus, Map.empty, Map.empty, bytebuffer))) when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) backend.driverEndpoint.send(ReviveOffers) 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 3f99e2b4598f..db2249412772 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -22,69 +22,74 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.resource.ResourceUtils.GPU + class ExecutorResourceInfoSuite extends SparkFunSuite { test("Track Executor Resource information") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) assert(info.assignedAddrs.isEmpty) + val reqResource = Seq("0", "1").map(addrs => addrs -> 1.0).toMap // Acquire addresses - info.acquire(Seq("0", "1")) + info.acquire(reqResource) assert(info.availableAddrs.sorted sameElements Seq("2", "3")) assert(info.assignedAddrs.sorted sameElements Seq("0", "1")) // release addresses - info.release(Array("0", "1")) + info.release(reqResource) assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) assert(info.assignedAddrs.isEmpty) } test("Don't allow acquire address that is not available") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) // Acquire some addresses. - info.acquire(Seq("0", "1")) + val reqResource = Seq("0", "1").map(addrs => addrs -> 1.0).toMap + info.acquire(reqResource) assert(!info.availableAddrs.contains("1")) // Acquire an address that is not available val e = intercept[SparkException] { - info.acquire(Array("1")) + info.acquire(Map("1" -> 1.0)) } - assert(e.getMessage.contains("Try to acquire an address that is not available.")) + assert(e.getMessage.contains("Try to acquire gpu address 1 amount: 1.0, but only 0.0 left.")) } test("Don't allow acquire address that doesn't exist") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) assert(!info.availableAddrs.contains("4")) // Acquire an address that doesn't exist val e = intercept[SparkException] { - info.acquire(Array("4")) + info.acquire(Map("4" -> 1.0)) } assert(e.getMessage.contains("Try to acquire an address that doesn't exist.")) } test("Don't allow release address that is not assigned") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) // Acquire addresses - info.acquire(Array("0", "1")) + val reqResource = Seq("0", "1").map(addrs => addrs -> 1.0).toMap + info.acquire(reqResource) assert(!info.assignedAddrs.contains("2")) // Release an address that is not assigned val e = intercept[SparkException] { - info.release(Array("2")) + info.release(Map("2" -> 1.0)) } - assert(e.getMessage.contains("Try to release an address that is not assigned.")) + assert(e.getMessage.contains("Try to release gpu address 2 amount: 1.0. " + + "But the total amount: 2.0 after release should be <= 1")) } test("Don't allow release address that doesn't exist") { // Init Executor Resource. - val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3"), 1) + val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) assert(!info.assignedAddrs.contains("4")) // Release an address that doesn't exist val e = intercept[SparkException] { - info.release(Array("4")) + info.release(Map("4" -> 1.0)) } assert(e.getMessage.contains("Try to release an address that doesn't exist.")) } @@ -93,24 +98,92 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val slotSeq = Seq(10, 9, 8, 7, 6, 5, 4, 3, 2, 1) val addresses = ArrayBuffer("0", "1", "2", "3") slotSeq.foreach { slots => - val info = new ExecutorResourceInfo(GPU, addresses.toSeq, slots) + val taskAmount = 1.0 / slots + val info = new ExecutorResourceInfo(GPU, addresses.toSeq) for (_ <- 0 until slots) { - addresses.foreach(addr => info.acquire(Seq(addr))) + addresses.foreach(addr => info.acquire(Map(addr -> taskAmount))) } - // assert that each address was assigned `slots` times - info.assignedAddrs - .groupBy(identity) - .view - .mapValues(_.size) - .foreach(x => assert(x._2 == slots)) + // All addresses has been assigned + assert(info.resourcesAmounts.values.toSeq.toSet.size == 1) + // The left amount of any address should < taskAmount + assert(info.resourcesAmounts("0") < taskAmount) addresses.foreach { addr => assertThrows[SparkException] { - info.acquire(Seq(addr)) + info.acquire(Map(addr -> taskAmount)) } - assert(!info.availableAddrs.contains(addr)) } } } + + def compareMaps(lhs: Map[String, Double], rhs: Map[String, Double], + eps: Double = 0.00000001): Boolean = { + lhs.size == rhs.size && + lhs.zip(rhs).forall { case ((lName, lAmount), (rName, rAmount)) => + lName == rName && (lAmount - rAmount).abs < eps + } + } + + test("assign/release resource for different task requirements") { + val execInfo = new ExecutorResourceInfo("gpu", Seq("0", "1", "2", "3")) + + def testAllocation(taskAddressAmount: Map[String, Double], + expectedLeftRes: Map[String, Double] + ): Unit = { + execInfo.acquire(taskAddressAmount) + val leftRes = execInfo.resourcesAmounts + assert(compareMaps(leftRes, expectedLeftRes)) + } + + def testRelease(releasedRes: Map[String, Double], + expectedLeftRes: Map[String, Double] + ): Unit = { + execInfo.release(releasedRes) + val leftRes = execInfo.resourcesAmounts + assert(compareMaps(leftRes, expectedLeftRes)) + } + + testAllocation(taskAddressAmount = Map("0" -> 0.2), + expectedLeftRes = Map("0" -> 0.8, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("0" -> 0.2), + expectedLeftRes = Map("0" -> 0.6, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("1" -> 1.0, "2" -> 1.0), + expectedLeftRes = Map("0" -> 0.6, "1" -> 0.0, "2" -> 0.0, "3" -> 1.0)) + + testRelease(releasedRes = Map("0" -> 0.1, "2" -> 0.8), + expectedLeftRes = Map("0" -> 0.7, "1" -> 0.0, "2" -> 0.8, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("0" -> 0.50002), + expectedLeftRes = Map("0" -> 0.19998, "1" -> 0.0, "2" -> 0.8, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("3" -> 1.0), + expectedLeftRes = Map("0" -> 0.19998, "1" -> 0.0, "2" -> 0.8, "3" -> 0.0)) + + testAllocation(taskAddressAmount = Map("2" -> 0.2), + expectedLeftRes = Map("0" -> 0.19998, "1" -> 0.0, "2" -> 0.6, "3" -> 0.0)) + + testRelease(releasedRes = Map("0" -> 0.80002, "1" -> 1.0, "2" -> 0.4, "3" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("0" -> 1.0), + expectedLeftRes = Map("0" -> 0.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testRelease(releasedRes = Map("0" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("0" -> 1.0, "1" -> 1.0), + expectedLeftRes = Map("0" -> 0.0, "1" -> 0.0, "2" -> 1.0, "3" -> 1.0)) + + testRelease(releasedRes = Map("0" -> 1.0, "1" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testAllocation(taskAddressAmount = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0), + expectedLeftRes = Map("0" -> 0.0, "1" -> 0.0, "2" -> 0.0, "3" -> 0.0)) + + testRelease(releasedRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala new file mode 100644 index 000000000000..21a73f5125b6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala @@ -0,0 +1,545 @@ +/* + * 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.{SparkException, SparkFunSuite} +import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests} +import org.apache.spark.resource.ResourceUtils.GPU + +class ExecutorResourcesAmountsSuite extends SparkFunSuite { + + def compareMaps(lhs: Map[String, Double], rhs: Map[String, Double], + eps: Double = 0.00000001): Boolean = { + lhs.size == rhs.size && + lhs.zip(rhs).forall { case ((lName, lAmount), (rName, rAmount)) => + lName == rName && (lAmount - rAmount).abs < eps + } + } + + test("assign to rp without task resources requirement") { + val executorsInfo = Map( + "gpu" -> new ExecutorResourceInfo("gpu", Seq("2", "4", "6")), + "fpga" -> new ExecutorResourceInfo("fpga", Seq("aa", "bb")) + ) + val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) + assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + + val treqs = new TaskResourceRequests().cpus(1) + val rp = new ResourceProfileBuilder().require(treqs).build() + + // assign nothing to rp without resource profile + val assigned = availableExecResAmounts.assignResources(rp) + assigned.foreach { case (resource, resourceAmounts) => + assert(resource.isEmpty) + assert(resourceAmounts.isEmpty) + } + } + + test("Convert ExecutorResourceInfos to ExecutorResourcesAmounts") { + val executorsInfo = Map( + "gpu" -> new ExecutorResourceInfo("gpu", Seq("2", "4", "6")), + "fpga" -> new ExecutorResourceInfo("fpga", Seq("aa", "bb")) + ) + // default resources amounts of executors info + executorsInfo.foreach { case (rName, rInfo) => + if (rName == "gpu") { + assert(compareMaps(rInfo.resourcesAmounts, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(rInfo.resourcesAmounts, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) + assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + + val availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + // Update executors info + // executors info shouldn't be changed. + executorsInfo.foreach { case (rName, rInfo) => + if (rName == "gpu") { + rInfo.acquire(Map("2" -> 0.4, "6" -> 0.6)) + } else { + rInfo.acquire(Map("aa" -> 0.2, "bb" -> 0.7)) + } + } + + executorsInfo.foreach { case (rName, rInfo) => + if (rName == "gpu") { + assert(compareMaps(rInfo.resourcesAmounts, Map("2" -> 0.6, "4" -> 1.0, "6" -> 0.4))) + } else { + assert(compareMaps(rInfo.resourcesAmounts, Map("aa" -> 0.8, "bb" -> 0.3))) + } + } + + val availableExecResAmounts1 = ExecutorResourcesAmounts(executorsInfo) + assert(availableExecResAmounts1.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + + val availableRes1 = availableExecResAmounts1.availableResources + availableRes1.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map("2" -> 0.6, "4" -> 1.0, "6" -> 0.4))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 0.8, "bb" -> 0.3))) + } + } + + } + + test("ExecutorResourcesAmounts shouldn't change ExecutorResourceInfo") { + val executorsInfo = Map( + "gpu" -> new ExecutorResourceInfo("gpu", Seq("2", "4", "6")), + "fpga" -> new ExecutorResourceInfo("fpga", Seq("aa", "bb")) + ) + + // default resources amounts of executors info + executorsInfo.foreach { case (rName, rInfo) => + if (rName == "gpu") { + assert(compareMaps(rInfo.resourcesAmounts, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(rInfo.resourcesAmounts, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) + assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + + val gpuTaskAmount = 0.1 + val treqs = new TaskResourceRequests().resource("gpu", gpuTaskAmount) + val rp = new ResourceProfileBuilder().require(treqs).build() + + // taskMount = 0.1 < 1.0 which can be assigned. + val assigned = availableExecResAmounts.assignResources(rp) + // update the value + availableExecResAmounts.acquire(assigned.get._2) + + val availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, + Map("2" -> (1.0 - gpuTaskAmount), "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + // executors info shouldn't be changed. + executorsInfo.foreach { case (rName, rInfo) => + if (rName == "gpu") { + assert(compareMaps(rInfo.resourcesAmounts, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(rInfo.resourcesAmounts, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + } + + test("executor resources are not matching to the task requirement") { + val totalRes = Map("gpu" -> Map("2" -> 0.4)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + val gpuTaskAmount = 0.6 + val treqs = new TaskResourceRequests() + .resource("gpu", gpuTaskAmount) + val rp = new ResourceProfileBuilder().require(treqs).build() + + val assigned = availableExecResAmounts.assignResources(rp) + assert(assigned.isEmpty) + } + + test("part of executor resources are not matching to the task requirement") { + val totalRes = Map("gpu" -> Map("2" -> 0.4), "fpga" -> Map("aa" -> 0.8)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + // normal allocation + val gpuTaskAmount = 0.3 + val fpgaTaskAmount = 0.8 + val treqs = new TaskResourceRequests() + .resource("gpu", gpuTaskAmount) + .resource("fpga", fpgaTaskAmount) + val rp = new ResourceProfileBuilder().require(treqs).build() + + var assigned = availableExecResAmounts.assignResources(rp) + assert(!assigned.isEmpty) + assigned.foreach { case (resource, resourcesAmounts) => + assert(!resource.isEmpty) + assert(!resourcesAmounts.isEmpty) + } + + val treqs1 = new TaskResourceRequests() + .resource("gpu", gpuTaskAmount) + .resource("fpga", 0.9) // couldn't allocate fpga + val rp1 = new ResourceProfileBuilder().require(treqs1).build() + + assigned = availableExecResAmounts.assignResources(rp1) + assert(assigned.isEmpty) + } + + test("the total amount after release should be <= 1.0") { + val totalRes = Map("gpu" -> Map("2" -> 0.4)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + val e = intercept[SparkException] { + availableExecResAmounts.release(Map("gpu" -> Map("2" -> 0.7))) + } + assert(e.getMessage.contains("after releasing gpu address 2 should be <= 1.0")) + + availableExecResAmounts.release(Map("gpu" -> Map("2" -> 0.6))) + assert(compareMaps(availableExecResAmounts.availableResources("gpu"), Map("2" -> 1.0))) + } + + test("the total amount after acquire should be >= 0") { + val totalRes = Map("gpu" -> Map("2" -> 0.4)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + val e = intercept[SparkException] { + availableExecResAmounts.acquire(Map("gpu" -> Map("2" -> 0.6))) + } + assert(e.getMessage.contains("after acquiring gpu address 2 should be >= 0")) + + availableExecResAmounts.acquire(Map("gpu" -> Map("2" -> 0.4))) + assert(compareMaps(availableExecResAmounts.availableResources("gpu"), Map("2" -> 0.0))) + } + + test("Ensure that we can acquire the same fractions of a resource") { + val slotSeq = Seq(31235, 1024, 512, 256, 128, 64, 32, 16, 12, 10, 9, 8, 7, 6, 5, 4, 3, 2, 1) + val addresses = ArrayBuffer("0", "1", "2", "3") + val info = new ExecutorResourceInfo(GPU, addresses.toSeq) + + slotSeq.foreach { slots => + val taskAmount = 1.0 / slots + val availableExecResAmounts = ExecutorResourcesAmounts(Map(GPU -> info)) + for (_ <- 0 until slots) { + addresses.foreach(addr => + availableExecResAmounts.acquire(Map(GPU -> Map(addr -> taskAmount)))) + } + + assert(availableExecResAmounts.availableResources.size === 1) + // All addresses has been assigned + assert(availableExecResAmounts.availableResources(GPU).values.toSeq.toSet.size === 1) + // The left amount of any address should < taskAmount + assert(availableExecResAmounts.availableResources(GPU)("0") < taskAmount) + + addresses.foreach { addr => + assertThrows[SparkException] { + availableExecResAmounts.acquire(Map(GPU -> Map(addr -> taskAmount))) + } + } + } + } + + test("assign acquire release on single task resource request") { + val executorsInfo = Map( + "gpu" -> new ExecutorResourceInfo("gpu", Seq("2", "4", "6")), + "fpga" -> new ExecutorResourceInfo("fpga", Seq("aa", "bb")) + ) + + val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) + + assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + + val gpuTaskAmount = 0.1 + val treqs = new TaskResourceRequests().resource("gpu", gpuTaskAmount) + val rp = new ResourceProfileBuilder().require(treqs).build() + + // taskMount = 0.1 < 1.0 which can be assigned. + val assigned = availableExecResAmounts.assignResources(rp) + assert(!assigned.isEmpty) + assigned.foreach { case (resource, resourceAmounts) => + assert(resource.size === 1) + assert(resource.keys.toSeq === Seq("gpu")) + assert(resource("gpu").name === "gpu") + assert(resource("gpu").addresses === Array("2")) + + assert(resourceAmounts.size === 1) + assert(resourceAmounts.keys.toSeq === Seq("gpu")) + assert(resourceAmounts("gpu").size === 1) + assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) + assert(resourceAmounts("gpu")(resource("gpu").addresses(0)) === gpuTaskAmount) + } + + // assign will not update the real value. + var availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + // acquire will updates the value + availableExecResAmounts.acquire(assigned.get._2) + + // after acquire + availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map( + "2" -> (1.0 - gpuTaskAmount), + "4" -> 1.0, + "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + // release + availableExecResAmounts.release(assigned.get._2) + + availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + } + + test("assign acquire release on multiple task resources request") { + val executorsInfo = Map( + "gpu" -> new ExecutorResourceInfo("gpu", Seq("2", "4", "6")), + "fpga" -> new ExecutorResourceInfo("fpga", Seq("aa", "bb")) + ) + + val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) + + assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + + val gpuTaskAmount = 0.1 + val fpgaTaskAmount = 0.3 + val treqs = new TaskResourceRequests() + .resource("gpu", gpuTaskAmount) + .resource("fpga", fpgaTaskAmount) + val rp = new ResourceProfileBuilder().require(treqs).build() + + // taskMount = 0.1 < 1.0 which can be assigned. + val assigned = availableExecResAmounts.assignResources(rp) + assert(!assigned.isEmpty) + assigned.foreach { case (resource, resourceAmounts) => + assert(resource.size === 2) + assert(resource.keys.toSeq.sorted === Seq("gpu", "fpga").sorted) + + assert(resource("gpu").name === "gpu") + assert(resource("gpu").addresses === Array("2")) + + assert(resource("fpga").name === "fpga") + assert(resource("fpga").addresses === Array("aa")) + + assert(resourceAmounts.size === 2) + assert(resourceAmounts.keys.toSeq.sorted === Seq("gpu", "fpga").sorted) + + assert(resourceAmounts("gpu").size === 1) + assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) + assert(resourceAmounts("gpu")(resource("gpu").addresses(0)) === gpuTaskAmount) + + assert(resourceAmounts("fpga").size === 1) + assert(resourceAmounts("fpga").keys.toSeq === Seq("aa")) + assert(resourceAmounts("fpga")(resource("fpga").addresses(0)) === fpgaTaskAmount) + } + + // assign will not update the real value. + var availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + + // acquire will updates the value + availableExecResAmounts.acquire(assigned.get._2) + + // after acquire + availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, + Map("2" -> (1.0 - gpuTaskAmount), "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> (1.0 - fpgaTaskAmount), "bb" -> 1.0))) + } + } + + // release + availableExecResAmounts.release(assigned.get._2) + + availableRes = availableExecResAmounts.availableResources + availableRes.foreach { case (rName, addressesAmount) => + if (rName == "gpu") { + assert(compareMaps(addressesAmount, Map("2" -> 1.0, "4" -> 1.0, "6" -> 1.0))) + } else { + assert(compareMaps(addressesAmount, Map("aa" -> 1.0, "bb" -> 1.0))) + } + } + } + + test("assign/release resource for different task requirements") { + val totalRes = Map("gpu" -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + def testAllocation(taskAmount: Double, + expectedAssignedAddress: Array[String], + expectedAssignedAmount: Array[Double], + expectedLeftRes: Map[String, Double] + ): Unit = { + val treqs = new TaskResourceRequests().resource("gpu", taskAmount) + val rp = new ResourceProfileBuilder().require(treqs).build() + val assigned = availableExecResAmounts.assignResources(rp) + assert(!assigned.isEmpty) + assigned.foreach { case (resource, resourceAmounts) => + assert(resource("gpu").addresses.sorted === expectedAssignedAddress.sorted) + assert(resourceAmounts("gpu").values.toArray.sorted === expectedAssignedAmount.sorted) + + availableExecResAmounts.acquire(resourceAmounts) + + val leftRes = availableExecResAmounts.availableResources + assert(leftRes.size == 1) + assert(leftRes.keys.toSeq(0) == "gpu") + assert(compareMaps(leftRes("gpu"), expectedLeftRes)) + } + } + + def testRelease(releasedRes: Map[String, Double], + expectedLeftRes: Map[String, Double] + ): Unit = { + availableExecResAmounts.release(Map("gpu" -> releasedRes)) + + val leftRes = availableExecResAmounts.availableResources + assert(leftRes.size == 1) + assert(leftRes.keys.toSeq(0) == "gpu") + assert(compareMaps(leftRes("gpu"), expectedLeftRes)) + } + + // request 0.2 gpu, ExecutorResourcesAmounts should assign "0", + testAllocation(taskAmount = 0.2, + expectedAssignedAddress = Array("0"), + expectedAssignedAmount = Array(0.2), + expectedLeftRes = Map("0" -> 0.8, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + // request 0.2 gpu again, ExecutorResourcesAmounts should assign "0", + testAllocation(taskAmount = 0.2, + expectedAssignedAddress = Array("0"), + expectedAssignedAmount = Array(0.2), + expectedLeftRes = Map("0" -> 0.6, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + // request 2 gpus, ExecutorResourcesAmounts should assign "1" and "2", + testAllocation(taskAmount = 2, + expectedAssignedAddress = Array("1", "2"), + expectedAssignedAmount = Array(1.0, 1.0), + expectedLeftRes = Map("0" -> 0.6, "1" -> 0.0, "2" -> 0.0, "3" -> 1.0)) + + testRelease(releasedRes = Map("0" -> 0.1, "2" -> 0.8), + expectedLeftRes = Map("0" -> 0.7, "1" -> 0.0, "2" -> 0.8, "3" -> 1.0)) + + // request 0.50002 gpu, ExecutorResourcesAmounts should assign "0", + testAllocation(taskAmount = 0.50002, + expectedAssignedAddress = Array("0"), + expectedAssignedAmount = Array(0.50002), + expectedLeftRes = Map("0" -> 0.19998, "1" -> 0.0, "2" -> 0.8, "3" -> 1.0)) + + // request 1 gpu, ExecutorResourcesAmounts should assign "3", + testAllocation(taskAmount = 1.0, + expectedAssignedAddress = Array("3"), + expectedAssignedAmount = Array(1.0), + expectedLeftRes = Map("0" -> 0.19998, "1" -> 0.0, "2" -> 0.8, "3" -> 0.0)) + + // request 0.2 gpu, ExecutorResourcesAmounts should assign "2", + testAllocation(taskAmount = 0.2, + expectedAssignedAddress = Array("2"), + expectedAssignedAmount = Array(0.2), + expectedLeftRes = Map("0" -> 0.19998, "1" -> 0.0, "2" -> 0.6, "3" -> 0.0)) + + testRelease(releasedRes = Map("0" -> 0.80002, "1" -> 1.0, "2" -> 0.4, "3" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + // request 1 gpus, ExecutorResourcesAmounts should assign "0" + testAllocation(taskAmount = 1.0, + expectedAssignedAddress = Array("0"), + expectedAssignedAmount = Array(1.0), + expectedLeftRes = Map("0" -> 0.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + testRelease(releasedRes = Map("0" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + // request 2 gpus, ExecutorResourcesAmounts should assign "0", "1" + testAllocation(taskAmount = 2.0, + expectedAssignedAddress = Array("0", "1"), + expectedAssignedAmount = Array(1.0, 1.0), + expectedLeftRes = Map("0" -> 0.0, "1" -> 0.0, "2" -> 1.0, "3" -> 1.0)) + + testRelease(releasedRes = Map("0" -> 1.0, "1" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + + // request 4 gpus, ExecutorResourcesAmounts should assign "0", "1", "2", "3" + testAllocation(taskAmount = 4.0, + expectedAssignedAddress = Array("0", "1", "2", "3"), + expectedAssignedAmount = Array(1.0, 1.0, 1.0, 1.0), + expectedLeftRes = Map("0" -> 0.0, "1" -> 0.0, "2" -> 0.0, "3" -> 0.0)) + + testRelease(releasedRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0), + expectedLeftRes = Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + } + + test("Don't allow acquire resource or address that is not available") { + // Init Executor Resource. + val totalRes = Map("gpu" -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + // Acquire an address from a resource that doesn't exist + val e = intercept[SparkException] { + availableExecResAmounts.acquire(Map("fpga" -> Map("1" -> 1.0))) + } + assert(e.getMessage.contains("Try to acquire an address from fpga that doesn't exist")) + + // Acquire an address that is not available + val e1 = intercept[SparkException] { + availableExecResAmounts.acquire(Map("gpu" -> Map("6" -> 1.0))) + } + assert(e1.getMessage.contains("Try to acquire an address that doesn't exist")) + } + + test("Don't allow release resource or address that is not available") { + // Init Executor Resource. + val totalRes = Map("gpu" -> Map("0" -> 0.5, "1" -> 0.5, "2" -> 0.5, "3" -> 0.5)) + val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) + + // Acquire an address from a resource that doesn't exist + val e = intercept[SparkException] { + availableExecResAmounts.release(Map("fpga" -> Map("1" -> 0.1))) + } + assert(e.getMessage.contains("Try to release an address from fpga that doesn't exist")) + + // Acquire an address that is not available + val e1 = intercept[SparkException] { + availableExecResAmounts.release(Map("gpu" -> Map("6" -> 0.1))) + } + assert(e1.getMessage.contains("Try to release an address that is not assigned")) + } + +} 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 b36363d0f4cd..402af94dc14c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -62,6 +62,8 @@ class TaskDescriptionSuite extends SparkFunSuite { val originalResources = Map(GPU -> new ResourceInformation(GPU, Array("1", "2", "3"))) + val originalResourcesAmounts = Map(GPU -> Map("1" -> 0.2, "2" -> 0.5, "3" -> 0.1)) + // Create a dummy byte buffer for the task. val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) @@ -83,6 +85,7 @@ class TaskDescriptionSuite extends SparkFunSuite { originalProperties, cpus = 2, originalResources, + originalResourcesAmounts, taskBuffer ) @@ -100,6 +103,7 @@ class TaskDescriptionSuite extends SparkFunSuite { assert(decodedTaskDescription.properties.equals(originalTaskDescription.properties)) assert(decodedTaskDescription.cpus.equals(originalTaskDescription.cpus)) assert(equalResources(decodedTaskDescription.resources, originalTaskDescription.resources)) + assert(decodedTaskDescription.resourcesAmounts === originalTaskDescription.resourcesAmounts) assert(decodedTaskDescription.serializedTask.equals(taskBuffer)) def equalResources(original: Map[String, ResourceInformation], 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 f0ae7fc74112..bd961d2a1f9e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -22,8 +22,10 @@ import java.util.Properties import java.util.concurrent.{CountDownLatch, ExecutorService, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.duration._ +import scala.language.implicitConversions import scala.language.reflectiveCalls import org.mockito.ArgumentMatchers.{any, anyInt, anyString, eq => meq} @@ -39,6 +41,8 @@ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.util.{Clock, ManualClock, ThreadUtils} + + class FakeSchedulerBackend extends SchedulerBackend { def start(): Unit = {} def stop(): Unit = {} @@ -130,11 +134,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. dagScheduler = new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} + override def taskSetFailed( - taskSet: TaskSet, - reason: String, - exception: Option[Throwable]): Unit = { + taskSet: TaskSet, + reason: String, + exception: Option[Throwable]): Unit = { // Normally the DAGScheduler puts this in the event loop, which will eventually fail // dependent jobs failedTaskSet = true @@ -145,6 +151,15 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler } + // Convert resources to ExecutorResourcesAmounts automatically + implicit def convertResources(resources: Map[String, mutable.Buffer[String]]): + ExecutorResourcesAmounts = { + // convert the old resources to ExecutorResourcesAmounts + new ExecutorResourcesAmounts(resources.map { case (rName, addresses) => + rName -> addresses.map(address => address -> 1.0).toMap + }) + } + test("SPARK-32653: Decommissioned host/executor should be considered as inactive") { val scheduler = setupScheduler() val exec0 = "exec0" @@ -228,8 +243,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(!failedTaskSet) } - private def setupTaskSchedulerForLocalityTests( - clock: ManualClock, + private def setupTaskSchedulerForLocalityTests(clock: ManualClock, conf: SparkConf = new SparkConf()): TaskSchedulerImpl = { sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) val taskScheduler = new TaskSchedulerImpl(sc, @@ -238,6 +252,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } + override def shuffleOffers(offers: IndexedSeq[WorkerOffer]): IndexedSeq[WorkerOffer] = { // Don't shuffle the offers around for this test. Instead, we'll just pass in all // the permutations we care about directly. @@ -557,6 +572,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext test("concurrent attempts for the same stage only have one active taskset") { val taskScheduler = setupScheduler() + def isTasksetZombie(taskset: TaskSet): Boolean = { taskScheduler.taskSetManagerForAttempt(taskset.stageId, taskset.stageAttemptId).get.isZombie } @@ -686,7 +702,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext test("scheduled tasks obey task and stage excludelist") { taskScheduler = setupSchedulerWithMockTaskSetExcludelist() - (0 to 2).foreach {stageId => + (0 to 2).foreach { stageId => val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) } @@ -721,8 +737,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } def tasksForStage(stageId: Int): Seq[TaskDescription] = { - firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} + firstTaskAttempts.filter { + _.name.contains(s"stage $stageId") + } } + tasksForStage(0).foreach { task => // executors 1 & 2 excluded for node // executor 0 excluded just for partition 0 @@ -868,7 +887,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } test("SPARK-22148 abort timer should kick in when task is completely excluded & no new " + - "executor can be acquired") { + "executor can be acquired") { // set the abort timer to fail immediately taskScheduler = setupSchedulerWithMockTaskSetExcludelist( config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0") @@ -1078,9 +1097,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext * in nodes and executors should be on that list. */ private def testExcludelistPerformance( - testName: String, - nodeExcludelist: Seq[String], - execExcludelist: Seq[String]): Unit = { + testName: String, + nodeExcludelist: Seq[String], + execExcludelist: Seq[String]): Unit = { // Because scheduling involves shuffling the order of offers around, we run this test a few // times to cover more possibilities. There are only 3 offers, which means 6 permutations, // so 10 iterations is pretty good. @@ -1230,14 +1249,15 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(tsm.isZombie) assert(failedTaskSet) val idx = failedTask.index - assert(failedTaskSetReason === s""" - |Aborting $taskSet because task $idx (partition $idx) - |cannot run anywhere due to node and executor excludeOnFailure. - |Most recent failure: - |${tsm.taskSetExcludelistHelperOpt.get.getLatestFailureReason} - | - |ExcludeOnFailure behavior can be configured via spark.excludeOnFailure.*. - |""".stripMargin) + assert(failedTaskSetReason === + s""" + |Aborting $taskSet because task $idx (partition $idx) + |cannot run anywhere due to node and executor excludeOnFailure. + |Most recent failure: + |${tsm.taskSetExcludelistHelperOpt.get.getLatestFailureReason} + | + |ExcludeOnFailure behavior can be configured via spark.excludeOnFailure.*. + |""".stripMargin) } test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") { @@ -1333,6 +1353,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} } @@ -1364,6 +1385,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} } @@ -1408,6 +1430,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // the permutations we care about directly. offers } + override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } @@ -1415,6 +1438,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} } taskScheduler.initialize(new FakeSchedulerBackend) @@ -1455,6 +1479,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} } taskScheduler.initialize(new FakeSchedulerBackend) @@ -1473,7 +1498,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // make an offer on a non-preferred location. Since the delay is 0, we should still schedule // immediately. val taskDescs = - taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec2", "host2", 1))).flatten + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec2", "host2", 1))).flatten assert(taskDescs.size === 1) assert(taskDescs.head.executorId === "exec2") } @@ -1517,7 +1542,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), Map("gpu" -> Seq("0").toBuffer)), new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), - Map("gpu" -> Seq("0").toBuffer))) + Map("gpu" -> Seq("0").toBuffer)) + ) val attempt1 = FakeTask.createBarrierTaskSet(3) taskScheduler.submitTasks(attempt1) @@ -1534,7 +1560,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val numFreeCores = 3 val workerOffers = IndexedSeq( new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), - Map("gpu" -> Seq("0").toBuffer)), + convertResources(Map("gpu" -> Seq("0").toBuffer))), new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), Map("gpu" -> Seq("0").toBuffer)), new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"), @@ -1644,10 +1670,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.initialize(new FakeSchedulerBackend { override def killTask( - taskId: Long, - executorId: String, - interruptThread: Boolean, - reason: String): Unit = { + taskId: Long, + executorId: String, + interruptThread: Boolean, + reason: String): Unit = { // Since we only submit one stage attempt, the following call is sufficient to mark the // task as killed. taskScheduler.taskSetManagerForAttempt(0, 0).get.runningTasksSet.remove(taskId) @@ -1676,10 +1702,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.initialize(new FakeSchedulerBackend { override def killTask( - taskId: Long, - executorId: String, - interruptThread: Boolean, - reason: String): Unit = { + taskId: Long, + executorId: String, + interruptThread: Boolean, + reason: String): Unit = { // Since we only submit one stage attempt, the following call is sufficient to mark the // task as killed. taskScheduler.taskSetManagerForAttempt(0, 0).get.runningTasksSet.remove(taskId) @@ -1709,7 +1735,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.submitTasks(attempt) val tsm = taskScheduler.taskSetManagerForAttempt(0, 0).get - val offers = (0 until 3).map{ idx => + val offers = (0 until 3).map { idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1, Some(s"192.168.0.101:4962$idx")) } taskScheduler.resourceOffers(offers) @@ -1738,7 +1764,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext 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)) taskScheduler.submitTasks(taskSet) // WorkerOffer doesn't contain GPU resource, don't launch any task. var taskDescriptions = taskScheduler.resourceOffers(zeroGpuWorkerOffers).flatten @@ -1805,7 +1831,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2, None, resourcesDefaultProf), - new WorkerOffer("executor1", "host1", 6, None, resources, rp.id)) + new WorkerOffer("executor1", "host1", 6, None, resources, rp.id)) taskScheduler.submitTasks(taskSet) taskScheduler.submitTasks(rpTaskSet) // should have 2 for default profile and 2 for additional resource profile @@ -1830,8 +1856,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // clear the first 2 worker offers so they don't have any room and add a third // for the resource profile val workerOffers3 = IndexedSeq( - new WorkerOffer("executor0", "host0", 0, None, Map.empty), - new WorkerOffer("executor1", "host1", 0, None, Map.empty, rp.id), + new WorkerOffer("executor0", "host0", 0, None), + new WorkerOffer("executor1", "host1", 0, None, ExecutorResourcesAmounts.empty, rp.id), new WorkerOffer("executor2", "host2", 6, None, resources3, rp.id)) taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten assert(2 === taskDescriptions.length) @@ -1890,8 +1916,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // clear the first 2 worker offers so they don't have any room and add a third // for the resource profile val workerOffers3 = IndexedSeq( - WorkerOffer("executor0", "host0", 0, None, Map.empty), - WorkerOffer("executor1", "host1", 0, None, Map.empty), + WorkerOffer("executor0", "host0", 0, None), + WorkerOffer("executor1", "host1", 0, None), WorkerOffer("executor2", "host2", 4, None, resources3)) taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten assert(2 === taskDescriptions.length) @@ -1922,11 +1948,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val workerOffers = IndexedSeq(WorkerOffer("executor0", "host0", 4, None, resources0), WorkerOffer("executor1", "host1", 4, None, resources1)) - val availableResourcesAmount = workerOffers.map(_.resources).map { resourceMap => - // available addresses already takes into account if there are fractional - // task resource requests - resourceMap.map { case (name, addresses) => (name, addresses.length) } - } + val availableResourcesAmount = workerOffers.map(_.resources).map { resAmounts => + // available addresses already takes into account if there are fractional + // task resource requests + resAmounts.resourceAmount + } val taskSlotsForRp = TaskSchedulerImpl.calculateAvailableSlots( taskScheduler, taskScheduler.conf, rp.id, workerOffers.map(_.resourceProfileId).toArray, @@ -1935,7 +1961,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } private def setupSchedulerForDecommissionTests(clock: Clock, numTasks: Int, - extraConf: Map[String, String] = Map.empty): TaskSchedulerImpl = { + extraConf: Map[String, String] = Map.empty): TaskSchedulerImpl = { // one task per host val numHosts = numTasks val conf = new SparkConf() @@ -1960,7 +1986,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext for (i <- 0 until numTasks) { val executorId = s"executor$i" val taskDescriptions = taskScheduler.resourceOffers(IndexedSeq(WorkerOffer( - executorId, s"host$i", 1))).flatten + executorId, s"host$i", 1))).flatten assert(taskDescriptions.size === 1) assert(taskDescriptions(0).executorId == executorId) assert(taskDescriptions(0).index === i) @@ -2108,7 +2134,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val normalTSM = sched.taskSetManagerForAttempt(1, 0).get assert(barrierTSM.myLocalityLevels === Array(TaskLocality.PROCESS_LOCAL, TaskLocality.NODE_LOCAL, TaskLocality.ANY)) - assert(normalTSM.myLocalityLevels === Array(TaskLocality.NO_PREF, TaskLocality.ANY)) + assert(normalTSM.myLocalityLevels === Array(TaskLocality.NO_PREF, TaskLocality.ANY)) // The barrier TaskSetManager can not launch all tasks because of delay scheduling. // So it will revert assigned resources and let the normal TaskSetManager to schedule first. @@ -2150,7 +2176,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext }) } } - def taskResultExecutor() : ExecutorService = getTaskResultExecutor + + def taskResultExecutor(): ExecutorService = getTaskResultExecutor } taskScheduler.taskResultGetter = resultGetter @@ -2177,14 +2204,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val resultBytes = ser.serialize(directResult) val busyTask = new Runnable { - val lock : Object = new Object - var running : AtomicBoolean = new AtomicBoolean(false) + val lock: Object = new Object + var running: AtomicBoolean = new AtomicBoolean(false) + override def run(): Unit = { lock.synchronized { running.set(true) lock.wait() } } + def markTaskDone: Unit = { lock.synchronized { lock.notify() @@ -2237,6 +2266,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} } @@ -2275,12 +2305,347 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext * that happens, since the operation is performed asynchronously by the TaskResultGetter. */ private def failTask( - tid: Long, - state: TaskState.TaskState, - reason: TaskFailedReason, - tsm: TaskSetManager): Unit = { + tid: Long, + state: TaskState.TaskState, + reason: TaskFailedReason, + tsm: TaskSetManager): Unit = { taskScheduler.statusUpdate(tid, state, ByteBuffer.allocate(0)) taskScheduler.handleFailedTask(tsm, tid, state, reason) } + Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => + test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + + s"restrict each task takes 1 gpu exclusively in the same executor") { + val taskCpus = 1 + val executorCpus = 1000 // cpu will not limit the concurrent tasks number + val executorGpus = 1 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> gpuTaskAmount.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val taskSet = FakeTask.createTaskSet(100) + + val resources = new ExecutorResourcesAmounts( + Map(GPU -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))) + + val workerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", executorCpus, None, resources)) + + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements. + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 === 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("2") === taskDescriptions(2).resources.get(GPU).get.addresses) + assert(ArrayBuffer("3") === taskDescriptions(3).resources.get(GPU).get.addresses) + } + } + + + Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => + test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + + s"restrict only 1 task will run on each executor") { + val taskCpus = 1 + val executorCpus = 1000 // cpu will not limit the concurrent tasks number + val executorGpus = 1 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> gpuTaskAmount.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val taskSet = FakeTask.createTaskSet(100) + + val workerOffers = + IndexedSeq( + new WorkerOffer("executor0", "host0", executorCpus, None, + new ExecutorResourcesAmounts(Map(GPU -> Map("0" -> 1.0)))), + new WorkerOffer("executor1", "host1", executorCpus, None, + new ExecutorResourcesAmounts(Map(GPU -> Map("7" -> 1.0)))), + new WorkerOffer("executor2", "host2", executorCpus, None, + new ExecutorResourcesAmounts(Map(GPU -> Map("9" -> 1.0)))), + new WorkerOffer("executor3", "host3", executorCpus, None, + new ExecutorResourcesAmounts(Map(GPU -> Map("20" -> 1.0))))) + + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements. + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 === taskDescriptions.length) + assert(!failedTaskSet) + + val assignedExecutorsGpus = ArrayBuffer[(String, String)]() + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + assert(tDesc.resources.get(GPU).get.addresses.length == 1) + assignedExecutorsGpus.append((tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) + } + + assert(assignedExecutorsGpus.sorted sameElements + ArrayBuffer( + ("executor0", "0"), + ("executor1", "7"), + ("executor2", "9"), + ("executor3", "20") + )) + } + } + + test("SPARK-45527 TaskResourceProfile: the left multiple gpu resources on 1 executor " + + "can assign to other taskset") { + val taskCpus = 1 + val taskGpus = 0.3 + val executorGpus = 4 + val executorCpus = 1000 + + // each tasks require 0.3 gpu + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString + ) + val lowerTaskSet = FakeTask.createTaskSet(100, 1, 0, 1, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + + // each task require 0.7 gpu + val treqs = new TaskResourceRequests().cpus(1).resource(GPU, 0.7) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val higherRpTaskSet = FakeTask.createTaskSet(1000, stageId = 2, stageAttemptId = 0, + priority = 0, rpId = rp.id) + + val workerOffers = + IndexedSeq( + // cpu won't be a problem + WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( + Map(GPU -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + ) + + taskScheduler.submitTasks(lowerTaskSet) + taskScheduler.submitTasks(higherRpTaskSet) + + // should have 3 for default profile and 2 for additional resource profile + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(8 === taskDescriptions.length) + var index = 0 + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + assert(tDesc.resources.get(GPU).get.addresses.length == 1) + if (index < 4) { // the first 4 tasks will grab 0.7 gpu + assert(tDesc.resources.get(GPU).get.addresses(0) == index.toString) + assert(tDesc.resourcesAmounts.get(GPU).get(index.toString) == 0.7) + } else { + assert(tDesc.resources.get(GPU).get.addresses(0) == (index - 4).toString) + assert(tDesc.resourcesAmounts.get(GPU).get((index - 4).toString) == 0.3) + } + index += 1 + } + } + + test("SPARK-45527 TaskResourceProfile: the left gpu resources on multiple executors " + + "can assign to other taskset") { + val taskCpus = 1 + val taskGpus = 0.3 + val executorGpus = 4 + val executorCpus = 1000 + + // each tasks require 0.3 gpu + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString + ) + val lowerTaskSet = FakeTask.createTaskSet(100, 1, 0, 1, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + + // each task require 0.7 gpu + val treqs = new TaskResourceRequests().cpus(1).resource(GPU, 0.7) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val higherRpTaskSet = FakeTask.createTaskSet(1000, stageId = 2, stageAttemptId = 0, + priority = 0, rpId = rp.id) + + val workerOffers = + IndexedSeq( + // cpu won't be a problem + WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( + Map(GPU -> Map("0" -> 1.0)))), + WorkerOffer("executor1", "host1", 1000, None, new ExecutorResourcesAmounts( + Map(GPU -> Map("1" -> 1.0)))), + WorkerOffer("executor2", "host2", 1000, None, new ExecutorResourcesAmounts( + Map(GPU -> Map("2" -> 1.0)))), + WorkerOffer("executor3", "host3", 1000, None, new ExecutorResourcesAmounts( + Map(GPU -> Map("3" -> 1.0)))) + ) + + taskScheduler.submitTasks(lowerTaskSet) + taskScheduler.submitTasks(higherRpTaskSet) + + // should have 3 for default profile and 2 for additional resource profile + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(8 === taskDescriptions.length) + + var index = 0 + val higherAssignedExecutorsGpus = ArrayBuffer[(String, String)]() + val lowerAssignedExecutorsGpus = ArrayBuffer[(String, String)]() + + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + assert(tDesc.resources.get(GPU).get.addresses.length == 1) + val address = tDesc.resources.get(GPU).get.addresses(0) + + // Executor 0, executor 1, executor 2, executor 3 + // task_0.7, task_03 task_0.7, task_03 task_0.7, task_03 task_0.7, task_03 + if (index % 2 == 0) { + higherAssignedExecutorsGpus.append( + (tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) + assert(tDesc.resourcesAmounts.get(GPU).get(address) == 0.7) + } else { + lowerAssignedExecutorsGpus.append( + (tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) + assert(tDesc.resourcesAmounts.get(GPU).get(address) == 0.3) + } + index += 1 + } + + assert(higherAssignedExecutorsGpus.sorted sameElements + ArrayBuffer(("executor0", "0"), ("executor1", "1"), ("executor2", "2"), ("executor3", "3") + )) + assert(lowerAssignedExecutorsGpus.sorted sameElements + ArrayBuffer(("executor0", "0"), ("executor1", "1"), ("executor2", "2"), ("executor3", "3") + )) + } + + test("SPARK-45527 TaskResourceProfile: the left multiple gpu resources on 1 executor " + + "can't assign to other taskset due to no enough gpu resource") { + val taskCpus = 1 + val taskGpus = 0.4 + val executorGpus = 4 + val executorCpus = 4 + + // each tasks require 0.3 gpu + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString + ) + val lowerTaskSet = FakeTask.createTaskSet(100, 1, 0, 1, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + + // each task require 0.7 gpu + val treqs = new TaskResourceRequests().cpus(1).resource(GPU, 0.7) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val higherRpTaskSet = FakeTask.createTaskSet(1000, stageId = 2, stageAttemptId = 0, + priority = 0, rpId = rp.id) + + val workerOffers = + IndexedSeq( + // cpu won't be a problem + WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( + Map(GPU -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + ) + + taskScheduler.submitTasks(lowerTaskSet) + taskScheduler.submitTasks(higherRpTaskSet) + + // only offer the resources to the higher taskset + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 === taskDescriptions.length) + var index = 0 + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + assert(tDesc.resources.get(GPU).get.addresses.length == 1) + assert(tDesc.resources.get(GPU).get.addresses(0) == index.toString) + assert(tDesc.resourcesAmounts.get(GPU).get(index.toString) == 0.7) + index += 1 + } + } + + test("SPARK-45527 schedule tasks for a barrier taskSet if all tasks can be launched together") { + val taskCpus = 2 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", + config.CPUS_PER_TASK.key -> taskCpus.toString) + + val numFreeCores = 3 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625")), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627")), + new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"))) + val attempt1 = FakeTask.createBarrierTaskSet(3) + + // submit attempt 1, offer some resources, all tasks get launched together + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(3 === taskDescriptions.length) + } + + Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => { + test(s"SPARK-45527 schedule tasks for a barrier taskSet if all tasks can be launched " + + s"together diff ResourceProfile with task.gpu.amount: ${gpuTaskAmount}, " + + s"but not enough gpus") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) + val execReqs = new ExecutorResourceRequests().cores(100).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", gpuTaskAmount) + val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val numFreeCores = 100 + // make each of the worker offers only have 1 GPU, thus making it not enough + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer), rp.id), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer), rp.id)) + val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(0 === taskDescriptions.length) + } + } + } + + Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => { + test(s"SPARK-45527 schedule tasks for a barrier taskSet if all tasks can be launched " + + s"together diff ResourceProfile with task.gpu.amount: ${gpuTaskAmount}") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) + val execReqs = new ExecutorResourceRequests().cores(100).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", gpuTaskAmount) + val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val numFreeCores = 100 + // make each of the worker offers only have 1 GPU, thus making it not enough + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer), rp.id), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer), rp.id), + new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49628"), + Map("gpu" -> Seq("0").toBuffer), rp.id)) + val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(3 === taskDescriptions.length) + } + } + } } From 2d68843128db87faaddbd120d9616c8de14cecba Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Fri, 27 Oct 2023 18:44:21 +0800 Subject: [PATCH 02/18] comments --- .../spark/deploy/master/WorkerInfo.scala | 18 ++-- .../spark/resource/ResourceAllocator.scala | 15 ++- .../scheduler/ExecutorResourceInfo.scala | 10 -- .../scheduler/ExecutorResourcesAmounts.scala | 45 ++++---- .../spark/scheduler/TaskDescription.scala | 2 + .../spark/scheduler/TaskSchedulerImpl.scala | 4 +- .../cluster/CoarseGrainedClusterMessage.scala | 1 - .../scheduler/ExecutorResourceInfoSuite.scala | 1 - .../scheduler/TaskSchedulerImplSuite.scala | 101 +++++++----------- 9 files changed, 91 insertions(+), 106 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 7408e227746d..16aeb2aa8af3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -20,7 +20,6 @@ package org.apache.spark.deploy.master import scala.collection.mutable import org.apache.spark.resource.{ResourceAllocator, ResourceInformation, ResourceRequirement} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -37,16 +36,13 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String */ def acquire(amount: Int): ResourceInformation = { - var count = amount - val allocated: mutable.HashMap[String, Double] = mutable.HashMap.empty - for (address <- availableAddrs if count > 0) { - if (addressAvailabilityMap(address) == RESOURCE_TOTAL_AMOUNT) { - allocated(address) = 1.0 - count -= 1 - } - } - acquire(allocated.toMap) - new ResourceInformation(resourceName, allocated.keys.toArray) + // Any available address from availableAddrs must be a whole resource + // since worker needs to do full resources to the executors. + val addresses = availableAddrs.take(amount) + assert(addresses.length == amount) + + acquire(addresses.map(addr => addr -> 1.0).toMap) + new ResourceInformation(resourceName, addresses.toArray) } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 454e6a057095..ea8368b4fcdb 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -70,15 +70,24 @@ private[spark] trait ResourceAllocator { * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means * the address is available, while value of 0 means the address is fully assigned. */ - protected lazy val addressAvailabilityMap = { + private lazy val addressAvailabilityMap = { mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*) } /** - * Sequence of currently available resource addresses which is not fully assigned. + * Get the resources and its amounts. + * @return the resources amounts + */ + def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map { + case (address, internalAmount) => + address -> (internalAmount.toDouble / RESOURCE_TOTAL_AMOUNT) + }.toMap + + /** + * Sequence of currently available resource addresses which are not fully assigned. */ def availableAddrs: Seq[String] = addressAvailabilityMap - .filter(addresses => addresses._2 > 0).keys.toSeq.sorted + .filter(addresses => addresses._2 > 0).keys.toSeq.sorted /** * Sequence of currently assigned resource addresses. 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 dde701daf6b8..9a8461d2fccc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import org.apache.spark.resource.{ResourceAllocator, ResourceInformation} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT /** * Class to hold information about a type of Resource on an Executor. This information is managed @@ -50,13 +49,4 @@ private[spark] class ExecutorResourceInfo( } } - /** - * Convert the internal address availability to the public resource format - * @return the resources amounts - */ - def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map { - case (address, internalAmount) => - address -> (internalAmount.toDouble / RESOURCE_TOTAL_AMOUNT) - }.toMap - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index bbc9d19334b6..7095209042b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -57,7 +57,9 @@ private[spark] class ExecutorResourcesAmounts( rName -> addressMap.size } - // convert internal resources back to the public. + /** + * for testing purpose. convert internal resources back to the "fraction" resources. + */ def availableResources: Map[String, Map[String, Double]] = { internalResources.map { case (rName, addressMap) => rName -> addressMap.map { case (address, amount) => @@ -66,7 +68,10 @@ private[spark] class ExecutorResourcesAmounts( } } - // Acquire the resource and update the resource + /** + * Acquire the resource and update the resource + * @param assignedResource the assigned resource information + */ def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = { assignedResource.foreach { case (rName, taskResAmounts) => val availableResourceAmounts = internalResources.getOrElse(rName, @@ -84,14 +89,14 @@ private[spark] class ExecutorResourcesAmounts( s"after acquiring $rName address $address should be >= 0") } internalResources(rName)(address) = internalLeft - // scalastyle:off println - println(s"Acquired. left ${realLeft}") - // scalastyle:on println } } } - // release the resources and update the values + /** + * release the assigned resources to the resource pool + * @param assignedResource resource to be released + */ def release(assignedResource: Map[String, Map[String, Double]]): Unit = { assignedResource.foreach { case (rName, taskResAmounts) => val availableResourceAmounts = internalResources.getOrElse(rName, @@ -108,15 +113,17 @@ private[spark] class ExecutorResourcesAmounts( s"after releasing $rName address $address should be <= 1.0") } internalResources(rName)(address) = internalTotal - // scalastyle:off println - println(s"Released. amount ${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT}") - // scalastyle:on println } } } - // Try to assign the address according to the task requirement. - // Please note that this function will not update the values. + /** + * Try to assign the address according to the task requirement. + * Please note that this function will not update the values. + * + * @param taskSetProf assign resources based on which resource profile + * @return the resource + */ def assignResources(taskSetProf: ResourceProfile): Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = { @@ -132,10 +139,8 @@ private[spark] class ExecutorResourcesAmounts( // we go through all resources here so that we can make sure they match and also get what the // assignments are for the next task for ((rName, taskReqs) <- tsResources) { - // if taskAmount = 1.5, we assign 2.0 gpu for user or - // just throw an exception in a very begging? - // TODO, just remove it, since we enabled the check at the very beginning. - val taskAmount = if (taskReqs.amount < 1.0) taskReqs.amount else Math.ceil(taskReqs.amount) + // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number + val taskAmount = taskReqs.amount internalResources.get(rName) match { case Some(addressesAmountMap) => @@ -154,7 +159,7 @@ private[spark] class ExecutorResourcesAmounts( // Try to assign this whole address first if (internalTaskAmount >= RESOURCE_TOTAL_AMOUNT) { internalTaskAmount -= RESOURCE_TOTAL_AMOUNT - // Assign the whole resource of the address + // Assign the full resource of the address allocatedAddressesMap(address) = 1.0 } else { // Assign the part of the address. @@ -186,10 +191,14 @@ private[spark] class ExecutorResourcesAmounts( private[spark] object ExecutorResourcesAmounts { - // Create an empty ExecutorResourcesAmounts + /** + * Create an empty ExecutorResourcesAmounts + */ def empty: ExecutorResourcesAmounts = new ExecutorResourcesAmounts(Map.empty) - // Converts executor infos to ExecutorResourcesAmounts + /** + * Converts executor infos to ExecutorResourcesAmounts + */ def apply(executorInfos: Map[String, ExecutorResourceInfo]): ExecutorResourcesAmounts = { new ExecutorResourcesAmounts( executorInfos.map { case (rName, rInfo) => rName -> rInfo.resourcesAmounts } 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 f0cd1e32062b..814aa5dc25a2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -58,6 +58,8 @@ private[spark] class TaskDescription( val properties: Properties, val cpus: Int, val resources: immutable.Map[String, ResourceInformation], + // resourcesAmounts is the total resources assigned to the task + // Eg, Map("gpu" -> Map("0" -> 0.7)): assign 0.7 of the gpu address "0" to this task val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]], val serializedTask: ByteBuffer) { 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 8bbb687e27b3..1684bf7199f1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -457,8 +457,8 @@ private[spark] class TaskSchedulerImpl( /** * Check whether the resources from the WorkerOffer are enough to run at least one task. * Returns None if the resources don't meet the task requirements, otherwise returns - * the task resource assignments to give to the next task. Note that the assignments maybe - * be empty if no custom resources are used. + * the task resource assignments and the resource amounts to give to the next task. + * Note that the assignments maybe be empty if no custom resources are used. */ private def resourcesMeetTaskRequirements( taskSet: TaskSetManager, 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 eeeeba94439c..1c0970c81636 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 @@ -25,7 +25,6 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.{ExecutorLossReason, MiscellaneousProcessDetails} import org.apache.spark.util.SerializableBuffer - private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { 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 db2249412772..db716e9c3b08 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.resource.ResourceUtils.GPU - class ExecutorResourceInfoSuite extends SparkFunSuite { test("Track Executor Resource information") { 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 bd961d2a1f9e..35874bbc4805 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -41,8 +41,6 @@ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.util.{Clock, ManualClock, ThreadUtils} - - class FakeSchedulerBackend extends SchedulerBackend { def start(): Unit = {} def stop(): Unit = {} @@ -134,13 +132,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. dagScheduler = new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} - override def executorAdded(execId: String, host: String): Unit = {} - override def taskSetFailed( - taskSet: TaskSet, - reason: String, - exception: Option[Throwable]): Unit = { + taskSet: TaskSet, + reason: String, + exception: Option[Throwable]): Unit = { // Normally the DAGScheduler puts this in the event loop, which will eventually fail // dependent jobs failedTaskSet = true @@ -243,7 +239,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(!failedTaskSet) } - private def setupTaskSchedulerForLocalityTests(clock: ManualClock, + private def setupTaskSchedulerForLocalityTests( + clock: ManualClock, conf: SparkConf = new SparkConf()): TaskSchedulerImpl = { sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) val taskScheduler = new TaskSchedulerImpl(sc, @@ -252,7 +249,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } - override def shuffleOffers(offers: IndexedSeq[WorkerOffer]): IndexedSeq[WorkerOffer] = { // Don't shuffle the offers around for this test. Instead, we'll just pass in all // the permutations we care about directly. @@ -572,7 +568,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext test("concurrent attempts for the same stage only have one active taskset") { val taskScheduler = setupScheduler() - def isTasksetZombie(taskset: TaskSet): Boolean = { taskScheduler.taskSetManagerForAttempt(taskset.stageId, taskset.stageAttemptId).get.isZombie } @@ -702,7 +697,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext test("scheduled tasks obey task and stage excludelist") { taskScheduler = setupSchedulerWithMockTaskSetExcludelist() - (0 to 2).foreach { stageId => + (0 to 2).foreach {stageId => val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) } @@ -737,11 +732,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } def tasksForStage(stageId: Int): Seq[TaskDescription] = { - firstTaskAttempts.filter { - _.name.contains(s"stage $stageId") - } + firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} } - tasksForStage(0).foreach { task => // executors 1 & 2 excluded for node // executor 0 excluded just for partition 0 @@ -887,7 +879,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } test("SPARK-22148 abort timer should kick in when task is completely excluded & no new " + - "executor can be acquired") { + "executor can be acquired") { // set the abort timer to fail immediately taskScheduler = setupSchedulerWithMockTaskSetExcludelist( config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0") @@ -1097,9 +1089,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext * in nodes and executors should be on that list. */ private def testExcludelistPerformance( - testName: String, - nodeExcludelist: Seq[String], - execExcludelist: Seq[String]): Unit = { + testName: String, + nodeExcludelist: Seq[String], + execExcludelist: Seq[String]): Unit = { // Because scheduling involves shuffling the order of offers around, we run this test a few // times to cover more possibilities. There are only 3 offers, which means 6 permutations, // so 10 iterations is pretty good. @@ -1249,15 +1241,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(tsm.isZombie) assert(failedTaskSet) val idx = failedTask.index - assert(failedTaskSetReason === - s""" - |Aborting $taskSet because task $idx (partition $idx) - |cannot run anywhere due to node and executor excludeOnFailure. - |Most recent failure: - |${tsm.taskSetExcludelistHelperOpt.get.getLatestFailureReason} - | - |ExcludeOnFailure behavior can be configured via spark.excludeOnFailure.*. - |""".stripMargin) + assert(failedTaskSetReason === s""" + |Aborting $taskSet because task $idx (partition $idx) + |cannot run anywhere due to node and executor excludeOnFailure. + |Most recent failure: + |${tsm.taskSetExcludelistHelperOpt.get.getLatestFailureReason} + | + |ExcludeOnFailure behavior can be configured via spark.excludeOnFailure.*. + |""".stripMargin) } test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") { @@ -1353,7 +1344,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} - override def executorAdded(execId: String, host: String): Unit = {} } @@ -1385,7 +1375,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} - override def executorAdded(execId: String, host: String): Unit = {} } @@ -1430,7 +1419,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // the permutations we care about directly. offers } - override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } @@ -1438,7 +1426,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} - override def executorAdded(execId: String, host: String): Unit = {} } taskScheduler.initialize(new FakeSchedulerBackend) @@ -1479,7 +1466,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} - override def executorAdded(execId: String, host: String): Unit = {} } taskScheduler.initialize(new FakeSchedulerBackend) @@ -1498,7 +1484,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // make an offer on a non-preferred location. Since the delay is 0, we should still schedule // immediately. val taskDescs = - taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec2", "host2", 1))).flatten + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec2", "host2", 1))).flatten assert(taskDescs.size === 1) assert(taskDescs.head.executorId === "exec2") } @@ -1542,8 +1528,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), Map("gpu" -> Seq("0").toBuffer)), new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), - Map("gpu" -> Seq("0").toBuffer)) - ) + Map("gpu" -> Seq("0").toBuffer))) val attempt1 = FakeTask.createBarrierTaskSet(3) taskScheduler.submitTasks(attempt1) @@ -1670,10 +1655,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.initialize(new FakeSchedulerBackend { override def killTask( - taskId: Long, - executorId: String, - interruptThread: Boolean, - reason: String): Unit = { + taskId: Long, + executorId: String, + interruptThread: Boolean, + reason: String): Unit = { // Since we only submit one stage attempt, the following call is sufficient to mark the // task as killed. taskScheduler.taskSetManagerForAttempt(0, 0).get.runningTasksSet.remove(taskId) @@ -1702,10 +1687,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.initialize(new FakeSchedulerBackend { override def killTask( - taskId: Long, - executorId: String, - interruptThread: Boolean, - reason: String): Unit = { + taskId: Long, + executorId: String, + interruptThread: Boolean, + reason: String): Unit = { // Since we only submit one stage attempt, the following call is sufficient to mark the // task as killed. taskScheduler.taskSetManagerForAttempt(0, 0).get.runningTasksSet.remove(taskId) @@ -1735,7 +1720,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.submitTasks(attempt) val tsm = taskScheduler.taskSetManagerForAttempt(0, 0).get - val offers = (0 until 3).map { idx => + val offers = (0 until 3).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1, Some(s"192.168.0.101:4962$idx")) } taskScheduler.resourceOffers(offers) @@ -1831,7 +1816,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 2, None, resourcesDefaultProf), - new WorkerOffer("executor1", "host1", 6, None, resources, rp.id)) + new WorkerOffer("executor1", "host1", 6, None, resources, rp.id)) taskScheduler.submitTasks(taskSet) taskScheduler.submitTasks(rpTaskSet) // should have 2 for default profile and 2 for additional resource profile @@ -1961,7 +1946,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } private def setupSchedulerForDecommissionTests(clock: Clock, numTasks: Int, - extraConf: Map[String, String] = Map.empty): TaskSchedulerImpl = { + extraConf: Map[String, String] = Map.empty): TaskSchedulerImpl = { // one task per host val numHosts = numTasks val conf = new SparkConf() @@ -1986,7 +1971,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext for (i <- 0 until numTasks) { val executorId = s"executor$i" val taskDescriptions = taskScheduler.resourceOffers(IndexedSeq(WorkerOffer( - executorId, s"host$i", 1))).flatten + executorId, s"host$i", 1))).flatten assert(taskDescriptions.size === 1) assert(taskDescriptions(0).executorId == executorId) assert(taskDescriptions(0).index === i) @@ -2134,7 +2119,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val normalTSM = sched.taskSetManagerForAttempt(1, 0).get assert(barrierTSM.myLocalityLevels === Array(TaskLocality.PROCESS_LOCAL, TaskLocality.NODE_LOCAL, TaskLocality.ANY)) - assert(normalTSM.myLocalityLevels === Array(TaskLocality.NO_PREF, TaskLocality.ANY)) + assert(normalTSM.myLocalityLevels === Array(TaskLocality.NO_PREF, TaskLocality.ANY)) // The barrier TaskSetManager can not launch all tasks because of delay scheduling. // So it will revert assigned resources and let the normal TaskSetManager to schedule first. @@ -2176,8 +2161,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext }) } } - - def taskResultExecutor(): ExecutorService = getTaskResultExecutor + def taskResultExecutor() : ExecutorService = getTaskResultExecutor } taskScheduler.taskResultGetter = resultGetter @@ -2204,16 +2188,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val resultBytes = ser.serialize(directResult) val busyTask = new Runnable { - val lock: Object = new Object - var running: AtomicBoolean = new AtomicBoolean(false) - + val lock : Object = new Object + var running : AtomicBoolean = new AtomicBoolean(false) override def run(): Unit = { lock.synchronized { running.set(true) lock.wait() } } - def markTaskDone: Unit = { lock.synchronized { lock.notify() @@ -2266,7 +2248,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} - override def executorAdded(execId: String, host: String): Unit = {} } @@ -2305,10 +2286,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext * that happens, since the operation is performed asynchronously by the TaskResultGetter. */ private def failTask( - tid: Long, - state: TaskState.TaskState, - reason: TaskFailedReason, - tsm: TaskSetManager): Unit = { + tid: Long, + state: TaskState.TaskState, + reason: TaskFailedReason, + tsm: TaskSetManager): Unit = { taskScheduler.statusUpdate(tid, state, ByteBuffer.allocate(0)) taskScheduler.handleFailedTask(tsm, tid, state, reason) } From fbd647d6d00ab7f035b050a0b45bdcdb611e2239 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 30 Oct 2023 20:22:27 +0800 Subject: [PATCH 03/18] comments --- .../scheduler/ExecutorResourcesAmounts.scala | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index 7095209042b2..32a1559cd17c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -39,11 +39,10 @@ import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT private[spark] class ExecutorResourcesAmounts( private val resources: Map[String, Map[String, Double]]) extends Serializable { - resources.foreach { case (_, addressMount) => - addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}} - - // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly. - // and convert the addressesAmounts to be mutable.HashMap + /** + * Multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly. + * and convert the addressesAmounts to be mutable.HashMap + */ private val internalResources: Map[String, HashMap[String, Long]] = { resources.map { case (rName, addressAmounts) => rName -> HashMap(addressAmounts.map { case (address, amount) => @@ -52,15 +51,20 @@ private[spark] class ExecutorResourcesAmounts( } } - // It maps from the resource name to its amount. + /** + * The total address count of each resource. Eg, + * Map("gpu" -> Map("0" -> 0.5, "1" -> 0.5, "2" -> 0.5), + * "fpga" -> Map("a" -> 0.5, "b" -> 0.5)) + * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2) + */ lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) => rName -> addressMap.size } /** - * for testing purpose. convert internal resources back to the "fraction" resources. + * For testing purpose. convert internal resources back to the "fraction" resources. */ - def availableResources: Map[String, Map[String, Double]] = { + private[spark] def availableResources: Map[String, Map[String, Double]] = { internalResources.map { case (rName, addressMap) => rName -> addressMap.map { case (address, amount) => address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT @@ -94,7 +98,7 @@ private[spark] class ExecutorResourcesAmounts( } /** - * release the assigned resources to the resource pool + * Release the assigned resources to the resource pool * @param assignedResource resource to be released */ def release(assignedResource: Map[String, Map[String, Double]]): Unit = { From 58899b21cfccc3c42306ddb76505d003f3d72e0c Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Thu, 2 Nov 2023 14:51:00 +0800 Subject: [PATCH 04/18] comments --- .../scheduler/ExecutorResourcesAmounts.scala | 35 ++++++++----------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index 32a1559cd17c..857a8347b6b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -144,42 +144,37 @@ private[spark] class ExecutorResourcesAmounts( // assignments are for the next task for ((rName, taskReqs) <- tsResources) { // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number - val taskAmount = taskReqs.amount + var taskAmount = taskReqs.amount internalResources.get(rName) match { case Some(addressesAmountMap) => - - var internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong val allocatedAddressesMap = HashMap[String, Double]() // always sort the addresses val addresses = addressesAmountMap.keys.toSeq.sorted - for (address <- addresses if internalTaskAmount > 0) { - val freeAmount = addressesAmountMap(address) - - // The address is still a whole resource - if (freeAmount == RESOURCE_TOTAL_AMOUNT) { - // Try to assign this whole address first - if (internalTaskAmount >= RESOURCE_TOTAL_AMOUNT) { - internalTaskAmount -= RESOURCE_TOTAL_AMOUNT + // task.amount is a whole number + if (taskAmount >= 1.0) { + for (address <- addresses if taskAmount > 0) { + // The address is still a whole resource + if (addressesAmountMap(address) == RESOURCE_TOTAL_AMOUNT) { + taskAmount -= 1.0 // Assign the full resource of the address allocatedAddressesMap(address) = 1.0 - } else { - // Assign the part of the address. - allocatedAddressesMap(address) = taskAmount - internalTaskAmount = 0 } - } else { - // The address is fraction - if (freeAmount - internalTaskAmount >= 0) { + } + } else if (taskAmount > 0.0) { // 0 < task.amount < 1.0 + val internalTaskAmount = taskAmount * RESOURCE_TOTAL_AMOUNT + for (address <- addresses if taskAmount > 0) { + if (addressesAmountMap(address) >= internalTaskAmount) { + // Assign the part of the address. allocatedAddressesMap(address) = taskAmount - internalTaskAmount = 0 + taskAmount = 0 } } } - if (internalTaskAmount == 0 && allocatedAddressesMap.size > 0) { + if (taskAmount == 0 && allocatedAddressesMap.size > 0) { localTaskReqAssign.put(rName, new ResourceInformation(rName, allocatedAddressesMap.keys.toArray)) allocatedAddresses.put(rName, allocatedAddressesMap.toMap) From cd1c0ef8cb0a66b286d0e84865021e9db6b5343a Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Thu, 2 Nov 2023 16:31:19 +0800 Subject: [PATCH 05/18] keep Long internal --- .../spark/deploy/master/WorkerInfo.scala | 8 +-- .../spark/resource/ResourceAllocator.scala | 35 ++++++------ .../scheduler/ExecutorResourcesAmounts.scala | 50 ++++++++--------- .../spark/scheduler/TaskDescription.scala | 17 +++--- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 6 +-- .../cluster/CoarseGrainedClusterMessage.scala | 4 +- .../CoarseGrainedExecutorBackendSuite.scala | 13 +++-- .../CoarseGrainedSchedulerBackendSuite.scala | 5 +- .../scheduler/ExecutorResourceInfoSuite.scala | 29 +++++++--- .../ExecutorResourcesAmountsSuite.scala | 53 +++++++++++++------ .../scheduler/TaskDescriptionSuite.scala | 6 ++- .../scheduler/TaskSchedulerImplSuite.scala | 43 +++++++++------ 13 files changed, 162 insertions(+), 109 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 16aeb2aa8af3..a8b719590bb1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import scala.collection.mutable import org.apache.spark.resource.{ResourceAllocator, ResourceInformation, ResourceRequirement} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -35,13 +36,12 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String * @return ResourceInformation */ def acquire(amount: Int): ResourceInformation = { - // Any available address from availableAddrs must be a whole resource // since worker needs to do full resources to the executors. val addresses = availableAddrs.take(amount) assert(addresses.length == amount) - acquire(addresses.map(addr => addr -> 1.0).toMap) + acquire(addresses.map(addr => addr -> RESOURCE_TOTAL_AMOUNT).toMap) new ResourceInformation(resourceName, addresses.toArray) } } @@ -171,7 +171,7 @@ private[spark] class WorkerInfo( */ def recoverResources(expected: Map[String, ResourceInformation]): Unit = { expected.foreach { case (rName, rInfo) => - resources(rName).acquire(rInfo.addresses.map(addr => addr -> 1.0).toMap) + resources(rName).acquire(rInfo.addresses.map(addr => addr -> RESOURCE_TOTAL_AMOUNT).toMap) } } @@ -181,7 +181,7 @@ private[spark] class WorkerInfo( */ private def releaseResources(allocated: Map[String, ResourceInformation]): Unit = { allocated.foreach { case (rName, rInfo) => - resources(rName).release(rInfo.addresses.map(addrs => addrs -> 1.0).toMap) + resources(rName).release(rInfo.addresses.map(addrs => addrs -> RESOURCE_TOTAL_AMOUNT).toMap) } } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index ea8368b4fcdb..3a5aea292699 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -67,21 +67,19 @@ private[spark] trait ResourceAllocator { protected def resourceAddresses: Seq[String] /** - * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means - * the address is available, while value of 0 means the address is fully assigned. + * Map from an address to its availability default to 1.0 (we multiply RESOURCE_TOTAL_AMOUNT + * to avoid precision error), a value > 0 means the address is available, while value of + * 0 means the address is fully assigned. */ private lazy val addressAvailabilityMap = { mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*) } /** - * Get the resources and its amounts. + * Get the amounts of resources that have been multiplied by RESOURCE_TOTAL_AMOUNT. * @return the resources amounts */ - def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map { - case (address, internalAmount) => - address -> (internalAmount.toDouble / RESOURCE_TOTAL_AMOUNT) - }.toMap + def resourcesAmounts: Map[String, Long] = addressAvailabilityMap.toMap /** * Sequence of currently available resource addresses which are not fully assigned. @@ -100,19 +98,20 @@ private[spark] trait ResourceAllocator { * 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(addressesAmounts: Map[String, Double]): Unit = { + def acquire(addressesAmounts: Map[String, Long]): Unit = { addressesAmounts.foreach { case (address, amount) => val prevAmount = addressAvailabilityMap.getOrElse(address, throw new SparkException(s"Try to acquire an address that doesn't exist. $resourceName " + s"address $address doesn't exist.")) - val internalLeft = addressAvailabilityMap(address) - (amount * RESOURCE_TOTAL_AMOUNT).toLong + val left = addressAvailabilityMap(address) - amount - if (internalLeft < 0) { + if (left < 0) { throw new SparkException(s"Try to acquire $resourceName address $address " + - s"amount: $amount, but only ${prevAmount.toDouble / RESOURCE_TOTAL_AMOUNT} left.") + s"amount: ${amount.toDouble / RESOURCE_TOTAL_AMOUNT}, but only " + + s"${prevAmount.toDouble / RESOURCE_TOTAL_AMOUNT} left.") } else { - addressAvailabilityMap(address) = internalLeft + addressAvailabilityMap(address) = left } } } @@ -122,21 +121,21 @@ private[spark] trait ResourceAllocator { * addresses are released when a task has finished. * Throw an Exception if an address is not assigned or doesn't exist. */ - def release (addressesAmounts: Map[String, Double]): Unit = { + def release (addressesAmounts: Map[String, Long]): Unit = { addressesAmounts.foreach { case (address, amount) => val prevAmount = addressAvailabilityMap.getOrElse(address, throw new SparkException(s"Try to release an address that doesn't exist. $resourceName " + s"address $address doesn't exist.")) - val internalTotal = prevAmount + (amount * RESOURCE_TOTAL_AMOUNT).toLong + val total = prevAmount + amount - if (internalTotal > RESOURCE_TOTAL_AMOUNT) { + if (total > RESOURCE_TOTAL_AMOUNT) { throw new SparkException(s"Try to release $resourceName address $address " + - s"amount: $amount. But the total amount: " + - s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " + + s"amount: ${amount.toDouble / RESOURCE_TOTAL_AMOUNT}. But the total amount: " + + s"${total.toDouble / RESOURCE_TOTAL_AMOUNT} " + s"after release should be <= 1") } else { - addressAvailabilityMap(address) = internalTotal + addressAvailabilityMap(address) = total } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index 857a8347b6b6..0172738a833e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -32,22 +32,21 @@ import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT * One example is GPUs, where the addresses would be the indices of the GPUs * * @param resources The executor available resources and amount. eg, - * Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0), - * "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9) + * Map("gpu" -> Map("0" -> 0.2*RESOURCE_TOTAL_AMOUNT, + * "1" -> 1.0*RESOURCE_TOTAL_AMOUNT), + * "fpga" -> Map("a" -> 0.3*RESOURCE_TOTAL_AMOUNT, + * "b" -> 0.9*RESOURCE_TOTAL_AMOUNT) * ) */ private[spark] class ExecutorResourcesAmounts( - private val resources: Map[String, Map[String, Double]]) extends Serializable { + private val resources: Map[String, Map[String, Long]]) extends Serializable { /** - * Multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly. - * and convert the addressesAmounts to be mutable.HashMap + * convert the addressesAmounts to be mutable.HashMap */ private val internalResources: Map[String, HashMap[String, Long]] = { resources.map { case (rName, addressAmounts) => - rName -> HashMap(addressAmounts.map { case (address, amount) => - address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong - }.toSeq: _*) + rName -> HashMap(addressAmounts.toSeq: _*) } } @@ -76,7 +75,7 @@ private[spark] class ExecutorResourcesAmounts( * Acquire the resource and update the resource * @param assignedResource the assigned resource information */ - def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = { + def acquire(assignedResource: Map[String, Map[String, Long]]): Unit = { assignedResource.foreach { case (rName, taskResAmounts) => val availableResourceAmounts = internalResources.getOrElse(rName, throw new SparkException(s"Try to acquire an address from $rName that doesn't exist")) @@ -85,14 +84,12 @@ private[spark] class ExecutorResourcesAmounts( throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " + s"address $address doesn't exist.")) - val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong - val internalLeft = prevInternalTotalAmount - internalTaskAmount - val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT - if (realLeft < 0) { - throw new SparkException(s"The total amount ${realLeft} " + + val left = prevInternalTotalAmount - amount + if (left < 0) { + throw new SparkException(s"The total amount ${left.toDouble / RESOURCE_TOTAL_AMOUNT} " + s"after acquiring $rName address $address should be >= 0") } - internalResources(rName)(address) = internalLeft + internalResources(rName)(address) = left } } } @@ -101,7 +98,7 @@ private[spark] class ExecutorResourcesAmounts( * Release the assigned resources to the resource pool * @param assignedResource resource to be released */ - def release(assignedResource: Map[String, Map[String, Double]]): Unit = { + def release(assignedResource: Map[String, Map[String, Long]]): Unit = { assignedResource.foreach { case (rName, taskResAmounts) => val availableResourceAmounts = internalResources.getOrElse(rName, throw new SparkException(s"Try to release an address from $rName that doesn't exist")) @@ -109,14 +106,13 @@ private[spark] class ExecutorResourcesAmounts( val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address, throw new SparkException(s"Try to release an address that is not assigned. $rName " + s"address $address is not assigned.")) - val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong - val internalTotal = prevInternalTotalAmount + internalTaskAmount - if (internalTotal > RESOURCE_TOTAL_AMOUNT) { + val total = prevInternalTotalAmount + amount + if (total > RESOURCE_TOTAL_AMOUNT) { throw new SparkException(s"The total amount " + - s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " + + s"${total.toDouble / RESOURCE_TOTAL_AMOUNT} " + s"after releasing $rName address $address should be <= 1.0") } - internalResources(rName)(address) = internalTotal + internalResources(rName)(address) = total } } } @@ -129,7 +125,7 @@ private[spark] class ExecutorResourcesAmounts( * @return the resource */ def assignResources(taskSetProf: ResourceProfile): - Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = { + Option[(Map[String, ResourceInformation], Map[String, Map[String, Long]])] = { // only look at the resource other than cpus val tsResources = taskSetProf.getCustomTaskResources() @@ -138,7 +134,7 @@ private[spark] class ExecutorResourcesAmounts( } val localTaskReqAssign = HashMap[String, ResourceInformation]() - val allocatedAddresses = HashMap[String, Map[String, Double]]() + val allocatedAddresses = HashMap[String, Map[String, Long]]() // we go through all resources here so that we can make sure they match and also get what the // assignments are for the next task @@ -148,7 +144,7 @@ private[spark] class ExecutorResourcesAmounts( internalResources.get(rName) match { case Some(addressesAmountMap) => - val allocatedAddressesMap = HashMap[String, Double]() + val allocatedAddressesMap = HashMap[String, Long]() // always sort the addresses val addresses = addressesAmountMap.keys.toSeq.sorted @@ -160,15 +156,15 @@ private[spark] class ExecutorResourcesAmounts( if (addressesAmountMap(address) == RESOURCE_TOTAL_AMOUNT) { taskAmount -= 1.0 // Assign the full resource of the address - allocatedAddressesMap(address) = 1.0 + allocatedAddressesMap(address) = RESOURCE_TOTAL_AMOUNT } } } else if (taskAmount > 0.0) { // 0 < task.amount < 1.0 - val internalTaskAmount = taskAmount * RESOURCE_TOTAL_AMOUNT + val internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong for (address <- addresses if taskAmount > 0) { if (addressesAmountMap(address) >= internalTaskAmount) { // Assign the part of the address. - allocatedAddressesMap(address) = taskAmount + allocatedAddressesMap(address) = internalTaskAmount taskAmount = 0 } } 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 814aa5dc25a2..8953730f8592 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -59,8 +59,9 @@ private[spark] class TaskDescription( val cpus: Int, val resources: immutable.Map[String, ResourceInformation], // resourcesAmounts is the total resources assigned to the task - // Eg, Map("gpu" -> Map("0" -> 0.7)): assign 0.7 of the gpu address "0" to this task - val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]], + // Eg, Map("gpu" -> Map("0" -> 0.7*RESOURCE_TOTAL_AMOUNT)): + // assign 0.7 of the gpu address "0" to this task + val resourcesAmounts: immutable.Map[String, immutable.Map[String, Long]], val serializedTask: ByteBuffer) { assert(cpus > 0, "CPUs per task should be > 0") @@ -89,7 +90,7 @@ private[spark] object TaskDescription { } - private def serializeResourcesAmounts(map: immutable.Map[String, immutable.Map[String, Double]], + private def serializeResourcesAmounts(map: immutable.Map[String, immutable.Map[String, Long]], dataOut: DataOutputStream): Unit = { dataOut.writeInt(map.size) map.foreach { case (rName, addressAmountMap) => @@ -97,7 +98,7 @@ private[spark] object TaskDescription { dataOut.writeInt(addressAmountMap.size) addressAmountMap.foreach { case (address, amount) => dataOut.writeUTF(address) - dataOut.writeDouble(amount) + dataOut.writeLong(amount) } } } @@ -213,18 +214,18 @@ private[spark] object TaskDescription { } private def deserializeResourcesAmounts(dataIn: DataInputStream): - immutable.Map[String, immutable.Map[String, Double]] = { - val map = new HashMap[String, immutable.Map[String, Double]]() + immutable.Map[String, immutable.Map[String, Long]] = { + val map = new HashMap[String, immutable.Map[String, Long]]() val mapSize = dataIn.readInt() var i = 0 while (i < mapSize) { val resType = dataIn.readUTF() - val addressAmountMap = new HashMap[String, Double]() + val addressAmountMap = new HashMap[String, Long]() val addressAmountSize = dataIn.readInt() var j = 0 while (j < addressAmountSize) { val address = dataIn.readUTF() - val amount = dataIn.readDouble() + val amount = dataIn.readLong() addressAmountMap(address) = amount j += 1 } 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 1684bf7199f1..5063444dcf2e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -464,7 +464,7 @@ private[spark] class TaskSchedulerImpl( taskSet: TaskSetManager, availCpus: Int, availWorkerResources: ExecutorResourcesAmounts): Option[(Map[String, ResourceInformation], - Map[String, Map[String, Double]])] = { + Map[String, Map[String, Long]])] = { val rpId = taskSet.taskSet.resourceProfileId val taskSetProf = sc.resourceProfileManager.resourceProfileFromId(rpId) val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(taskSetProf, conf) 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 31bd10759717..852452d7c5a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -444,7 +444,7 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality, taskCpus: Int = sched.CPUS_PER_TASK, taskResourceAssignments: Map[String, ResourceInformation] = Map.empty, - taskResourceAmounts: Map[String, Map[String, Double]] = Map.empty) + taskResourceAmounts: Map[String, Map[String, Long]] = Map.empty) : (Option[TaskDescription], Boolean, Int) = { val offerExcluded = taskSetExcludelistHelperOpt.exists { excludeList => @@ -516,7 +516,7 @@ private[spark] class TaskSetManager( taskCpus: Int, taskResourceAssignments: Map[String, ResourceInformation], launchTime: Long, - resourcesAmounts: Map[String, Map[String, Double]]): TaskDescription = { + resourcesAmounts: Map[String, Map[String, Long]]): TaskDescription = { // Found a task; do some bookkeeping and return a task description val task = tasks(index) val taskId = sched.newTaskId() @@ -1385,7 +1385,7 @@ private[scheduler] case class BarrierPendingLaunchTask( index: Int, taskLocality: TaskLocality.TaskLocality, assignedResources: Map[String, ResourceInformation], - assignedResourcesAmount: Map[String, Map[String, Double]]) { + assignedResourcesAmount: Map[String, Map[String, Long]]) { // Stored the corresponding index of the WorkerOffer which is responsible to launch the task. // Used to revert the assigned resources (e.g., cores, custome resources) when the barrier // task set doesn't launch successfully in a single resourceOffers round. 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 1c0970c81636..80b5d6f9dadd 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 @@ -81,7 +81,7 @@ private[spark] object CoarseGrainedClusterMessages { data: SerializableBuffer, taskCpus: Int, resources: Map[String, ResourceInformation] = Map.empty, - resourcesAmounts: Map[String, Map[String, Double]] = Map.empty) + resourcesAmounts: Map[String, Map[String, Long]] = Map.empty) extends CoarseGrainedClusterMessage object StatusUpdate { @@ -93,7 +93,7 @@ private[spark] object CoarseGrainedClusterMessages { data: ByteBuffer, taskCpus: Int, resources: Map[String, ResourceInformation], - resourcesAmounts: Map[String, Map[String, Double]]): StatusUpdate = { + resourcesAmounts: Map[String, Map[String, Long]]): StatusUpdate = { StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), taskCpus, resources, resourcesAmounts) } 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 9a22c098baf6..281db5793c5c 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.TestUtils._ import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} import org.apache.spark.internal.config.PLUGINS import org.apache.spark.resource._ +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEnv @@ -307,7 +308,9 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(backend.taskResources.isEmpty) val taskId = 1000000L - val resourcesAmounts = Map(GPU -> Map("0" -> 0.15, "1" -> 0.76)) + val resourcesAmounts = Map(GPU -> Map( + "0" -> (0.15*RESOURCE_TOTAL_AMOUNT).toLong, + "1" -> (0.76*RESOURCE_TOTAL_AMOUNT).toLong)) // 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, @@ -428,7 +431,9 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val tasksKilled = new TrieMap[Long, Boolean]() val tasksExecuted = new TrieMap[Long, Boolean]() - val resourcesAmounts = Map(GPU -> Map("0" -> 0.15, "1" -> 0.76)) + val resourcesAmounts = Map(GPU -> Map( + "0" -> (0.15 * RESOURCE_TOTAL_AMOUNT).toLong, + "1" -> (0.76 * RESOURCE_TOTAL_AMOUNT).toLong)) // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { @@ -519,7 +524,9 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val tasksKilled = new TrieMap[Long, Boolean]() val tasksExecuted = new TrieMap[Long, Boolean]() - val resourcesAmounts = Map(GPU -> Map("0" -> 0.15, "1" -> 0.76)) + val resourcesAmounts = Map(GPU -> Map( + "0" -> (0.15*RESOURCE_TOTAL_AMOUNT).toLong, + "1" -> (0.76*RESOURCE_TOTAL_AMOUNT).toLong)) // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { 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 de115bd45453..801ab5eae6b1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -37,6 +37,7 @@ 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.resource.{ExecutorResourceRequests, ResourceInformation, ResourceProfile, TaskResourceRequests} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout} @@ -256,7 +257,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) val taskResourcesAmount = Map(GPU -> - taskResources(GPU).addresses.map(address => address -> 1.0).toMap) + taskResources(GPU).addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap) val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", @@ -367,7 +368,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) val taskResourcesAmount = Map(GPU -> - taskResources(GPU).addresses.map(address => address -> 1.0).toMap) + taskResources(GPU).addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap) val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", 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 db716e9c3b08..20a19a85eeb6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -18,12 +18,27 @@ package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceUtils.GPU class ExecutorResourceInfoSuite extends SparkFunSuite { + implicit def convertMapLongToDouble(resources: Map[String, Long]): Map[String, Double] = { + resources.map { case (k, v) => k -> v.toDouble / RESOURCE_TOTAL_AMOUNT }.toMap + } + + implicit def convertMapDoubleToLong(resources: Map[String, Double]): Map[String, Long] = { + resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong }.toMap + } + + implicit def convertResMapDoubleToMapLong(resources: Map[String, Map[String, Double]]): + Map[String, Map[String, Long]] = { + resources.map { case (k, v) => k -> convertMapDoubleToLong(v) } + } + test("Track Executor Resource information") { // Init Executor Resource. val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) @@ -51,7 +66,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { assert(!info.availableAddrs.contains("1")) // Acquire an address that is not available val e = intercept[SparkException] { - info.acquire(Map("1" -> 1.0)) + info.acquire(convertMapDoubleToLong(Map("1" -> 1.0))) } assert(e.getMessage.contains("Try to acquire gpu address 1 amount: 1.0, but only 0.0 left.")) } @@ -62,7 +77,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { assert(!info.availableAddrs.contains("4")) // Acquire an address that doesn't exist val e = intercept[SparkException] { - info.acquire(Map("4" -> 1.0)) + info.acquire(convertMapDoubleToLong(Map("4" -> 1.0))) } assert(e.getMessage.contains("Try to acquire an address that doesn't exist.")) } @@ -76,7 +91,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { assert(!info.assignedAddrs.contains("2")) // Release an address that is not assigned val e = intercept[SparkException] { - info.release(Map("2" -> 1.0)) + info.release(convertMapDoubleToLong(Map("2" -> 1.0))) } assert(e.getMessage.contains("Try to release gpu address 2 amount: 1.0. " + "But the total amount: 2.0 after release should be <= 1")) @@ -88,7 +103,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { assert(!info.assignedAddrs.contains("4")) // Release an address that doesn't exist val e = intercept[SparkException] { - info.release(Map("4" -> 1.0)) + info.release(convertMapDoubleToLong(Map("4" -> 1.0))) } assert(e.getMessage.contains("Try to release an address that doesn't exist.")) } @@ -100,17 +115,17 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { val taskAmount = 1.0 / slots val info = new ExecutorResourceInfo(GPU, addresses.toSeq) for (_ <- 0 until slots) { - addresses.foreach(addr => info.acquire(Map(addr -> taskAmount))) + addresses.foreach(addr => info.acquire(convertMapDoubleToLong(Map(addr -> taskAmount)))) } // All addresses has been assigned assert(info.resourcesAmounts.values.toSeq.toSet.size == 1) // The left amount of any address should < taskAmount - assert(info.resourcesAmounts("0") < taskAmount) + assert(info.resourcesAmounts("0").toDouble/RESOURCE_TOTAL_AMOUNT < taskAmount) addresses.foreach { addr => assertThrows[SparkException] { - info.acquire(Map(addr -> taskAmount)) + info.acquire(convertMapDoubleToLong(Map(addr -> taskAmount))) } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala index 21a73f5125b6..592bc54fcb05 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala @@ -18,13 +18,28 @@ package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceUtils.GPU class ExecutorResourcesAmountsSuite extends SparkFunSuite { + implicit def convertMapLongToMapDouble(resources: Map[String, Long]): Map[String, Double] = { + resources.map { case (k, v) => k -> v.toDouble / RESOURCE_TOTAL_AMOUNT } + } + + implicit def convertMapDoubleToLong(resources: Map[String, Double]): Map[String, Long] = { + resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong } + } + + implicit def convertResMapDoubleToMapLong(resources: Map[String, Map[String, Double]]): + Map[String, Map[String, Long]] = { + resources.map { case (k, v) => k -> convertMapDoubleToLong(v) } + } + def compareMaps(lhs: Map[String, Double], rhs: Map[String, Double], eps: Double = 0.00000001): Boolean = { lhs.size == rhs.size && @@ -82,9 +97,9 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // executors info shouldn't be changed. executorsInfo.foreach { case (rName, rInfo) => if (rName == "gpu") { - rInfo.acquire(Map("2" -> 0.4, "6" -> 0.6)) + rInfo.acquire(convertMapDoubleToLong(Map("2" -> 0.4, "6" -> 0.6))) } else { - rInfo.acquire(Map("aa" -> 0.2, "bb" -> 0.7)) + rInfo.acquire(convertMapDoubleToLong(Map("aa" -> 0.2, "bb" -> 0.7))) } } @@ -203,11 +218,11 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) val e = intercept[SparkException] { - availableExecResAmounts.release(Map("gpu" -> Map("2" -> 0.7))) + availableExecResAmounts.release(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.7)))) } assert(e.getMessage.contains("after releasing gpu address 2 should be <= 1.0")) - availableExecResAmounts.release(Map("gpu" -> Map("2" -> 0.6))) + availableExecResAmounts.release(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.6)))) assert(compareMaps(availableExecResAmounts.availableResources("gpu"), Map("2" -> 1.0))) } @@ -216,11 +231,11 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) val e = intercept[SparkException] { - availableExecResAmounts.acquire(Map("gpu" -> Map("2" -> 0.6))) + availableExecResAmounts.acquire(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.6)))) } assert(e.getMessage.contains("after acquiring gpu address 2 should be >= 0")) - availableExecResAmounts.acquire(Map("gpu" -> Map("2" -> 0.4))) + availableExecResAmounts.acquire(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.4)))) assert(compareMaps(availableExecResAmounts.availableResources("gpu"), Map("2" -> 0.0))) } @@ -234,7 +249,8 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = ExecutorResourcesAmounts(Map(GPU -> info)) for (_ <- 0 until slots) { addresses.foreach(addr => - availableExecResAmounts.acquire(Map(GPU -> Map(addr -> taskAmount)))) + availableExecResAmounts.acquire( + Map(GPU -> convertMapDoubleToLong(Map(addr -> taskAmount))))) } assert(availableExecResAmounts.availableResources.size === 1) @@ -245,7 +261,8 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { addresses.foreach { addr => assertThrows[SparkException] { - availableExecResAmounts.acquire(Map(GPU -> Map(addr -> taskAmount))) + availableExecResAmounts.acquire( + Map(GPU -> convertMapDoubleToLong(Map(addr -> taskAmount)))) } } } @@ -278,7 +295,8 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { assert(resourceAmounts.keys.toSeq === Seq("gpu")) assert(resourceAmounts("gpu").size === 1) assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) - assert(resourceAmounts("gpu")(resource("gpu").addresses(0)) === gpuTaskAmount) + assert(resourceAmounts("gpu")(resource("gpu").addresses(0)).toDouble/RESOURCE_TOTAL_AMOUNT === + gpuTaskAmount) } // assign will not update the real value. @@ -355,11 +373,13 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { assert(resourceAmounts("gpu").size === 1) assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) - assert(resourceAmounts("gpu")(resource("gpu").addresses(0)) === gpuTaskAmount) + assert(resourceAmounts("gpu")(resource("gpu").addresses(0)).toDouble/RESOURCE_TOTAL_AMOUNT + === gpuTaskAmount) assert(resourceAmounts("fpga").size === 1) assert(resourceAmounts("fpga").keys.toSeq === Seq("aa")) - assert(resourceAmounts("fpga")(resource("fpga").addresses(0)) === fpgaTaskAmount) + assert(resourceAmounts("fpga")(resource("fpga").addresses(0)).toDouble/RESOURCE_TOTAL_AMOUNT + === fpgaTaskAmount) } // assign will not update the real value. @@ -414,7 +434,8 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { assert(!assigned.isEmpty) assigned.foreach { case (resource, resourceAmounts) => assert(resource("gpu").addresses.sorted === expectedAssignedAddress.sorted) - assert(resourceAmounts("gpu").values.toArray.sorted === expectedAssignedAmount.sorted) + assert(resourceAmounts("gpu").values.toArray.sorted.map(_.toDouble/RESOURCE_TOTAL_AMOUNT) + === expectedAssignedAmount.sorted) availableExecResAmounts.acquire(resourceAmounts) @@ -513,13 +534,13 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // Acquire an address from a resource that doesn't exist val e = intercept[SparkException] { - availableExecResAmounts.acquire(Map("fpga" -> Map("1" -> 1.0))) + availableExecResAmounts.acquire(Map("fpga" -> convertMapDoubleToLong(Map("1" -> 1.0)))) } assert(e.getMessage.contains("Try to acquire an address from fpga that doesn't exist")) // Acquire an address that is not available val e1 = intercept[SparkException] { - availableExecResAmounts.acquire(Map("gpu" -> Map("6" -> 1.0))) + availableExecResAmounts.acquire(Map("gpu" -> convertMapDoubleToLong(Map("6" -> 1.0)))) } assert(e1.getMessage.contains("Try to acquire an address that doesn't exist")) } @@ -531,13 +552,13 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // Acquire an address from a resource that doesn't exist val e = intercept[SparkException] { - availableExecResAmounts.release(Map("fpga" -> Map("1" -> 0.1))) + availableExecResAmounts.release(Map("fpga" -> convertMapDoubleToLong(Map("1" -> 0.1)))) } assert(e.getMessage.contains("Try to release an address from fpga that doesn't exist")) // Acquire an address that is not available val e1 = intercept[SparkException] { - availableExecResAmounts.release(Map("gpu" -> Map("6" -> 0.1))) + availableExecResAmounts.release(Map("gpu" -> convertMapDoubleToLong(Map("6" -> 0.1)))) } assert(e1.getMessage.contains("Try to release an address that is not assigned")) } 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 402af94dc14c..448f865bf62e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler + import java.io.{ByteArrayOutputStream, DataOutputStream, UTFDataFormatException} import java.nio.ByteBuffer import java.util.Properties @@ -24,6 +25,7 @@ import java.util.Properties import scala.collection.mutable.HashMap import org.apache.spark.{JobArtifactSet, SparkFunSuite} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceInformation import org.apache.spark.resource.ResourceUtils.GPU @@ -62,7 +64,9 @@ class TaskDescriptionSuite extends SparkFunSuite { val originalResources = Map(GPU -> new ResourceInformation(GPU, Array("1", "2", "3"))) - val originalResourcesAmounts = Map(GPU -> Map("1" -> 0.2, "2" -> 0.5, "3" -> 0.1)) + val originalResourcesAmounts = Map(GPU -> Map("1" -> (0.2*RESOURCE_TOTAL_AMOUNT).toLong, + "2" -> (0.5*RESOURCE_TOTAL_AMOUNT).toLong, + "3" -> (0.1*RESOURCE_TOTAL_AMOUNT).toLong)) // 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/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 35874bbc4805..56f3b2e9c880 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -36,6 +36,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceProfile, TaskResourceRequests} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.status.api.v1.ThreadStackTrace @@ -152,7 +153,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext ExecutorResourcesAmounts = { // convert the old resources to ExecutorResourcesAmounts new ExecutorResourcesAmounts(resources.map { case (rName, addresses) => - rName -> addresses.map(address => address -> 1.0).toMap + rName -> addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap }) } @@ -2294,6 +2295,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.handleFailedTask(tsm, tid, state, reason) } + private implicit def convertMapDoubleToLong(resources: Map[String, Double]): Map[String, Long] = { + resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong } + } + Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + s"restrict each task takes 1 gpu exclusively in the same executor") { @@ -2310,7 +2315,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val taskSet = FakeTask.createTaskSet(100) val resources = new ExecutorResourcesAmounts( - Map(GPU -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))) + Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", executorCpus, None, resources)) @@ -2346,13 +2351,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val workerOffers = IndexedSeq( new WorkerOffer("executor0", "host0", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> Map("0" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0))))), new WorkerOffer("executor1", "host1", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> Map("7" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("7" -> 1.0))))), new WorkerOffer("executor2", "host2", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> Map("9" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("9" -> 1.0))))), new WorkerOffer("executor3", "host3", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> Map("20" -> 1.0))))) + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("20" -> 1.0)))))) taskScheduler.submitTasks(taskSet) // Launch tasks on executor that satisfies resource requirements. @@ -2406,7 +2411,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext IndexedSeq( // cpu won't be a problem WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))) ) taskScheduler.submitTasks(lowerTaskSet) @@ -2421,10 +2426,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(tDesc.resources.get(GPU).get.addresses.length == 1) if (index < 4) { // the first 4 tasks will grab 0.7 gpu assert(tDesc.resources.get(GPU).get.addresses(0) == index.toString) - assert(tDesc.resourcesAmounts.get(GPU).get(index.toString) == 0.7) + assert(tDesc.resourcesAmounts.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT + == 0.7) } else { assert(tDesc.resources.get(GPU).get.addresses(0) == (index - 4).toString) - assert(tDesc.resourcesAmounts.get(GPU).get((index - 4).toString) == 0.3) + assert( + tDesc.resourcesAmounts.get(GPU).get((index - 4).toString).toDouble/RESOURCE_TOTAL_AMOUNT + == 0.3) } index += 1 } @@ -2459,13 +2467,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext IndexedSeq( // cpu won't be a problem WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> Map("0" -> 1.0)))), + Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0))))), WorkerOffer("executor1", "host1", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> Map("1" -> 1.0)))), + Map(GPU -> convertMapDoubleToLong(Map("1" -> 1.0))))), WorkerOffer("executor2", "host2", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> Map("2" -> 1.0)))), + Map(GPU -> convertMapDoubleToLong(Map("2" -> 1.0))))), WorkerOffer("executor3", "host3", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> Map("3" -> 1.0)))) + Map(GPU -> convertMapDoubleToLong(Map("3" -> 1.0))))) ) taskScheduler.submitTasks(lowerTaskSet) @@ -2489,11 +2497,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext if (index % 2 == 0) { higherAssignedExecutorsGpus.append( (tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) - assert(tDesc.resourcesAmounts.get(GPU).get(address) == 0.7) + assert(tDesc.resourcesAmounts.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) } else { lowerAssignedExecutorsGpus.append( (tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) - assert(tDesc.resourcesAmounts.get(GPU).get(address) == 0.3) + assert(tDesc.resourcesAmounts.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.3) } index += 1 } @@ -2535,7 +2543,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext IndexedSeq( // cpu won't be a problem WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))) ) taskScheduler.submitTasks(lowerTaskSet) @@ -2549,7 +2557,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(tDesc.resources.contains(GPU)) assert(tDesc.resources.get(GPU).get.addresses.length == 1) assert(tDesc.resources.get(GPU).get.addresses(0) == index.toString) - assert(tDesc.resourcesAmounts.get(GPU).get(index.toString) == 0.7) + assert(tDesc.resourcesAmounts.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT + == 0.7) index += 1 } } From 51ac5cb236a3594bfc805992dc414bbf41074e4d Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 6 Nov 2023 10:35:17 +0800 Subject: [PATCH 06/18] remove duplicated resources --- .../CoarseGrainedExecutorBackend.scala | 10 +-- .../org/apache/spark/executor/Executor.scala | 6 +- .../scheduler/ExecutorResourceInfo.scala | 1 - .../scheduler/ExecutorResourcesAmounts.scala | 29 ++++---- .../spark/scheduler/TaskDescription.scala | 51 ++----------- .../spark/scheduler/TaskSchedulerImpl.scala | 27 ++++--- .../spark/scheduler/TaskSetManager.scala | 19 ++--- .../cluster/CoarseGrainedClusterMessage.scala | 9 +-- .../CoarseGrainedSchedulerBackend.scala | 6 +- .../CoarseGrainedExecutorBackendSuite.scala | 13 ++-- .../apache/spark/executor/ExecutorSuite.scala | 3 +- .../CoarseGrainedSchedulerBackendSuite.scala | 22 ++---- .../scheduler/ExecutorResourceInfoSuite.scala | 5 -- .../ExecutorResourcesAmountsSuite.scala | 59 +++++---------- .../scheduler/TaskDescriptionSuite.scala | 26 ++----- .../scheduler/TaskSchedulerImplSuite.scala | 71 ++++++++++--------- .../spark/scheduler/TaskSetManagerSuite.scala | 10 ++- 17 files changed, 133 insertions(+), 234 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 44cce1c28ce9..34f530ae0cad 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -191,7 +191,10 @@ private[spark] class CoarseGrainedExecutorBackend( } else { val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) - taskResources.put(taskDesc.taskId, taskDesc.resources) + // Convert resources amounts into ResourceInformation + val resources = taskDesc.resources.map { case (rName, addressesAmounts) => + rName -> new ResourceInformation(rName, addressesAmounts.keys.toSeq.sorted.toArray)} + taskResources.put(taskDesc.taskId, resources) executor.launchTask(this, taskDesc) } @@ -271,10 +274,9 @@ private[spark] class CoarseGrainedExecutorBackend( } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = { - val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation]) + val resources = executor.runningTasks.get(taskId).taskDescription.resources val cpus = executor.runningTasks.get(taskId).taskDescription.cpus - val resourcesAmounts = executor.runningTasks.get(taskId).taskDescription.resourcesAmounts - val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources, resourcesAmounts) + val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources) if (TaskState.isFinished(state)) { taskResources.remove(taskId) } 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 b12b5e213121..516956bd08c2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -610,13 +610,17 @@ private[spark] class Executor( threadMXBean.getCurrentThreadCpuTime } else 0L var threwException = true + // Convert resources amounts info to ResourceInformation + val resources = taskDescription.resources.map { case (rName, addressesAmounts) => + rName -> new ResourceInformation(rName, addressesAmounts.keys.toSeq.sorted.toArray) + } val value = Utils.tryWithSafeFinally { val res = task.run( taskAttemptId = taskId, attemptNumber = taskDescription.attemptNumber, metricsSystem = env.metricsSystem, cpus = taskDescription.cpus, - resources = taskDescription.resources, + resources = resources, plugins = plugins) threwException = false res 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 9a8461d2fccc..d9fbd23f3aa4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -32,7 +32,6 @@ private[spark] class ExecutorResourceInfo( extends ResourceInformation(name, addresses.toArray) with ResourceAllocator { override protected def resourceName = this.name - override protected def resourceAddresses = this.addresses /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index 0172738a833e..946b289f7b1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -20,8 +20,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap import org.apache.spark.SparkException -import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceProfile /** * Class to hold information about a series of resources belonging to an executor. @@ -33,16 +33,16 @@ import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT * * @param resources The executor available resources and amount. eg, * Map("gpu" -> Map("0" -> 0.2*RESOURCE_TOTAL_AMOUNT, - * "1" -> 1.0*RESOURCE_TOTAL_AMOUNT), + * "1" -> 1.0*RESOURCE_TOTAL_AMOUNT), * "fpga" -> Map("a" -> 0.3*RESOURCE_TOTAL_AMOUNT, - * "b" -> 0.9*RESOURCE_TOTAL_AMOUNT) + * "b" -> 0.9*RESOURCE_TOTAL_AMOUNT) * ) */ private[spark] class ExecutorResourcesAmounts( private val resources: Map[String, Map[String, Long]]) extends Serializable { /** - * convert the addressesAmounts to be mutable.HashMap + * convert the resources to be mutable HashMap */ private val internalResources: Map[String, HashMap[String, Long]] = { resources.map { case (rName, addressAmounts) => @@ -52,8 +52,11 @@ private[spark] class ExecutorResourcesAmounts( /** * The total address count of each resource. Eg, - * Map("gpu" -> Map("0" -> 0.5, "1" -> 0.5, "2" -> 0.5), - * "fpga" -> Map("a" -> 0.5, "b" -> 0.5)) + * Map("gpu" -> Map("0" -> 0.5 * RESOURCE_TOTAL_AMOUNT, + * "1" -> 0.5 * RESOURCE_TOTAL_AMOUNT, + * "2" -> 0.5 * RESOURCE_TOTAL_AMOUNT), + * "fpga" -> Map("a" -> 0.5 * RESOURCE_TOTAL_AMOUNT, + * "b" -> 0.5 * RESOURCE_TOTAL_AMOUNT)) * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2) */ lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) => @@ -118,22 +121,20 @@ private[spark] class ExecutorResourcesAmounts( } /** - * Try to assign the address according to the task requirement. + * Try to assign the addresses according to the task requirement. * Please note that this function will not update the values. * * @param taskSetProf assign resources based on which resource profile - * @return the resource + * @return the optional resources amounts */ - def assignResources(taskSetProf: ResourceProfile): - Option[(Map[String, ResourceInformation], Map[String, Map[String, Long]])] = { + def assignResources(taskSetProf: ResourceProfile): Option[Map[String, Map[String, Long]]] = { // only look at the resource other than cpus val tsResources = taskSetProf.getCustomTaskResources() if (tsResources.isEmpty) { - return Some(Map.empty, Map.empty) + return Some(Map.empty) } - val localTaskReqAssign = HashMap[String, ResourceInformation]() val allocatedAddresses = HashMap[String, Map[String, Long]]() // we go through all resources here so that we can make sure they match and also get what the @@ -171,15 +172,13 @@ private[spark] class ExecutorResourcesAmounts( } if (taskAmount == 0 && allocatedAddressesMap.size > 0) { - localTaskReqAssign.put(rName, new ResourceInformation(rName, - allocatedAddressesMap.keys.toArray)) allocatedAddresses.put(rName, allocatedAddressesMap.toMap) } else return None case None => return None } } - Some(localTaskReqAssign.toMap, allocatedAddresses.toMap) + Some(allocatedAddresses.toMap) } } 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 8953730f8592..459865d7a166 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -23,11 +23,10 @@ import java.nio.charset.StandardCharsets import java.util.Properties import scala.collection.immutable -import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import scala.collection.mutable.{HashMap, Map} import scala.jdk.CollectionConverters._ import org.apache.spark.{JobArtifactSet, JobArtifactState} -import org.apache.spark.resource.ResourceInformation import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} /** @@ -57,11 +56,10 @@ private[spark] class TaskDescription( val artifacts: JobArtifactSet, val properties: Properties, val cpus: Int, - val resources: immutable.Map[String, ResourceInformation], - // resourcesAmounts is the total resources assigned to the task + // resources is the total resources assigned to the task // Eg, Map("gpu" -> Map("0" -> 0.7*RESOURCE_TOTAL_AMOUNT)): // assign 0.7 of the gpu address "0" to this task - val resourcesAmounts: immutable.Map[String, immutable.Map[String, Long]], + val resources: immutable.Map[String, immutable.Map[String, Long]], val serializedTask: ByteBuffer) { assert(cpus > 0, "CPUs per task should be > 0") @@ -78,19 +76,7 @@ private[spark] object TaskDescription { } } - private def serializeResources(map: immutable.Map[String, ResourceInformation], - dataOut: DataOutputStream): Unit = { - dataOut.writeInt(map.size) - map.foreach { case (key, value) => - dataOut.writeUTF(key) - dataOut.writeUTF(value.name) - dataOut.writeInt(value.addresses.size) - value.addresses.foreach(dataOut.writeUTF(_)) - } - } - - - private def serializeResourcesAmounts(map: immutable.Map[String, immutable.Map[String, Long]], + private def serializeResources(map: immutable.Map[String, immutable.Map[String, Long]], dataOut: DataOutputStream): Unit = { dataOut.writeInt(map.size) map.foreach { case (rName, addressAmountMap) => @@ -133,9 +119,6 @@ private[spark] object TaskDescription { // Write resources. serializeResources(taskDescription.resources, dataOut) - // Write resourcesAmounts. - serializeResourcesAmounts(taskDescription.resourcesAmounts, dataOut) - // Write the task. The task is already serialized, so write it directly to the byte buffer. Utils.writeByteBuffer(taskDescription.serializedTask, bytesOut) @@ -193,27 +176,6 @@ private[spark] object TaskDescription { } private def deserializeResources(dataIn: DataInputStream): - immutable.Map[String, ResourceInformation] = { - val map = new HashMap[String, ResourceInformation]() - val mapSize = dataIn.readInt() - var i = 0 - while (i < mapSize) { - val resType = dataIn.readUTF() - val name = dataIn.readUTF() - val numIdentifier = dataIn.readInt() - val identifiers = new ArrayBuffer[String](numIdentifier) - var j = 0 - while (j < numIdentifier) { - identifiers += dataIn.readUTF() - j += 1 - } - map(resType) = new ResourceInformation(name, identifiers.toArray) - i += 1 - } - map.toMap - } - - private def deserializeResourcesAmounts(dataIn: DataInputStream): immutable.Map[String, immutable.Map[String, Long]] = { val map = new HashMap[String, immutable.Map[String, Long]]() val mapSize = dataIn.readInt() @@ -264,13 +226,10 @@ private[spark] object TaskDescription { // Read resources. val resources = deserializeResources(dataIn) - // Read resources. - val resourcesAmounts = deserializeResourcesAmounts(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, artifacts, - properties, cpus, resources, resourcesAmounts, serializedTask) + properties, cpus, 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 5063444dcf2e..822911a48beb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -35,7 +35,7 @@ import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ -import org.apache.spark.resource.{ResourceInformation, ResourceProfile} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcEndpoint import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality @@ -406,31 +406,30 @@ private[spark] class TaskSchedulerImpl( .canBeScheduled(taskSetRpID, shuffledOffers(i).resourceProfileId)) { val taskResAssignmentsOpt = resourcesMeetTaskRequirements(taskSet, availableCpus(i), availableResources(i)) - taskResAssignmentsOpt.foreach { case (taskResAssignments, taskResAmounts) => + taskResAssignmentsOpt.foreach { taskResAssignments => try { val prof = sc.resourceProfileManager.resourceProfileFromId(taskSetRpID) val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(prof, conf) val (taskDescOption, didReject, index) = - taskSet.resourceOffer(execId, host, maxLocality, taskCpus, taskResAssignments, - taskResAmounts) + taskSet.resourceOffer(execId, host, maxLocality, taskCpus, taskResAssignments) noDelayScheduleRejects &= !didReject for (task <- taskDescOption) { - val (locality, resourcesAmounts) = if (task != null) { + val (locality, resources) = if (task != null) { tasks(i) += task addRunningTask(task.taskId, execId, taskSet) - (taskSet.taskInfos(task.taskId).taskLocality, task.resourcesAmounts) + (taskSet.taskInfos(task.taskId).taskLocality, task.resources) } else { assert(taskSet.isBarrier, "TaskDescription can only be null for barrier task") val barrierTask = taskSet.barrierPendingLaunchTasks(index) barrierTask.assignedOfferIndex = i barrierTask.assignedCores = taskCpus - (barrierTask.taskLocality, barrierTask.assignedResourcesAmount) + (barrierTask.taskLocality, barrierTask.assignedResources) } minLaunchedLocality = minTaskLocality(minLaunchedLocality, Some(locality)) availableCpus(i) -= taskCpus assert(availableCpus(i) >= 0) - availableResources(i).acquire(resourcesAmounts) + availableResources(i).acquire(resources) } } catch { case e: TaskNotSerializableException => @@ -457,14 +456,13 @@ private[spark] class TaskSchedulerImpl( /** * Check whether the resources from the WorkerOffer are enough to run at least one task. * Returns None if the resources don't meet the task requirements, otherwise returns - * the task resource assignments and the resource amounts to give to the next task. - * Note that the assignments maybe be empty if no custom resources are used. + * the task resource assignments to give to the next task. Note that the assignments maybe + * be empty if no custom resources are used. */ private def resourcesMeetTaskRequirements( taskSet: TaskSetManager, availCpus: Int, - availWorkerResources: ExecutorResourcesAmounts): Option[(Map[String, ResourceInformation], - Map[String, Map[String, Long]])] = { + availWorkerResources: ExecutorResourcesAmounts): Option[Map[String, Map[String, Long]]] = { val rpId = taskSet.taskSet.resourceProfileId val taskSetProf = sc.resourceProfileManager.resourceProfileFromId(rpId) val taskCpus = ResourceProfile.getTaskCpusOrDefaultForProfile(taskSetProf, conf) @@ -688,7 +686,7 @@ private[spark] class TaskSchedulerImpl( // revert all assigned resources availableCpus(task.assignedOfferIndex) += task.assignedCores availableResources(task.assignedOfferIndex).release( - task.assignedResourcesAmount) + task.assignedResources) // re-add the task to the schedule pending list taskSet.addPendingTask(task.index) } @@ -706,8 +704,7 @@ private[spark] class TaskSchedulerImpl( false, task.assignedCores, task.assignedResources, - launchTime, - task.assignedResourcesAmount) + launchTime) addRunningTask(taskDesc.taskId, taskDesc.executorId, taskSet) tasks(task.assignedOfferIndex) += taskDesc shuffledOffers(task.assignedOfferIndex).address.get -> taskDesc 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 852452d7c5a6..e6fdddbb6c9d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -32,7 +32,6 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ -import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.util.{AccumulatorV2, Clock, LongAccumulator, SystemClock, Utils} import org.apache.spark.util.collection.PercentileHeap @@ -443,8 +442,7 @@ private[spark] class TaskSetManager( host: String, maxLocality: TaskLocality.TaskLocality, taskCpus: Int = sched.CPUS_PER_TASK, - taskResourceAssignments: Map[String, ResourceInformation] = Map.empty, - taskResourceAmounts: Map[String, Map[String, Long]] = Map.empty) + taskResourceAssignments: Map[String, Map[String, Long]] = Map.empty) : (Option[TaskDescription], Boolean, Int) = { val offerExcluded = taskSetExcludelistHelperOpt.exists { excludeList => @@ -479,8 +477,7 @@ private[spark] class TaskSetManager( host, index, taskLocality, - taskResourceAssignments, - taskResourceAmounts) + taskResourceAssignments) // return null since the TaskDescription for the barrier task is not ready yet null } else { @@ -492,8 +489,7 @@ private[spark] class TaskSetManager( speculative, taskCpus, taskResourceAssignments, - curTime, - taskResourceAmounts) + curTime) } } val hasPendingTasks = pendingTasks.all.nonEmpty || pendingSpeculatableTasks.all.nonEmpty @@ -514,9 +510,8 @@ private[spark] class TaskSetManager( taskLocality: TaskLocality.Value, speculative: Boolean, taskCpus: Int, - taskResourceAssignments: Map[String, ResourceInformation], - launchTime: Long, - resourcesAmounts: Map[String, Map[String, Long]]): TaskDescription = { + taskResourceAssignments: Map[String, Map[String, Long]], + launchTime: Long): TaskDescription = { // Found a task; do some bookkeeping and return a task description val task = tasks(index) val taskId = sched.newTaskId() @@ -570,7 +565,6 @@ private[spark] class TaskSetManager( task.localProperties, taskCpus, taskResourceAssignments, - resourcesAmounts, serializedTask) } @@ -1384,8 +1378,7 @@ private[scheduler] case class BarrierPendingLaunchTask( host: String, index: Int, taskLocality: TaskLocality.TaskLocality, - assignedResources: Map[String, ResourceInformation], - assignedResourcesAmount: Map[String, Map[String, Long]]) { + assignedResources: Map[String, Map[String, Long]]) { // Stored the corresponding index of the WorkerOffer which is responsible to launch the task. // Used to revert the assigned resources (e.g., cores, custome resources) when the barrier // task set doesn't launch successfully in a single resourceOffers round. 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 80b5d6f9dadd..1f452ae7d109 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 @@ -80,8 +80,7 @@ private[spark] object CoarseGrainedClusterMessages { state: TaskState, data: SerializableBuffer, taskCpus: Int, - resources: Map[String, ResourceInformation] = Map.empty, - resourcesAmounts: Map[String, Map[String, Long]] = Map.empty) + resources: Map[String, Map[String, Long]] = Map.empty) extends CoarseGrainedClusterMessage object StatusUpdate { @@ -92,10 +91,8 @@ private[spark] object CoarseGrainedClusterMessages { state: TaskState, data: ByteBuffer, taskCpus: Int, - resources: Map[String, ResourceInformation], - resourcesAmounts: Map[String, Map[String, Long]]): StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), taskCpus, resources, - resourcesAmounts) + resources: Map[String, Map[String, Long]]): StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), taskCpus, 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 39a7e3cdbfd2..853b4cea2770 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 @@ -165,13 +165,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } override def receive: PartialFunction[Any, Unit] = { - case StatusUpdate(executorId, taskId, state, data, taskCpus, resources, resourcesAmounts) => + case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { executorDataMap.get(executorId) match { case Some(executorInfo) => executorInfo.freeCores += taskCpus - resourcesAmounts.foreach { case (rName, addressAmount) => + resources.foreach { case (rName, addressAmount) => addressAmount.foreach { case (address, amount) => executorInfo.resourcesInfo.get(rName).foreach { r => r.release(Map(address -> amount)) @@ -440,7 +440,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Do resources allocation here. The allocated resources will get released after the task // finishes. executorData.freeCores -= task.cpus - task.resourcesAmounts.foreach { case (rName, addressAmounts) => + task.resources.foreach { case (rName, addressAmounts) => addressAmounts.foreach { case (address, amount) => executorData.resourcesInfo(rName).acquire(Map(address -> amount)) } 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 281db5793c5c..f82ea652a5e5 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -314,8 +314,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // 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, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, - Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), resourcesAmounts, data) + 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, resourcesAmounts, data) val serializedTaskDescription = TaskDescription.encode(taskDescription) backend.rpcEnv.setupEndpoint("Executor 1", backend) backend.executor = mock[Executor](CALLS_REAL_METHODS) @@ -349,7 +348,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(backend.taskResources.size == 1) val resources = backend.taskResources.get(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) - assert(executor.runningTasks.get(taskId).taskDescription.resourcesAmounts + assert(executor.runningTasks.get(taskId).taskDescription.resources === resourcesAmounts) } @@ -358,7 +357,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(backend.taskResources.size == 1) val resources = backend.taskResources.get(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) - assert(executor.runningTasks.get(taskId).taskDescription.resourcesAmounts + assert(executor.runningTasks.get(taskId).taskDescription.resources === resourcesAmounts) // Update the status of a finished task shall remove the entry from `taskResources` map. @@ -438,8 +437,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { taskId => new TaskDescription(taskId, 2, "1", s"TASK $taskId", 19, - 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, - Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), resourcesAmounts, data) + 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, resourcesAmounts, data) } assert(taskDescriptions.length == numTasks) @@ -531,8 +529,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { taskId => new TaskDescription(taskId, 2, "1", s"TASK $taskId", 19, - 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, - Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), resourcesAmounts, data) + 1, JobArtifactSet.emptyJobArtifactSet, new Properties, 1, resourcesAmounts, data) } assert(taskDescriptions.length == numTasks) 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 a5bfb679086b..805e7ca46749 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -642,8 +642,7 @@ class ExecutorSuite extends SparkFunSuite JobArtifactSet.emptyJobArtifactSet, properties = new Properties, cpus = 1, - resources = immutable.Map[String, ResourceInformation](), - Map.empty, + resources = Map.empty, 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 801ab5eae6b1..04e50f0e5f5c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -255,14 +255,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") assert(exec3ResourceProfileId === rp.id) - val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) - val taskResourcesAmount = Map(GPU -> - taskResources(GPU).addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap) - + val taskResources = Map(GPU -> Map("0" -> RESOURCE_TOTAL_AMOUNT)) val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), - taskCpus, taskResources, taskResourcesAmount, bytebuffer))) + taskCpus, taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) @@ -278,8 +275,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo // make sure that `availableAddrs` below won't change when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(Seq.empty) backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources, - taskResourcesAmount)) + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources)) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") @@ -366,14 +362,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") assert(exec3ResourceProfileId === rp.id) - val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) - val taskResourcesAmount = Map(GPU -> - taskResources(GPU).addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap) - + val taskResources = Map(GPU -> Map("0" -> RESOURCE_TOTAL_AMOUNT)) val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), - taskCpus, taskResources, taskResourcesAmount, bytebuffer))) + taskCpus, taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) @@ -389,8 +382,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo // make sure that `availableAddrs` below won't change when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(Seq.empty) backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources, - taskResourcesAmount)) + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources)) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") @@ -467,7 +459,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val taskCpus = 2 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), - taskCpus, Map.empty, Map.empty, bytebuffer))) + taskCpus, Map.empty, bytebuffer))) when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]], any[Boolean])).thenReturn(taskDescs) backend.driverEndpoint.send(ReviveOffers) 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 20a19a85eeb6..1614101a955c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -34,11 +34,6 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong }.toMap } - implicit def convertResMapDoubleToMapLong(resources: Map[String, Map[String, Double]]): - Map[String, Map[String, Long]] = { - resources.map { case (k, v) => k -> convertMapDoubleToLong(v) } - } - test("Track Executor Resource information") { // Init Executor Resource. val info = new ExecutorResourceInfo(GPU, Seq("0", "1", "2", "3")) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala index 592bc54fcb05..e0eddd4f3b52 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala @@ -61,10 +61,8 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // assign nothing to rp without resource profile val assigned = availableExecResAmounts.assignResources(rp) - assigned.foreach { case (resource, resourceAmounts) => - assert(resource.isEmpty) - assert(resourceAmounts.isEmpty) - } + assert(assigned.isDefined) + assigned.foreach { case resource => assert(resource.isEmpty) } } test("Convert ExecutorResourceInfos to ExecutorResourcesAmounts") { @@ -150,7 +148,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // taskMount = 0.1 < 1.0 which can be assigned. val assigned = availableExecResAmounts.assignResources(rp) // update the value - availableExecResAmounts.acquire(assigned.get._2) + availableExecResAmounts.acquire(assigned.get) val availableRes = availableExecResAmounts.availableResources availableRes.foreach { case (rName, addressesAmount) => @@ -199,10 +197,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { var assigned = availableExecResAmounts.assignResources(rp) assert(!assigned.isEmpty) - assigned.foreach { case (resource, resourcesAmounts) => - assert(!resource.isEmpty) - assert(!resourcesAmounts.isEmpty) - } + assigned.foreach { case resource => assert(!resource.isEmpty)} val treqs1 = new TaskResourceRequests() .resource("gpu", gpuTaskAmount) @@ -285,18 +280,12 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // taskMount = 0.1 < 1.0 which can be assigned. val assigned = availableExecResAmounts.assignResources(rp) assert(!assigned.isEmpty) - assigned.foreach { case (resource, resourceAmounts) => + assigned.foreach { case resource => assert(resource.size === 1) assert(resource.keys.toSeq === Seq("gpu")) - assert(resource("gpu").name === "gpu") - assert(resource("gpu").addresses === Array("2")) - - assert(resourceAmounts.size === 1) - assert(resourceAmounts.keys.toSeq === Seq("gpu")) - assert(resourceAmounts("gpu").size === 1) - assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) - assert(resourceAmounts("gpu")(resource("gpu").addresses(0)).toDouble/RESOURCE_TOTAL_AMOUNT === - gpuTaskAmount) + assert(resource("gpu").size === 1) + assert(resource("gpu").keys.toSeq === Seq("2")) + assert(resource("gpu")("2").toDouble / RESOURCE_TOTAL_AMOUNT === gpuTaskAmount) } // assign will not update the real value. @@ -310,7 +299,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } // acquire will updates the value - availableExecResAmounts.acquire(assigned.get._2) + availableExecResAmounts.acquire(assigned.get) // after acquire availableRes = availableExecResAmounts.availableResources @@ -326,7 +315,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } // release - availableExecResAmounts.release(assigned.get._2) + availableExecResAmounts.release(assigned.get) availableRes = availableExecResAmounts.availableResources availableRes.foreach { case (rName, addressesAmount) => @@ -358,28 +347,17 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // taskMount = 0.1 < 1.0 which can be assigned. val assigned = availableExecResAmounts.assignResources(rp) assert(!assigned.isEmpty) - assigned.foreach { case (resource, resourceAmounts) => - assert(resource.size === 2) - assert(resource.keys.toSeq.sorted === Seq("gpu", "fpga").sorted) - - assert(resource("gpu").name === "gpu") - assert(resource("gpu").addresses === Array("2")) - - assert(resource("fpga").name === "fpga") - assert(resource("fpga").addresses === Array("aa")) - + assigned.foreach { case resourceAmounts => assert(resourceAmounts.size === 2) assert(resourceAmounts.keys.toSeq.sorted === Seq("gpu", "fpga").sorted) assert(resourceAmounts("gpu").size === 1) assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) - assert(resourceAmounts("gpu")(resource("gpu").addresses(0)).toDouble/RESOURCE_TOTAL_AMOUNT - === gpuTaskAmount) + assert(resourceAmounts("gpu")("2").toDouble / RESOURCE_TOTAL_AMOUNT === gpuTaskAmount) assert(resourceAmounts("fpga").size === 1) assert(resourceAmounts("fpga").keys.toSeq === Seq("aa")) - assert(resourceAmounts("fpga")(resource("fpga").addresses(0)).toDouble/RESOURCE_TOTAL_AMOUNT - === fpgaTaskAmount) + assert(resourceAmounts("fpga")("aa").toDouble / RESOURCE_TOTAL_AMOUNT === fpgaTaskAmount) } // assign will not update the real value. @@ -393,7 +371,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } // acquire will updates the value - availableExecResAmounts.acquire(assigned.get._2) + availableExecResAmounts.acquire(assigned.get) // after acquire availableRes = availableExecResAmounts.availableResources @@ -407,7 +385,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } // release - availableExecResAmounts.release(assigned.get._2) + availableExecResAmounts.release(assigned.get) availableRes = availableExecResAmounts.availableResources availableRes.foreach { case (rName, addressesAmount) => @@ -432,12 +410,11 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val rp = new ResourceProfileBuilder().require(treqs).build() val assigned = availableExecResAmounts.assignResources(rp) assert(!assigned.isEmpty) - assigned.foreach { case (resource, resourceAmounts) => - assert(resource("gpu").addresses.sorted === expectedAssignedAddress.sorted) - assert(resourceAmounts("gpu").values.toArray.sorted.map(_.toDouble/RESOURCE_TOTAL_AMOUNT) + assigned.foreach { case resources => + assert(resources("gpu").values.toArray.sorted.map(_.toDouble / RESOURCE_TOTAL_AMOUNT) === expectedAssignedAmount.sorted) - availableExecResAmounts.acquire(resourceAmounts) + availableExecResAmounts.acquire(resources) val leftRes = availableExecResAmounts.availableResources assert(leftRes.size == 1) 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 448f865bf62e..a2a7377899af 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler - import java.io.{ByteArrayOutputStream, DataOutputStream, UTFDataFormatException} import java.nio.ByteBuffer import java.util.Properties @@ -26,7 +25,6 @@ import scala.collection.mutable.HashMap import org.apache.spark.{JobArtifactSet, SparkFunSuite} import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT -import org.apache.spark.resource.ResourceInformation import org.apache.spark.resource.ResourceUtils.GPU class TaskDescriptionSuite extends SparkFunSuite { @@ -61,12 +59,9 @@ class TaskDescriptionSuite extends SparkFunSuite { } } - val originalResources = - Map(GPU -> new ResourceInformation(GPU, Array("1", "2", "3"))) - - val originalResourcesAmounts = Map(GPU -> Map("1" -> (0.2*RESOURCE_TOTAL_AMOUNT).toLong, - "2" -> (0.5*RESOURCE_TOTAL_AMOUNT).toLong, - "3" -> (0.1*RESOURCE_TOTAL_AMOUNT).toLong)) + val originalResources = Map(GPU -> Map("1" -> (0.2 * RESOURCE_TOTAL_AMOUNT).toLong, + "2" -> (0.5 * RESOURCE_TOTAL_AMOUNT).toLong, + "3" -> (0.1 * RESOURCE_TOTAL_AMOUNT).toLong)) // Create a dummy byte buffer for the task. val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) @@ -89,7 +84,6 @@ class TaskDescriptionSuite extends SparkFunSuite { originalProperties, cpus = 2, originalResources, - originalResourcesAmounts, taskBuffer ) @@ -106,18 +100,8 @@ class TaskDescriptionSuite extends SparkFunSuite { assert(decodedTaskDescription.artifacts.equals(artifacts)) assert(decodedTaskDescription.properties.equals(originalTaskDescription.properties)) assert(decodedTaskDescription.cpus.equals(originalTaskDescription.cpus)) - assert(equalResources(decodedTaskDescription.resources, originalTaskDescription.resources)) - assert(decodedTaskDescription.resourcesAmounts === originalTaskDescription.resourcesAmounts) + assert(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.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 56f3b2e9c880..406028d66768 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -150,7 +150,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext // Convert resources to ExecutorResourcesAmounts automatically implicit def convertResources(resources: Map[String, mutable.Buffer[String]]): - ExecutorResourcesAmounts = { + ExecutorResourcesAmounts = { // convert the old resources to ExecutorResourcesAmounts new ExecutorResourcesAmounts(resources.map { case (rName, addresses) => rName -> addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap @@ -1760,8 +1760,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext 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.keys.toArray.sorted) + assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.keys.toArray.sorted) } test("Scheduler correctly accounts for GPUs per task with fractional amount") { @@ -1787,9 +1787,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten assert(3 === taskDescriptions.length) assert(!failedTaskSet) - assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.addresses) - assert(ArrayBuffer("0") === taskDescriptions(1).resources.get(GPU).get.addresses) - assert(ArrayBuffer("0") === taskDescriptions(2).resources.get(GPU).get.addresses) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.keys.toArray.sorted) + assert(ArrayBuffer("0") === taskDescriptions(1).resources.get(GPU).get.keys.toArray.sorted) + assert(ArrayBuffer("0") === taskDescriptions(2).resources.get(GPU).get.keys.toArray.sorted) } test("Scheduler works with multiple ResourceProfiles and gpus") { @@ -1827,10 +1827,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext var has1Gpu = 0 for (tDesc <- taskDescriptions) { assert(tDesc.resources.contains(GPU)) - if (tDesc.resources(GPU).addresses.size == 2) { + if (tDesc.resources(GPU).keys.size == 2) { has2Gpus += 1 } - if (tDesc.resources(GPU).addresses.size == 1) { + if (tDesc.resources(GPU).keys.size == 1) { has1Gpu += 1 } } @@ -1848,7 +1848,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten assert(2 === taskDescriptions.length) assert(taskDescriptions.head.resources.contains(GPU)) - assert(2 == taskDescriptions.head.resources(GPU).addresses.size) + assert(2 == taskDescriptions.head.resources(GPU).keys.size) } test("Scheduler works with task resource profiles") { @@ -1887,10 +1887,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext var has1Gpu = 0 for (tDesc <- taskDescriptions) { assert(tDesc.resources.contains(GPU)) - if (tDesc.resources(GPU).addresses.size == 2) { + if (tDesc.resources(GPU).keys.size == 2) { has2Gpus += 1 } - if (tDesc.resources(GPU).addresses.size == 1) { + if (tDesc.resources(GPU).keys.size == 1) { has1Gpu += 1 } } @@ -1908,7 +1908,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten assert(2 === taskDescriptions.length) assert(taskDescriptions.head.resources.contains(GPU)) - assert(2 == taskDescriptions.head.resources(GPU).addresses.size) + assert(2 == taskDescriptions.head.resources(GPU).keys.size) } test("Calculate available tasks slots for task resource profiles") { @@ -2325,10 +2325,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten assert(4 === 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("2") === taskDescriptions(2).resources.get(GPU).get.addresses) - assert(ArrayBuffer("3") === taskDescriptions(3).resources.get(GPU).get.addresses) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.keys.toArray.sorted) + assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.keys.toArray.sorted) + assert(ArrayBuffer("2") === taskDescriptions(2).resources.get(GPU).get.keys.toArray.sorted) + assert(ArrayBuffer("3") === taskDescriptions(3).resources.get(GPU).get.keys.toArray.sorted) } } @@ -2368,8 +2368,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val assignedExecutorsGpus = ArrayBuffer[(String, String)]() for (tDesc <- taskDescriptions) { assert(tDesc.resources.contains(GPU)) - assert(tDesc.resources.get(GPU).get.addresses.length == 1) - assignedExecutorsGpus.append((tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) + assert(tDesc.resources.get(GPU).get.keys.size == 1) + val address = tDesc.resources.get(GPU).get.keys.toArray.sorted + assignedExecutorsGpus.append((tDesc.executorId, address(0))) } assert(assignedExecutorsGpus.sorted sameElements @@ -2423,15 +2424,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext var index = 0 for (tDesc <- taskDescriptions) { assert(tDesc.resources.contains(GPU)) - assert(tDesc.resources.get(GPU).get.addresses.length == 1) + val addresses = tDesc.resources.get(GPU).get.keys.toArray.sorted + assert(addresses.length == 1) if (index < 4) { // the first 4 tasks will grab 0.7 gpu - assert(tDesc.resources.get(GPU).get.addresses(0) == index.toString) - assert(tDesc.resourcesAmounts.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT - == 0.7) + assert(addresses(0) == index.toString) + assert(tDesc.resources.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) } else { - assert(tDesc.resources.get(GPU).get.addresses(0) == (index - 4).toString) - assert( - tDesc.resourcesAmounts.get(GPU).get((index - 4).toString).toDouble/RESOURCE_TOTAL_AMOUNT + assert(addresses(0) == (index - 4).toString) + assert(tDesc.resources.get(GPU).get((index - 4).toString).toDouble/RESOURCE_TOTAL_AMOUNT == 0.3) } index += 1 @@ -2489,19 +2489,20 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext for (tDesc <- taskDescriptions) { assert(tDesc.resources.contains(GPU)) - assert(tDesc.resources.get(GPU).get.addresses.length == 1) - val address = tDesc.resources.get(GPU).get.addresses(0) + val addresses = tDesc.resources.get(GPU).get.keys.toArray.sorted + assert(addresses.length == 1) + val address = addresses(0) // Executor 0, executor 1, executor 2, executor 3 // task_0.7, task_03 task_0.7, task_03 task_0.7, task_03 task_0.7, task_03 if (index % 2 == 0) { higherAssignedExecutorsGpus.append( - (tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) - assert(tDesc.resourcesAmounts.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) + (tDesc.executorId, address)) + assert(tDesc.resources.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) } else { lowerAssignedExecutorsGpus.append( - (tDesc.executorId, tDesc.resources.get(GPU).get.addresses(0))) - assert(tDesc.resourcesAmounts.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.3) + (tDesc.executorId, address)) + assert(tDesc.resources.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.3) } index += 1 } @@ -2555,10 +2556,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext var index = 0 for (tDesc <- taskDescriptions) { assert(tDesc.resources.contains(GPU)) - assert(tDesc.resources.get(GPU).get.addresses.length == 1) - assert(tDesc.resources.get(GPU).get.addresses(0) == index.toString) - assert(tDesc.resourcesAmounts.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT - == 0.7) + val addresses = tDesc.resources.get(GPU).get.keys.toArray.sorted + assert(addresses.length == 1) + assert(addresses(0) == index.toString) + assert(tDesc.resources.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) index += 1 } } 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 299ef5160599..161e50c5dfaa 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -38,7 +38,8 @@ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.{SKIP_VALIDATE_CORES_TESTING, TEST_DYNAMIC_ALLOCATION_SCHEDULE_ENABLED} -import org.apache.spark.resource.{ResourceInformation, ResourceProfile} +import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceProfile import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -1824,7 +1825,8 @@ class TaskSetManagerSuite val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val taskResourceAssignments = Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))) + val taskResourceAssignments = Map( + GPU -> Map("0" -> RESOURCE_TOTAL_AMOUNT, "1" -> RESOURCE_TOTAL_AMOUNT)) val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, 2, taskResourceAssignments)._1 assert(taskOption.isDefined) @@ -1832,7 +1834,9 @@ class TaskSetManagerSuite val allocatedResources = taskOption.get.resources assert(allocatedCpus == 2) assert(allocatedResources.size == 1) - assert(allocatedResources(GPU).addresses sameElements Array("0", "1")) + assert(allocatedResources(GPU).keys.toArray.sorted sameElements Array("0", "1")) + assert(allocatedResources === taskResourceAssignments) + } test("SPARK-26755 Ensure that a speculative task is submitted only once for execution") { From d02a3be9e51b33efae54eb87663ff13b9c35a0f4 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 6 Nov 2023 16:03:18 +0800 Subject: [PATCH 07/18] fix CI --- .../org/apache/spark/resource/ResourceAllocator.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 3a5aea292699..930bf81336ef 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -26,14 +26,14 @@ private[spark] object ResourceAmountUtils { /** * Using "double" to do the resource calculation may encounter a problem of precision loss. Eg * - * scala> val taskAmount = 1.0 / 9 + * scala> val taskAmount = 1.0 / 9 * taskAmount: Double = 0.1111111111111111 * - * scala> var total = 1.0 + * scala> var total = 1.0 * total: Double = 1.0 * - * scala> for (i <- 1 to 9 ) { - * | if (total >= taskAmount) { + * scala> for (i <- 1 to 9 ) { + * | if (total >= taskAmount) { * | total -= taskAmount * | println(s"assign $taskAmount for task $i, total left: $total") * | } else { @@ -68,7 +68,7 @@ private[spark] trait ResourceAllocator { /** * Map from an address to its availability default to 1.0 (we multiply RESOURCE_TOTAL_AMOUNT - * to avoid precision error), a value > 0 means the address is available, while value of + * to avoid precision error), a value > 0 means the address is available, while value of * 0 means the address is fully assigned. */ private lazy val addressAvailabilityMap = { From 2127a7b6e6f07a28d69466326def9363300cf801 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 13 Nov 2023 14:25:36 +0800 Subject: [PATCH 08/18] validate the task.amount in the ResourceProfile and TaskResourceProfile --- .../spark/resource/ResourceProfile.scala | 20 ++++++++++++++ .../spark/resource/ResourceProfileSuite.scala | 27 +++++++++++++++++++ 2 files changed, 47 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index fdaa68b6931e..04d5206fdb02 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -49,6 +49,8 @@ class ResourceProfile( val executorResources: Map[String, ExecutorResourceRequest], val taskResources: Map[String, TaskResourceRequest]) extends Serializable with Logging { + validate() + // _id is only a var for testing purposes private var _id = ResourceProfile.getNextProfileId // This is used for any resources that use fractional amounts, the key is the resource name @@ -59,6 +61,19 @@ class ResourceProfile( private var _maxTasksPerExecutor: Option[Int] = None private var _coresLimitKnown: Boolean = false + /** + * Validate the ResourceProfile + */ + protected def validate(): Unit = { + // The task.amount in ResourceProfile falls within the range of 0 to 0.5, + // or it's a whole number + for ((_, taskReq) <- taskResources) { + val taskAmount = taskReq.amount + assert(taskAmount <= 0.5 || taskAmount % 1 == 0, + s"The task resource amount ${taskAmount} must be either <= 0.5, or a whole number.") + } + } + /** * A unique id of this ResourceProfile */ @@ -280,6 +295,11 @@ private[spark] class TaskResourceProfile( override val taskResources: Map[String, TaskResourceRequest]) extends ResourceProfile(Map.empty, taskResources) { + // The task.amount in TaskResourceProfile falls within the range of 0 to 1.0, + // or it's a whole number, and it has been checked in the TaskResourceRequest. + // Therefore, we can safely skip this check. + override protected def validate(): Unit = {} + override protected[spark] def getCustomExecutorResources() : Map[String, ExecutorResourceRequest] = { if (SparkEnv.get == null) { diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index f549b5f4dcae..f49efc52e6db 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -392,6 +392,33 @@ class ResourceProfileSuite extends SparkFunSuite with MockitoSugar { "Task resources should have 1 custom resource") } + test("SPARK-45527 fractional TaskResourceRequests in ResourceProfile") { + val ereqs = new ExecutorResourceRequests().cores(6).resource("gpus", 6) + var treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.1) + new ResourceProfileBuilder().require(ereqs).require(treqs).build() + + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.5) + new ResourceProfileBuilder().require(ereqs).require(treqs).build() + + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.7) + + val msg = intercept[AssertionError] { + new ResourceProfileBuilder().require(ereqs).require(treqs).build() + }.getMessage + assert(msg.contains("The task resource amount 0.7 must be either <= 0.5, or a whole number")) + } + + test("SPARK-45527 fractional TaskResourceRequests in TaskResourceProfile") { + var treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.1) + new ResourceProfileBuilder().require(treqs).build() + + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.5) + new ResourceProfileBuilder().require(treqs).build() + + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.7) + new ResourceProfileBuilder().require(treqs).build() + } + private def withMockSparkEnv(conf: SparkConf)(f: => Unit): Unit = { val previousEnv = SparkEnv.get val mockEnv = mock[SparkEnv] From 61bcb34b12f2f0a38faadad1fbfe975e01e0161a Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 13 Nov 2023 16:26:00 +0800 Subject: [PATCH 09/18] add tests --- .../spark/resource/ResourceProfile.scala | 4 - .../spark/resource/ResourceUtilsSuite.scala | 90 +++++++++++++++++++ 2 files changed, 90 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 04d5206fdb02..ca69ad291308 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -122,10 +122,6 @@ class ResourceProfile( * This function takes into account fractional amounts for the task resource requirement. * Spark only supports fractional amounts < 1 to basically allow for multiple tasks * to use the same resource address. - * The way the scheduler handles this is it adds the same address the number of slots per - * address times and then the amount becomes 1. This way it re-uses the same address - * the correct number of times. ie task requirement amount=0.25 -> addrs["0", "0", "0", "0"] - * and scheduler task amount=1. See ResourceAllocator.slotsPerAddress. */ private[spark] def getSchedulerTaskResourceAmount(resource: String): Int = { val taskAmount = taskResources.getOrElse(resource, diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index 1ab9f7c5d2b0..6aab2e25f4e8 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -26,6 +26,7 @@ import org.json4s.{DefaultFormats, Extraction} import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.RESOURCES_WARNING_TESTING import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.util.Utils @@ -336,4 +337,93 @@ class ResourceUtilsSuite extends SparkFunSuite assert(error.contains("User is expecting to use resource: gpu, but " + "didn't specify a discovery script!")) } + + test("SPARK-45527 warnOnWastedResources for ResourceProfile") { + val conf = new SparkConf() + conf.set("spark.executor.cores", "10") + conf.set("spark.task.cpus", "1") + conf.set(RESOURCES_WARNING_TESTING, true) + + // cpu limiting task number = 10/1, gpu limiting task number = 1/0.1 + var ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) + var treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.1) + var rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + // no exception, + warnOnWastedResources(rp, conf) + + // cpu limiting task number = 10/2, gpu limiting task number = 1/0.1 + ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) + treqs = new TaskResourceRequests().cpus(2).resource("gpu", 0.1) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + var msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + + assert(msg.contains("The configuration of resource: gpu (exec = 1, task = 0.1/10, runnable " + + "tasks = 10) will result in wasted resources due to resource cpus limiting the number of " + + "runnable tasks per executor to: 5. Please adjust your configuration.")) + + // cpu limiting task number = 10/1, gpu limiting task number = 1/0.2 = 5 + ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.2) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + + assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + "tasks per executor to: 5. Please adjust your configuration")) + + // cpu limiting task number = 10/1, gpu limiting task number = 1/0.5 = 2 + ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.5) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + + assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + "tasks per executor to: 2. Please adjust your configuration")) + + // executor gpu amount > 1 + // cpu limiting task number = 100/2 = 50, gpu limiting task number = 2/0.2 = 10 + ereqs = new ExecutorResourceRequests().cores(100).resource("gpu", 2) + treqs = new TaskResourceRequests().cpus(2).resource("gpu", 0.2) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + + assert(msg.contains("The configuration of cores (exec = 100 task = 2, runnable tasks = 50) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + "tasks per executor to: 10. Please adjust your configuration")) + + // executor gpu amount > 1 and task gpu amount > 1 + // cpu limiting task number = 10/1 = 10, gpu limiting task number = 6/2 = 3 + ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 6) + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 2) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + + assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + "tasks per executor to: 3. Please adjust your configuration")) + + // executor gpu amount > 1 and task gpu amount > 1 + // cpu limiting task number = 10/1 = 10, gpu limiting task number = 6/0.2 = 3 + ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 6) + treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.2) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + + assert(msg.contains("The configuration of resource: gpu (exec = 6, task = 0.2/5, runnable " + + "tasks = 30) will result in wasted resources due to resource cpus limiting the number " + + "of runnable tasks per executor to: 10. Please adjust your configuration")) + } } From eb7f918f7ede6280ac689aea8be4b03db465f6b1 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Tue, 14 Nov 2023 10:23:05 +0800 Subject: [PATCH 10/18] add warnOnWastedResources for TaskResourceProfile --- .../spark/resource/ResourceUtilsSuite.scala | 285 ++++++++++++++---- 1 file changed, 221 insertions(+), 64 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index 6aab2e25f4e8..9bf79601e848 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -351,79 +351,236 @@ class ResourceUtilsSuite extends SparkFunSuite // no exception, warnOnWastedResources(rp, conf) - // cpu limiting task number = 10/2, gpu limiting task number = 1/0.1 ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) - treqs = new TaskResourceRequests().cpus(2).resource("gpu", 0.1) + treqs = new TaskResourceRequests().cpus(2).resource("gpu", 0.2) rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() - var msg = intercept[SparkException] { - warnOnWastedResources(rp, conf) - }.getMessage - - assert(msg.contains("The configuration of resource: gpu (exec = 1, task = 0.1/10, runnable " + - "tasks = 10) will result in wasted resources due to resource cpus limiting the number of " + - "runnable tasks per executor to: 5. Please adjust your configuration.")) + // no exception, + warnOnWastedResources(rp, conf) - // cpu limiting task number = 10/1, gpu limiting task number = 1/0.2 = 5 - ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) - treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.2) + ereqs = new ExecutorResourceRequests().cores(20).resource("gpu", 2) + treqs = new TaskResourceRequests().cpus(2).resource("gpu", 0.2) rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() - msg = intercept[SparkException] { - warnOnWastedResources(rp, conf) - }.getMessage + // no exception, + warnOnWastedResources(rp, conf) - assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + - "will result in wasted resources due to resource gpu limiting the number of runnable " + - "tasks per executor to: 5. Please adjust your configuration")) + var msg: String = "" + + // Test cpu limiting task number + // format: (executor.core, task.cpus, executor.gpu, task.gpu, expected runnable tasks) + Seq( + (10, 2, 1, 0.1, 5), // cpu limiting task number=10/2=5, gpu limiting task number = 1/0.1 = 10 + (10, 3, 1, 0.1, 3), // cpu limiting task number=10/3=3, gpu limiting task number = 1/0.1 = 10 + (10, 4, 1, 0.1, 2), // cpu limiting task number=10/4=2, gpu limiting task number = 1/0.1 = 10 + (10, 5, 1, 0.1, 2), // cpu limiting task number=10/5=2, gpu limiting task number = 1/0.1 = 10 + (10, 6, 1, 0.1, 1), // cpu limiting task number=10/6=1, gpu limiting task number = 1/0.1 = 10 + (10, 10, 1, 0.1, 1), // cpu limiting task number=10/6=1, gpu limiting task number = 1/0.1 = 10 + (20, 7, 1, 0.1, 2), // cpu limiting task number=20/7=3, gpu limiting task number = 1/0.1 = 10 + (30, 7, 1, 0.1, 4), // cpu limiting task number=30/7=4, gpu limiting task number = 1/0.1 = 10 + (50, 14, 1, 0.1, 3) // cpu limiting task number=50/14=3, gpu limiting task number = 1/0.1=10 + ).foreach { case (executorCores, taskCpus: Int, executorGpus: Int, taskGpus: Double, + expectedTaskNumber: Int) => + ereqs = new ExecutorResourceRequests().cores(executorCores).resource("gpu", executorGpus) + treqs = new TaskResourceRequests().cpus(taskCpus).resource("gpu", taskGpus) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + assert(msg.contains("The configuration of resource: gpu (exec = 1, task = 0.1/10, runnable " + + "tasks = 10) will result in wasted resources due to resource cpus limiting the number of " + + s"runnable tasks per executor to: ${expectedTaskNumber}. Please adjust your configuration.") + ) + } - // cpu limiting task number = 10/1, gpu limiting task number = 1/0.5 = 2 - ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 1) - treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.5) - rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() - msg = intercept[SparkException] { - warnOnWastedResources(rp, conf) - }.getMessage + // Test gpu limiting task number + // format: (executor.core, task.cpus, executor.gpu, task.gpu, expected runnable tasks) + Seq( + (10, 1, 1, 1.0/9, 9), // cpu limiting task number=10, gpu limiting task number = 9 + (10, 1, 1, 1.0/8, 8), // cpu limiting task number=10, gpu limiting task number = 8 + (10, 1, 1, 1.0/7, 7), // cpu limiting task number=10, gpu limiting task number = 7 + (10, 1, 1, 1.0/6, 6), // cpu limiting task number=10, gpu limiting task number = 6 + (10, 1, 1, 1.0/5, 5), // cpu limiting task number=10, gpu limiting task number = 5 + (10, 1, 1, 1.0/4, 4), // cpu limiting task number=10, gpu limiting task number = 4 + (10, 1, 1, 1.0/3, 3), // cpu limiting task number=10, gpu limiting task number = 3 + (10, 1, 1, 1.0/2, 2), // cpu limiting task number=10, gpu limiting task number = 2 + (10, 1, 1, 1.0, 1), // cpu limiting task number=10, gpu limiting task number = 1 + (30, 1, 2, 1.0/9, 2*9), // cpu limiting task number=30, gpu limiting task number = 2*9 + (30, 1, 2, 1.0/8, 2*8), // cpu limiting task number=30, gpu limiting task number = 2*8 + (30, 1, 2, 1.0/7, 2*7), // cpu limiting task number=30, gpu limiting task number = 2*7 + (30, 1, 2, 1.0/6, 2*6), // cpu limiting task number=30, gpu limiting task number = 2*6 + (30, 1, 2, 1.0/5, 2*5), // cpu limiting task number=30, gpu limiting task number = 2*5 + (30, 1, 2, 1.0/4, 2*4), // cpu limiting task number=30, gpu limiting task number = 2*4 + (30, 1, 2, 1.0/3, 2*3), // cpu limiting task number=30, gpu limiting task number = 2*3 + (30, 1, 2, 1.0/2, 2*2), // cpu limiting task number=30, gpu limiting task number = 2*2 + (30, 1, 2, 1.0, 2*1), // cpu limiting task number=30, gpu limiting task number = 2*1 + (30, 1, 2, 2.0, 1), // cpu limiting task number=30, gpu limiting task number = 1 + (70, 2, 7, 0.5, 7*2), // cpu limiting task number=30, gpu limiting task number = 7*1/0.5= + (80, 3, 9, 2.0, 9/2) // cpu limiting task number=30, gpu limiting task number = 9/2 + ).foreach { case (executorCores, taskCpus: Int, executorGpus: Int, taskGpus: Double, + expectedTaskNumber: Int) => + ereqs = new ExecutorResourceRequests().cores(executorCores).resource("gpu", executorGpus) + treqs = new TaskResourceRequests().cpus(taskCpus).resource("gpu", taskGpus) + rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + assert(msg.contains(s"The configuration of cores (exec = ${executorCores} task = " + + s"${taskCpus}, runnable tasks = ${executorCores/taskCpus}) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + s"tasks per executor to: ${expectedTaskNumber}. Please adjust your configuration")) + } + } + + private class FakedTaskResourceProfile( + val defaultRp: ResourceProfile, + override val taskResources: Map[String, TaskResourceRequest]) + extends TaskResourceProfile(taskResources) { + override protected[spark] def getCustomExecutorResources() + : Map[String, ExecutorResourceRequest] = defaultRp.getCustomExecutorResources() + } + + test("SPARK-45527 warnOnWastedResources for TaskResourceProfile when executor number = 1") { + val conf = new SparkConf() + conf.set("spark.executor.cores", "10") + conf.set("spark.task.cpus", "1") + conf.set(TASK_GPU_ID.amountConf, "0.1") + conf.set(EXECUTOR_GPU_ID.amountConf, "1") + conf.set(RESOURCES_WARNING_TESTING, true) + + val defaultDp = ResourceProfile.getOrCreateDefaultProfile(conf) + + // cpu limiting task number = 10/1, gpu limiting task number = 1/0.1 + var treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.1) + var rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + // no exception, + warnOnWastedResources(rp, conf) + + var msg: String = "" + + // Test cpu limiting task number + // format: (task cpu cores, task gpu amount, expected runnable tasks) + // spark.executor.cores=60, spark.task.cpus=1, EXECUTOR_GPU_ID=6, TASK_GPU_ID=0.1 + Seq( + (2, 0.1, 5), // cpu limiting task number = 10/2=5, gpu limiting task number = 1/0.1 = 10 + (3, 0.1, 3), // cpu limiting task number = 10/3 = 3, gpu limiting task number = 1/0.1 = 10 + (4, 0.1, 2), // cpu limiting task number = 10/4 = 2, gpu limiting task number = 1/0.1 = 10 + (5, 0.1, 2), // cpu limiting task number = 10/5 = 2, gpu limiting task number = 1/0.1 = 10 + (6, 0.1, 1), // cpu limiting task number = 10/6 = 1, gpu limiting task number = 1/0.1 = 10 + (7, 0.1, 1), // cpu limiting task number = 10/7 = 1, gpu limiting task number = 1/0.1 = 10 + (10, 0.1, 1) // cpu limiting task number = 10/10 = 1, gpu limiting task number = 1/0.1 = 10 + ).foreach { case (cores: Int, gpus: Double, expectedTaskNumber: Int) => + treqs = new TaskResourceRequests().cpus(cores).resource("gpu", gpus) + rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + assert(msg.contains("The configuration of resource: gpu (exec = 1, task = 0.1/10, runnable " + + "tasks = 10) will result in wasted resources due to resource cpus limiting the number of " + + s"runnable tasks per executor to: $expectedTaskNumber. Please adjust your configuration.") + ) + } + + // Test gpu limiting task number + // format: (task cpu cores, task gpu amount, expected runnable tasks) + // spark.executor.cores=60, spark.task.cpus=1, EXECUTOR_GPU_ID=6, TASK_GPU_ID=0.1 + Seq( + (1, 0.111, 9), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.111=9 + (1, 0.125, 8), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.125=8 + (1, 0.142, 7), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.142=7 + (1, 0.166, 6), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.166=6 + (1, 0.2, 5), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.2=5 + (1, 0.25, 4), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.25=4 + (1, 0.333, 3), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.333=3 + (1, 0.5, 2), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.166=2 + (1, 0.6, 1), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.6=1 + (1, 0.7, 1), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.7=1 + (1, 0.8, 1), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.8=1 + (1, 0.9, 1), // cpu limiting task number = 10/1, gpu limiting task number = 1/0.9=1 + (1, 1.0, 1) // cpu limiting task number = 10/1, gpu limiting task number = 1/1.0=1 + ).foreach { case (cores: Int, gpus: Double, expectedTaskNumber: Int) => + treqs = new TaskResourceRequests().cpus(cores).resource("gpu", gpus) + rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + s"tasks per executor to: $expectedTaskNumber. Please adjust your configuration")) + } + } + + test("SPARK-45527 warnOnWastedResources for TaskResourceProfile when executor number > 1") { + val conf = new SparkConf() + conf.set("spark.executor.cores", "60") + conf.set("spark.task.cpus", "1") + conf.set(TASK_GPU_ID.amountConf, "0.1") + conf.set(EXECUTOR_GPU_ID.amountConf, "6") + conf.set(RESOURCES_WARNING_TESTING, true) - assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + - "will result in wasted resources due to resource gpu limiting the number of runnable " + - "tasks per executor to: 2. Please adjust your configuration")) + val defaultDp = ResourceProfile.getOrCreateDefaultProfile(conf) - // executor gpu amount > 1 - // cpu limiting task number = 100/2 = 50, gpu limiting task number = 2/0.2 = 10 - ereqs = new ExecutorResourceRequests().cores(100).resource("gpu", 2) + // cpu limiting task number = 60/1, gpu limiting task number = 6/0.1 + var treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.1) + var rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + // no exception, + warnOnWastedResources(rp, conf) + + // cpu limiting task number = 60/2 = 30, gpu limiting task number = 6/0.2 = 30 treqs = new TaskResourceRequests().cpus(2).resource("gpu", 0.2) - rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() - msg = intercept[SparkException] { - warnOnWastedResources(rp, conf) - }.getMessage - - assert(msg.contains("The configuration of cores (exec = 100 task = 2, runnable tasks = 50) " + - "will result in wasted resources due to resource gpu limiting the number of runnable " + - "tasks per executor to: 10. Please adjust your configuration")) - - // executor gpu amount > 1 and task gpu amount > 1 - // cpu limiting task number = 10/1 = 10, gpu limiting task number = 6/2 = 3 - ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 6) - treqs = new TaskResourceRequests().cpus(1).resource("gpu", 2) - rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() - msg = intercept[SparkException] { - warnOnWastedResources(rp, conf) - }.getMessage - - assert(msg.contains("The configuration of cores (exec = 10 task = 1, runnable tasks = 10) " + - "will result in wasted resources due to resource gpu limiting the number of runnable " + - "tasks per executor to: 3. Please adjust your configuration")) - - // executor gpu amount > 1 and task gpu amount > 1 - // cpu limiting task number = 10/1 = 10, gpu limiting task number = 6/0.2 = 3 - ereqs = new ExecutorResourceRequests().cores(10).resource("gpu", 6) - treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.2) - rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build() - msg = intercept[SparkException] { - warnOnWastedResources(rp, conf) - }.getMessage + rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + // no exception + warnOnWastedResources(rp, conf) - assert(msg.contains("The configuration of resource: gpu (exec = 6, task = 0.2/5, runnable " + - "tasks = 30) will result in wasted resources due to resource cpus limiting the number " + - "of runnable tasks per executor to: 10. Please adjust your configuration")) + var msg: String = "" + + // Test cpu limiting task number + // format: (task cpu cores, task gpu amount, expected runnable tasks) + // spark.executor.cores=60, spark.task.cpus=1, EXECUTOR_GPU_ID=6, TASK_GPU_ID=0.1 + Seq( + (4, 0.2, 15), // cpu limiting task number = 60/4=15, gpu limiting task number = 6/0.2 = 30 + (7, 0.2, 8), // cpu limiting task number = 60/7 = 8, gpu limiting task number = 6/0.2 = 30 + (30, 0.2, 2), // cpu limiting task number = 60/30 = 2, gpu limiting task number = 6/0.2 = 30 + (31, 0.2, 1), // cpu limiting task number = 60/31 = 1, gpu limiting task number = 6/0.2 = 30 + (55, 0.2, 1), // cpu limiting task number = 60/55 = 1, gpu limiting task number = 6/0.2 = 30 + (60, 0.2, 1) // cpu limiting task number = 60/60 = 1, gpu limiting task number = 6/0.2 = 30 + ).foreach { case (cores: Int, gpus: Double, expectedTaskNumber: Int) => + treqs = new TaskResourceRequests().cpus(cores).resource("gpu", gpus) + rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + assert(msg.contains("The configuration of resource: gpu (exec = 6, task = 0.2/5, runnable " + + "tasks = 30) will result in wasted resources due to resource cpus limiting the number of " + + s"runnable tasks per executor to: $expectedTaskNumber. Please adjust your configuration.") + ) + } + + // Test gpu limiting task number + // format: (task cpu cores, task gpu amount, expected runnable tasks) + // spark.executor.cores=60, spark.task.cpus=1, EXECUTOR_GPU_ID=6, TASK_GPU_ID=0.1 + Seq( + (1, 0.111, 54), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.111=54 + (1, 0.125, 48), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.125=48 + (1, 0.142, 42), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.142=42 + (1, 0.166, 36), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.166=36 + (1, 0.2, 30), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.2=30 + (1, 0.25, 24), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.25=24 + (1, 0.33, 18), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.33=18 + (1, 0.5, 12), // cpu limiting task number = 60/1, gpu limiting task number = 6*1/0.5=12 + (1, 0.7, 6), // cpu limiting task number = 60/1 = 60, gpu limiting task number = 6*1/0.7 = 6 + (1, 1.0, 6), // cpu limiting task number = 60/1 = 60, gpu limiting task number = 6/1 = 6 + (1, 2.0, 3), // cpu limiting task number = 60/1 = 60, gpu limiting task number = 6/2 = 3 + (1, 3.0, 2), // cpu limiting task number = 60/1 = 60, gpu limiting task number = 6/3 = 2 + (1, 4.0, 1), // cpu limiting task number = 60/1 = 60, gpu limiting task number = 6/4 = 1 + (1, 6.0, 1) // cpu limiting task number = 60/1 = 60, gpu limiting task number = 6/6 = 1 + ).foreach { case (cores: Int, gpus: Double, expectedTaskNumber: Int) => + treqs = new TaskResourceRequests().cpus(cores).resource("gpu", gpus) + rp = new FakedTaskResourceProfile(defaultDp, treqs.requests) + msg = intercept[SparkException] { + warnOnWastedResources(rp, conf) + }.getMessage + assert(msg.contains("The configuration of cores (exec = 60 task = 1, runnable tasks = 60) " + + "will result in wasted resources due to resource gpu limiting the number of runnable " + + s"tasks per executor to: $expectedTaskNumber. Please adjust your configuration")) + } } } From 18084e6383943b57812427359cf0acce43fc8d41 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Tue, 14 Nov 2023 12:23:31 +0800 Subject: [PATCH 11/18] change the rp corresponding tests --- .../scheduler/TaskSchedulerImplSuite.scala | 324 +++++++++++------- 1 file changed, 199 insertions(+), 125 deletions(-) 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 406028d66768..29c3a19fada7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -2299,87 +2299,218 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong } } - Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => - test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + - s"restrict each task takes 1 gpu exclusively in the same executor") { - val taskCpus = 1 - val executorCpus = 1000 // cpu will not limit the concurrent tasks number - val executorGpus = 1 + // 1 executor with 4 GPUS + Seq(true, false).foreach { barrierMode => + val barrier = if (barrierMode) "barrier" else "" + (1 to 20).foreach { taskNum => + val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + + s"restrict $taskNum $barrier tasks run in the same executor") { + val taskCpus = 1 + val executorCpus = 100 // cpu will not limit the concurrent tasks number + val executorGpus = 1 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> gpuTaskAmount.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val taskSet = if (barrierMode) { + FakeTask.createTaskSet(100) + } else { + FakeTask.createBarrierTaskSet(4 * taskNum) + } - val taskScheduler = setupScheduler(numCores = executorCpus, - config.CPUS_PER_TASK.key -> taskCpus.toString, - TASK_GPU_ID.amountConf -> gpuTaskAmount.toString, - EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, - config.EXECUTOR_CORES.key -> executorCpus.toString) + val resources = new ExecutorResourcesAmounts( + Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) - val taskSet = FakeTask.createTaskSet(100) + val workerOffers = + IndexedSeq(WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources)) - val resources = new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements. + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 * taskNum === taskDescriptions.length) + assert(!failedTaskSet) + var gpuAddress = -1 + for (taskId <- 0 until 4 * taskNum) { + if (taskId % taskNum == 0) { + gpuAddress += 1 + } + assert(ArrayBuffer(gpuAddress.toString) === + taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted) + } + } + } + } - val workerOffers = - IndexedSeq(new WorkerOffer("executor0", "host0", executorCpus, None, resources)) + // 4 executors, each of which has 1 GPU + Seq(true, false).foreach { barrierMode => + val barrier = if (barrierMode) "barrier" else "" + (1 to 20).foreach { taskNum => + val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + + s"restrict $taskNum $barrier tasks run on the different executor") { + val taskCpus = 1 + val executorCpus = 100 // cpu will not limit the concurrent tasks number + val executorGpus = 1 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> gpuTaskAmount.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val taskSet = if (barrierMode) { + FakeTask.createTaskSet(100) + } else { + FakeTask.createBarrierTaskSet(4 * taskNum) + } - taskScheduler.submitTasks(taskSet) - // Launch tasks on executor that satisfies resource requirements. - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(4 === taskDescriptions.length) - assert(!failedTaskSet) - assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.keys.toArray.sorted) - assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.keys.toArray.sorted) - assert(ArrayBuffer("2") === taskDescriptions(2).resources.get(GPU).get.keys.toArray.sorted) - assert(ArrayBuffer("3") === taskDescriptions(3).resources.get(GPU).get.keys.toArray.sorted) + val workerOffers = + IndexedSeq( + WorkerOffer("executor0", "host0", executorCpus, Some("host0"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0))))), + WorkerOffer("executor1", "host1", executorCpus, Some("host1"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("1" -> 1.0))))), + WorkerOffer("executor2", "host2", executorCpus, Some("host2"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("2" -> 1.0))))), + WorkerOffer("executor3", "host3", executorCpus, Some("host3"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("3" -> 1.0)))))) + + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements + + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 * taskNum === taskDescriptions.length) + assert(!failedTaskSet) + val assignedGpus: HashMap[String, Int] = HashMap.empty + for (taskId <- 0 until 4 * taskNum) { + val gpus = taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted + assert(gpus.length == 1) + val addr = gpus(0) + if (!assignedGpus.contains(addr)) { + assignedGpus(addr) = 1 + } else { + assignedGpus(addr) += 1 + } + } + assert(assignedGpus.toMap === + Map("0" -> taskNum, "1" -> taskNum, "2" -> taskNum, "3" -> taskNum)) + } } } + // 1 executor with 4 GPUS + Seq(true, false).foreach { barrierMode => + val barrier = if (barrierMode) "barrier" else "" + (1 to 20).foreach { taskNum => + val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " + + s"restrict $taskNum $barrier tasks run in the same executor") { + val executorCpus = 100 // cpu will not limit the concurrent tasks number + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> "1", + TASK_GPU_ID.amountConf -> "0.1", + EXECUTOR_GPU_ID.amountConf -> "4", + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val treqs = new TaskResourceRequests().cpus(1).resource(GPU, gpuTaskAmount) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val taskSet = if (barrierMode) { + FakeTask.createTaskSet(100, 0, 1, 1, rp.id) + } else { + FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id) + } + val resources = new ExecutorResourcesAmounts( + Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) - Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => - test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + - s"restrict only 1 task will run on each executor") { - val taskCpus = 1 - val executorCpus = 1000 // cpu will not limit the concurrent tasks number - val executorGpus = 1 + val workerOffers = IndexedSeq( + WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources, rp.id) + ) - val taskScheduler = setupScheduler(numCores = executorCpus, - config.CPUS_PER_TASK.key -> taskCpus.toString, - TASK_GPU_ID.amountConf -> gpuTaskAmount.toString, - EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, - config.EXECUTOR_CORES.key -> executorCpus.toString) + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements. + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 * taskNum === taskDescriptions.length) + assert(!failedTaskSet) + var gpuAddress = -1 + for (taskId <- 0 until 4 * taskNum) { + if (taskId % taskNum == 0) { + gpuAddress += 1 + } + assert(ArrayBuffer(gpuAddress.toString) === + taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted) + } + } + } + } - val taskSet = FakeTask.createTaskSet(100) + // 4 executors, each of which has 1 GPU + Seq(true, false).foreach { barrierMode => + val barrier = if (barrierMode) "barrier" else "" + (1 to 20).foreach { taskNum => + val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " + + s"restrict $taskNum $barrier tasks run on the different executor") { + val executorCpus = 100 // cpu will not limit the concurrent tasks number + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> "1", + TASK_GPU_ID.amountConf -> "0.1", + EXECUTOR_GPU_ID.amountConf -> "1", + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val treqs = new TaskResourceRequests().cpus(1).resource(GPU, gpuTaskAmount) + val rp = new TaskResourceProfile(treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val taskSet = if (barrierMode) { + FakeTask.createTaskSet(100, 0, 1, 1, rp.id) + } else { + FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id) + } - val workerOffers = - IndexedSeq( - new WorkerOffer("executor0", "host0", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0))))), - new WorkerOffer("executor1", "host1", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("7" -> 1.0))))), - new WorkerOffer("executor2", "host2", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("9" -> 1.0))))), - new WorkerOffer("executor3", "host3", executorCpus, None, - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("20" -> 1.0)))))) + val workerOffers = + IndexedSeq( + WorkerOffer("executor0", "host0", executorCpus, Some("host1"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0)))), + rp.id), + WorkerOffer("executor1", "host1", executorCpus, Some("host2"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("1" -> 1.0)))), + rp.id), + WorkerOffer("executor2", "host2", executorCpus, Some("host3"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("2" -> 1.0)))), + rp.id), + WorkerOffer("executor3", "host3", executorCpus, Some("host4"), + new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("3" -> 1.0)))), + rp.id) + ) - taskScheduler.submitTasks(taskSet) - // Launch tasks on executor that satisfies resource requirements. - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(4 === taskDescriptions.length) - assert(!failedTaskSet) - - val assignedExecutorsGpus = ArrayBuffer[(String, String)]() - for (tDesc <- taskDescriptions) { - assert(tDesc.resources.contains(GPU)) - assert(tDesc.resources.get(GPU).get.keys.size == 1) - val address = tDesc.resources.get(GPU).get.keys.toArray.sorted - assignedExecutorsGpus.append((tDesc.executorId, address(0))) + taskScheduler.submitTasks(taskSet) + // Launch tasks on executor that satisfies resource requirements + + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(4 * taskNum === taskDescriptions.length) + assert(!failedTaskSet) + val assignedGpus: HashMap[String, Int] = HashMap.empty + for (taskId <- 0 until 4 * taskNum) { + val gpus = taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted + assert(gpus.length == 1) + val addr = gpus(0) + if (!assignedGpus.contains(addr)) { + assignedGpus(addr) = 1 + } else { + assignedGpus(addr) += 1 + } + } + assert(assignedGpus.toMap === + Map("0" -> taskNum, "1" -> taskNum, "2" -> taskNum, "3" -> taskNum)) } - - assert(assignedExecutorsGpus.sorted sameElements - ArrayBuffer( - ("executor0", "0"), - ("executor1", "7"), - ("executor2", "9"), - ("executor3", "20") - )) } } @@ -2516,7 +2647,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } test("SPARK-45527 TaskResourceProfile: the left multiple gpu resources on 1 executor " + - "can't assign to other taskset due to no enough gpu resource") { + "can't assign to other taskset due to not enough gpu resource") { val taskCpus = 1 val taskGpus = 0.4 val executorGpus = 4 @@ -2582,61 +2713,4 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten assert(3 === taskDescriptions.length) } - - Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => { - test(s"SPARK-45527 schedule tasks for a barrier taskSet if all tasks can be launched " + - s"together diff ResourceProfile with task.gpu.amount: ${gpuTaskAmount}, " + - s"but not enough gpus") { - val taskCpus = 1 - val taskScheduler = setupSchedulerWithMaster( - s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) - val execReqs = new ExecutorResourceRequests().cores(100).resource("gpu", 2) - val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", gpuTaskAmount) - val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) - taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) - - val numFreeCores = 100 - // make each of the worker offers only have 1 GPU, thus making it not enough - val workerOffers = IndexedSeq( - new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), - Map("gpu" -> Seq("0").toBuffer), rp.id), - new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), - Map("gpu" -> Seq("0").toBuffer), rp.id)) - val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) - - taskScheduler.submitTasks(attempt1) - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(0 === taskDescriptions.length) - } - } - } - - Seq(0.51, 0.6, 0.7, 0.8, 0.9, 1.0).foreach { gpuTaskAmount => { - test(s"SPARK-45527 schedule tasks for a barrier taskSet if all tasks can be launched " + - s"together diff ResourceProfile with task.gpu.amount: ${gpuTaskAmount}") { - val taskCpus = 1 - val taskScheduler = setupSchedulerWithMaster( - s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) - val execReqs = new ExecutorResourceRequests().cores(100).resource("gpu", 2) - val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", gpuTaskAmount) - val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) - taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) - - val numFreeCores = 100 - // make each of the worker offers only have 1 GPU, thus making it not enough - val workerOffers = IndexedSeq( - new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), - Map("gpu" -> Seq("0").toBuffer), rp.id), - new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), - Map("gpu" -> Seq("0").toBuffer), rp.id), - new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49628"), - Map("gpu" -> Seq("0").toBuffer), rp.id)) - val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) - - taskScheduler.submitTasks(attempt1) - val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten - assert(3 === taskDescriptions.length) - } - } - } } From e9e7a26c858d403454b6e61e07bac95537d3f624 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Thu, 16 Nov 2023 10:09:40 +0800 Subject: [PATCH 12/18] Comments --- .../spark/deploy/master/WorkerInfo.scala | 8 +- .../CoarseGrainedExecutorBackend.scala | 15 ---- .../spark/resource/ResourceAllocator.scala | 43 ++++++++--- .../scheduler/ExecutorResourcesAmounts.scala | 42 +++++------ .../spark/scheduler/TaskDescription.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../CoarseGrainedExecutorBackendSuite.scala | 25 +++---- .../CoarseGrainedSchedulerBackendSuite.scala | 6 +- .../scheduler/ExecutorResourceInfoSuite.scala | 8 +- .../ExecutorResourcesAmountsSuite.scala | 67 +++++++++-------- .../scheduler/TaskDescriptionSuite.scala | 9 ++- .../scheduler/TaskSchedulerImplSuite.scala | 73 ++++++++++--------- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- 13 files changed, 153 insertions(+), 151 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index a8b719590bb1..bdc9e9c6106c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.master import scala.collection.mutable import org.apache.spark.resource.{ResourceAllocator, ResourceInformation, ResourceRequirement} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -41,7 +41,7 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String val addresses = availableAddrs.take(amount) assert(addresses.length == amount) - acquire(addresses.map(addr => addr -> RESOURCE_TOTAL_AMOUNT).toMap) + acquire(addresses.map(addr => addr -> ONE_ENTIRE_RESOURCE).toMap) new ResourceInformation(resourceName, addresses.toArray) } } @@ -171,7 +171,7 @@ private[spark] class WorkerInfo( */ def recoverResources(expected: Map[String, ResourceInformation]): Unit = { expected.foreach { case (rName, rInfo) => - resources(rName).acquire(rInfo.addresses.map(addr => addr -> RESOURCE_TOTAL_AMOUNT).toMap) + resources(rName).acquire(rInfo.addresses.map(addr => addr -> ONE_ENTIRE_RESOURCE).toMap) } } @@ -181,7 +181,7 @@ private[spark] class WorkerInfo( */ private def releaseResources(allocated: Map[String, ResourceInformation]): Unit = { allocated.foreach { case (rName, rInfo) => - resources(rName).release(rInfo.addresses.map(addrs => addrs -> RESOURCE_TOTAL_AMOUNT).toMap) + resources(rName).release(rInfo.addresses.map(addrs => addrs -> ONE_ENTIRE_RESOURCE).toMap) } } } 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 34f530ae0cad..05ab61bb9030 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -20,7 +20,6 @@ package org.apache.spark.executor import java.net.URL import java.nio.ByteBuffer import java.util.Locale -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean import scala.util.{Failure, Success} @@ -68,16 +67,6 @@ private[spark] class CoarseGrainedExecutorBackend( private var _resources = Map.empty[String, ResourceInformation] - /** - * Map each taskId to the information about the resource allocated to it, Please refer to - * [[ResourceInformation]] for specifics. - * CHM is used to ensure thread-safety (https://issues.apache.org/jira/browse/SPARK-45227) - * Exposed for testing only. - */ - private[executor] val taskResources = new ConcurrentHashMap[ - Long, Map[String, ResourceInformation] - ] - private var decommissioned = false override def onStart(): Unit = { @@ -194,7 +183,6 @@ private[spark] class CoarseGrainedExecutorBackend( // Convert resources amounts into ResourceInformation val resources = taskDesc.resources.map { case (rName, addressesAmounts) => rName -> new ResourceInformation(rName, addressesAmounts.keys.toSeq.sorted.toArray)} - taskResources.put(taskDesc.taskId, resources) executor.launchTask(this, taskDesc) } @@ -277,9 +265,6 @@ private[spark] class CoarseGrainedExecutorBackend( val resources = executor.runningTasks.get(taskId).taskDescription.resources val cpus = executor.runningTasks.get(taskId).taskDescription.cpus val msg = StatusUpdate(executorId, taskId, state, data, cpus, 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/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 930bf81336ef..0055712667d8 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -20,7 +20,7 @@ package org.apache.spark.resource import scala.collection.mutable import org.apache.spark.SparkException -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE private[spark] object ResourceAmountUtils { /** @@ -50,11 +50,30 @@ private[spark] object ResourceAmountUtils { * assign 0.1111111111111111 for task 8, total left: 0.11111111111111094 * ERROR Can't assign 0.1111111111111111 for task 9, total left: 0.11111111111111094 * - * So we multiply RESOURCE_TOTAL_AMOUNT to convert the double to long to avoid this limitation. + * So we multiply ONE_ENTIRE_RESOURCE to convert the double to long to avoid this limitation. * Double can display up to 16 decimal places, so we set the factor to * 10, 000, 000, 000, 000, 000L. */ - final val RESOURCE_TOTAL_AMOUNT: Long = 10000000000000000L + final val ONE_ENTIRE_RESOURCE: Long = 10000000000000000L + + def isOneEntireResource(amount: Long): Boolean = amount == ONE_ENTIRE_RESOURCE + + def toInternalResource(amount: Double): Long = (amount * ONE_ENTIRE_RESOURCE).toLong + + private[spark] def toInternalResource(resources: Map[String, Double]): Map[String, Long] = { + resources.map { case (k, v) => k -> toInternalResource(v) } + } + + def toFractionalResource(amount: Long): Double = amount.toDouble / ONE_ENTIRE_RESOURCE + + private[spark] def toFractionalResource(resources: Map[String, Long]): Map[String, Double] = { + resources.map { case (k, v) => k -> toFractionalResource(v) } + } + + private[spark] def toInternalResourceMapMap(resources: Map[String, Map[String, Double]]): + Map[String, Map[String, Long]] = { + resources.map { case (k, v) => k -> toInternalResource(v) } + } } /** @@ -67,16 +86,16 @@ private[spark] trait ResourceAllocator { protected def resourceAddresses: Seq[String] /** - * Map from an address to its availability default to 1.0 (we multiply RESOURCE_TOTAL_AMOUNT + * Map from an address to its availability default to 1.0 (we multiply ONE_ENTIRE_RESOURCE * to avoid precision error), a value > 0 means the address is available, while value of * 0 means the address is fully assigned. */ private lazy val addressAvailabilityMap = { - mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*) + mutable.HashMap(resourceAddresses.map(address => address -> ONE_ENTIRE_RESOURCE): _*) } /** - * Get the amounts of resources that have been multiplied by RESOURCE_TOTAL_AMOUNT. + * Get the amounts of resources that have been multiplied by ONE_ENTIRE_RESOURCE. * @return the resources amounts */ def resourcesAmounts: Map[String, Long] = addressAvailabilityMap.toMap @@ -91,7 +110,7 @@ private[spark] trait ResourceAllocator { * Sequence of currently assigned resource addresses. */ private[spark] def assignedAddrs: Seq[String] = addressAvailabilityMap - .filter(addresses => addresses._2 < RESOURCE_TOTAL_AMOUNT).keys.toSeq.sorted + .filter(addresses => addresses._2 < ONE_ENTIRE_RESOURCE).keys.toSeq.sorted /** * Acquire a sequence of resource addresses (to a launched task), these addresses must be @@ -108,8 +127,8 @@ private[spark] trait ResourceAllocator { if (left < 0) { throw new SparkException(s"Try to acquire $resourceName address $address " + - s"amount: ${amount.toDouble / RESOURCE_TOTAL_AMOUNT}, but only " + - s"${prevAmount.toDouble / RESOURCE_TOTAL_AMOUNT} left.") + s"amount: ${ResourceAmountUtils.toFractionalResource(amount)}, but only " + + s"${ResourceAmountUtils.toFractionalResource(prevAmount)} left.") } else { addressAvailabilityMap(address) = left } @@ -129,10 +148,10 @@ private[spark] trait ResourceAllocator { val total = prevAmount + amount - if (total > RESOURCE_TOTAL_AMOUNT) { + if (total > ONE_ENTIRE_RESOURCE) { throw new SparkException(s"Try to release $resourceName address $address " + - s"amount: ${amount.toDouble / RESOURCE_TOTAL_AMOUNT}. But the total amount: " + - s"${total.toDouble / RESOURCE_TOTAL_AMOUNT} " + + s"amount: ${ResourceAmountUtils.toFractionalResource(amount)}. But the total amount: " + + s"${ResourceAmountUtils.toFractionalResource(total)} " + s"after release should be <= 1") } else { addressAvailabilityMap(address) = total diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index 946b289f7b1d..44053d33fcfa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -20,8 +20,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap import org.apache.spark.SparkException -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT -import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.{ResourceAmountUtils, ResourceProfile} +import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE /** * Class to hold information about a series of resources belonging to an executor. @@ -32,10 +32,10 @@ import org.apache.spark.resource.ResourceProfile * One example is GPUs, where the addresses would be the indices of the GPUs * * @param resources The executor available resources and amount. eg, - * Map("gpu" -> Map("0" -> 0.2*RESOURCE_TOTAL_AMOUNT, - * "1" -> 1.0*RESOURCE_TOTAL_AMOUNT), - * "fpga" -> Map("a" -> 0.3*RESOURCE_TOTAL_AMOUNT, - * "b" -> 0.9*RESOURCE_TOTAL_AMOUNT) + * Map("gpu" -> Map("0" -> ResourceAmountUtils.toInternalResource(0.2), + * "1" -> ResourceAmountUtils.toInternalResource(1.0)), + * "fpga" -> Map("a" -> ResourceAmountUtils.toInternalResource(0.3), + * "b" -> ResourceAmountUtils.toInternalResource(0.9)) * ) */ private[spark] class ExecutorResourcesAmounts( @@ -52,15 +52,15 @@ private[spark] class ExecutorResourcesAmounts( /** * The total address count of each resource. Eg, - * Map("gpu" -> Map("0" -> 0.5 * RESOURCE_TOTAL_AMOUNT, - * "1" -> 0.5 * RESOURCE_TOTAL_AMOUNT, - * "2" -> 0.5 * RESOURCE_TOTAL_AMOUNT), - * "fpga" -> Map("a" -> 0.5 * RESOURCE_TOTAL_AMOUNT, - * "b" -> 0.5 * RESOURCE_TOTAL_AMOUNT)) + * Map("gpu" -> Map("0" -> ResourceAmountUtils.toInternalResource(0.5), + * "1" -> ResourceAmountUtils.toInternalResource(0.5), + * "2" -> ResourceAmountUtils.toInternalResource(0.5)), + * "fpga" -> Map("a" -> ResourceAmountUtils.toInternalResource(0.5), + * "b" -> ResourceAmountUtils.toInternalResource(0.5))) * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2) */ - lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) => - rName -> addressMap.size + lazy val resourceAddressAmount: Map[String, Int] = internalResources.map { + case (rName, addressMap) => rName -> addressMap.size } /** @@ -69,7 +69,7 @@ private[spark] class ExecutorResourcesAmounts( private[spark] def availableResources: Map[String, Map[String, Double]] = { internalResources.map { case (rName, addressMap) => rName -> addressMap.map { case (address, amount) => - address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT + address -> ResourceAmountUtils.toFractionalResource(amount) }.toMap } } @@ -89,7 +89,8 @@ private[spark] class ExecutorResourcesAmounts( val left = prevInternalTotalAmount - amount if (left < 0) { - throw new SparkException(s"The total amount ${left.toDouble / RESOURCE_TOTAL_AMOUNT} " + + throw new SparkException(s"The total amount " + + s"${ResourceAmountUtils.toFractionalResource(left)} " + s"after acquiring $rName address $address should be >= 0") } internalResources(rName)(address) = left @@ -110,9 +111,9 @@ private[spark] class ExecutorResourcesAmounts( throw new SparkException(s"Try to release an address that is not assigned. $rName " + s"address $address is not assigned.")) val total = prevInternalTotalAmount + amount - if (total > RESOURCE_TOTAL_AMOUNT) { + if (total > ONE_ENTIRE_RESOURCE) { throw new SparkException(s"The total amount " + - s"${total.toDouble / RESOURCE_TOTAL_AMOUNT} " + + s"${ResourceAmountUtils.toFractionalResource(total)} " + s"after releasing $rName address $address should be <= 1.0") } internalResources(rName)(address) = total @@ -128,7 +129,6 @@ private[spark] class ExecutorResourcesAmounts( * @return the optional resources amounts */ def assignResources(taskSetProf: ResourceProfile): Option[Map[String, Map[String, Long]]] = { - // only look at the resource other than cpus val tsResources = taskSetProf.getCustomTaskResources() if (tsResources.isEmpty) { @@ -154,14 +154,14 @@ private[spark] class ExecutorResourcesAmounts( if (taskAmount >= 1.0) { for (address <- addresses if taskAmount > 0) { // The address is still a whole resource - if (addressesAmountMap(address) == RESOURCE_TOTAL_AMOUNT) { + if (ResourceAmountUtils.isOneEntireResource(addressesAmountMap(address))) { taskAmount -= 1.0 // Assign the full resource of the address - allocatedAddressesMap(address) = RESOURCE_TOTAL_AMOUNT + allocatedAddressesMap(address) = ONE_ENTIRE_RESOURCE } } } else if (taskAmount > 0.0) { // 0 < task.amount < 1.0 - val internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong + val internalTaskAmount = ResourceAmountUtils.toInternalResource(taskAmount) for (address <- addresses if taskAmount > 0) { if (addressesAmountMap(address) >= internalTaskAmount) { // Assign the part of the address. 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 459865d7a166..df5f32612bea 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -57,7 +57,7 @@ private[spark] class TaskDescription( val properties: Properties, val cpus: Int, // resources is the total resources assigned to the task - // Eg, Map("gpu" -> Map("0" -> 0.7*RESOURCE_TOTAL_AMOUNT)): + // Eg, Map("gpu" -> Map("0" -> ResourceAmountUtils.toInternalResource(0.7))): // assign 0.7 of the gpu address "0" to this task val resources: immutable.Map[String, immutable.Map[String, Long]], val serializedTask: ByteBuffer) { 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 822911a48beb..bbdc70f8e9b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -551,7 +551,7 @@ private[spark] class TaskSchedulerImpl( // value is -1 val numBarrierSlotsAvailable = if (taskSet.isBarrier) { val rpId = taskSet.taskSet.resourceProfileId - val resAmounts = availableResources.map(_.resourceAmount) + val resAmounts = availableResources.map(_.resourceAddressAmount) calculateAvailableSlots(this, conf, rpId, resourceProfileIds, availableCpus, resAmounts) } else { -1 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 da59bd83fb82..9a89ef117321 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -39,7 +39,6 @@ import org.apache.spark.TestUtils._ import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} import org.apache.spark.internal.config.PLUGINS import org.apache.spark.resource._ -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEnv @@ -305,12 +304,11 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", "host1", "host1", 4, env, None, resourceProfile = ResourceProfile.getOrCreateDefaultProfile(conf)) - assert(backend.taskResources.isEmpty) val taskId = 1000000L val resourcesAmounts = Map(GPU -> Map( - "0" -> (0.15*RESOURCE_TOTAL_AMOUNT).toLong, - "1" -> (0.76*RESOURCE_TOTAL_AMOUNT).toLong)) + "0" -> ResourceAmountUtils.toInternalResource(0.15), + "1" -> ResourceAmountUtils.toInternalResource(0.76))) // 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, @@ -345,25 +343,22 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // 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(runningTasks.size == 1) - val resources = backend.taskResources.get(taskId) - assert(resources(GPU).addresses sameElements Array("0", "1")) + val resources = backend.executor.runningTasks.get(taskId).taskDescription.resources + assert(resources(GPU).keys.toArray.sorted sameElements Array("0", "1")) assert(executor.runningTasks.get(taskId).taskDescription.resources === resourcesAmounts) } // Update the status of a running task shall not affect `taskResources` map. backend.statusUpdate(taskId, TaskState.RUNNING, data) - assert(backend.taskResources.size == 1) - val resources = backend.taskResources.get(taskId) - assert(resources(GPU).addresses sameElements Array("0", "1")) + val resources = backend.executor.runningTasks.get(taskId).taskDescription.resources + assert(resources(GPU).keys.toArray.sorted sameElements Array("0", "1")) assert(executor.runningTasks.get(taskId).taskDescription.resources === resourcesAmounts) // 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() @@ -432,8 +427,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val tasksExecuted = new TrieMap[Long, Boolean]() val resourcesAmounts = Map(GPU -> Map( - "0" -> (0.15 * RESOURCE_TOTAL_AMOUNT).toLong, - "1" -> (0.76 * RESOURCE_TOTAL_AMOUNT).toLong)) + "0" -> ResourceAmountUtils.toInternalResource(0.15), + "1" -> ResourceAmountUtils.toInternalResource(0.76))) // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { @@ -524,8 +519,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val tasksExecuted = new TrieMap[Long, Boolean]() val resourcesAmounts = Map(GPU -> Map( - "0" -> (0.15*RESOURCE_TOTAL_AMOUNT).toLong, - "1" -> (0.76*RESOURCE_TOTAL_AMOUNT).toLong)) + "0" -> ResourceAmountUtils.toInternalResource(0.15), + "1" -> ResourceAmountUtils.toInternalResource(0.76))) // Fake tasks with different taskIds. val taskDescriptions = (1 to numTasks).map { 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 04e50f0e5f5c..c24103fdf037 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -37,7 +37,7 @@ 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.resource.{ExecutorResourceRequests, ResourceInformation, ResourceProfile, TaskResourceRequests} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout} @@ -255,7 +255,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") assert(exec3ResourceProfileId === rp.id) - val taskResources = Map(GPU -> Map("0" -> RESOURCE_TOTAL_AMOUNT)) + val taskResources = Map(GPU -> Map("0" -> ONE_ENTIRE_RESOURCE)) val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), @@ -362,7 +362,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") assert(exec3ResourceProfileId === rp.id) - val taskResources = Map(GPU -> Map("0" -> RESOURCE_TOTAL_AMOUNT)) + val taskResources = Map(GPU -> Map("0" -> ONE_ENTIRE_RESOURCE)) val taskCpus = 1 val taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, JobArtifactSet.emptyJobArtifactSet, new Properties(), 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 1614101a955c..ac1a6b7c3ec0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -21,17 +21,17 @@ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceAmountUtils import org.apache.spark.resource.ResourceUtils.GPU class ExecutorResourceInfoSuite extends SparkFunSuite { implicit def convertMapLongToDouble(resources: Map[String, Long]): Map[String, Double] = { - resources.map { case (k, v) => k -> v.toDouble / RESOURCE_TOTAL_AMOUNT }.toMap + resources.map { case (k, v) => k -> ResourceAmountUtils.toFractionalResource(v) } } implicit def convertMapDoubleToLong(resources: Map[String, Double]): Map[String, Long] = { - resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong }.toMap + resources.map { case (k, v) => k -> ResourceAmountUtils.toInternalResource(v) } } test("Track Executor Resource information") { @@ -116,7 +116,7 @@ class ExecutorResourceInfoSuite extends SparkFunSuite { // All addresses has been assigned assert(info.resourcesAmounts.values.toSeq.toSet.size == 1) // The left amount of any address should < taskAmount - assert(info.resourcesAmounts("0").toDouble/RESOURCE_TOTAL_AMOUNT < taskAmount) + assert(ResourceAmountUtils.toFractionalResource(info.resourcesAmounts("0")) < taskAmount) addresses.foreach { addr => assertThrows[SparkException] { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala index e0eddd4f3b52..eb6129f84bb6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala @@ -21,24 +21,20 @@ import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.{ResourceAmountUtils, ResourceProfileBuilder, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils.GPU class ExecutorResourcesAmountsSuite extends SparkFunSuite { - implicit def convertMapLongToMapDouble(resources: Map[String, Long]): Map[String, Double] = { - resources.map { case (k, v) => k -> v.toDouble / RESOURCE_TOTAL_AMOUNT } - } + implicit def toFractionalResource(resources: Map[String, Long]): Map[String, Double] = + ResourceAmountUtils.toFractionalResource(resources) - implicit def convertMapDoubleToLong(resources: Map[String, Double]): Map[String, Long] = { - resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong } - } + implicit def toInternalResource(resources: Map[String, Double]): Map[String, Long] = + ResourceAmountUtils.toInternalResource(resources) + + implicit def toInternalResourceMap(resources: Map[String, Map[String, Double]]): + Map[String, Map[String, Long]] = ResourceAmountUtils.toInternalResourceMapMap(resources) - implicit def convertResMapDoubleToMapLong(resources: Map[String, Map[String, Double]]): - Map[String, Map[String, Long]] = { - resources.map { case (k, v) => k -> convertMapDoubleToLong(v) } - } def compareMaps(lhs: Map[String, Double], rhs: Map[String, Double], eps: Double = 0.00000001): Boolean = { @@ -54,7 +50,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { "fpga" -> new ExecutorResourceInfo("fpga", Seq("aa", "bb")) ) val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) - assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + assert(availableExecResAmounts.resourceAddressAmount === Map("gpu" -> 3, "fpga" -> 2)) val treqs = new TaskResourceRequests().cpus(1) val rp = new ResourceProfileBuilder().require(treqs).build() @@ -80,7 +76,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) - assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + assert(availableExecResAmounts.resourceAddressAmount === Map("gpu" -> 3, "fpga" -> 2)) val availableRes = availableExecResAmounts.availableResources availableRes.foreach { case (rName, addressesAmount) => @@ -95,9 +91,9 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // executors info shouldn't be changed. executorsInfo.foreach { case (rName, rInfo) => if (rName == "gpu") { - rInfo.acquire(convertMapDoubleToLong(Map("2" -> 0.4, "6" -> 0.6))) + rInfo.acquire(toInternalResource(Map("2" -> 0.4, "6" -> 0.6))) } else { - rInfo.acquire(convertMapDoubleToLong(Map("aa" -> 0.2, "bb" -> 0.7))) + rInfo.acquire(toInternalResource(Map("aa" -> 0.2, "bb" -> 0.7))) } } @@ -110,7 +106,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } val availableExecResAmounts1 = ExecutorResourcesAmounts(executorsInfo) - assert(availableExecResAmounts1.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + assert(availableExecResAmounts1.resourceAddressAmount === Map("gpu" -> 3, "fpga" -> 2)) val availableRes1 = availableExecResAmounts1.availableResources availableRes1.foreach { case (rName, addressesAmount) => @@ -139,7 +135,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { } val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) - assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + assert(availableExecResAmounts.resourceAddressAmount === Map("gpu" -> 3, "fpga" -> 2)) val gpuTaskAmount = 0.1 val treqs = new TaskResourceRequests().resource("gpu", gpuTaskAmount) @@ -213,11 +209,11 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) val e = intercept[SparkException] { - availableExecResAmounts.release(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.7)))) + availableExecResAmounts.release(toInternalResourceMap(Map("gpu" -> Map("2" -> 0.7)))) } assert(e.getMessage.contains("after releasing gpu address 2 should be <= 1.0")) - availableExecResAmounts.release(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.6)))) + availableExecResAmounts.release(toInternalResourceMap(Map("gpu" -> Map("2" -> 0.6)))) assert(compareMaps(availableExecResAmounts.availableResources("gpu"), Map("2" -> 1.0))) } @@ -226,11 +222,11 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = new ExecutorResourcesAmounts(totalRes) val e = intercept[SparkException] { - availableExecResAmounts.acquire(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.6)))) + availableExecResAmounts.acquire(toInternalResourceMap(Map("gpu" -> Map("2" -> 0.6)))) } assert(e.getMessage.contains("after acquiring gpu address 2 should be >= 0")) - availableExecResAmounts.acquire(Map("gpu" -> convertMapDoubleToLong(Map("2" -> 0.4)))) + availableExecResAmounts.acquire(toInternalResourceMap(Map("gpu" -> Map("2" -> 0.4)))) assert(compareMaps(availableExecResAmounts.availableResources("gpu"), Map("2" -> 0.0))) } @@ -245,7 +241,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { for (_ <- 0 until slots) { addresses.foreach(addr => availableExecResAmounts.acquire( - Map(GPU -> convertMapDoubleToLong(Map(addr -> taskAmount))))) + toInternalResourceMap(Map(GPU -> Map(addr -> taskAmount))))) } assert(availableExecResAmounts.availableResources.size === 1) @@ -257,7 +253,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { addresses.foreach { addr => assertThrows[SparkException] { availableExecResAmounts.acquire( - Map(GPU -> convertMapDoubleToLong(Map(addr -> taskAmount)))) + toInternalResourceMap(Map(GPU -> Map(addr -> taskAmount)))) } } } @@ -271,7 +267,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) - assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + assert(availableExecResAmounts.resourceAddressAmount === Map("gpu" -> 3, "fpga" -> 2)) val gpuTaskAmount = 0.1 val treqs = new TaskResourceRequests().resource("gpu", gpuTaskAmount) @@ -285,7 +281,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { assert(resource.keys.toSeq === Seq("gpu")) assert(resource("gpu").size === 1) assert(resource("gpu").keys.toSeq === Seq("2")) - assert(resource("gpu")("2").toDouble / RESOURCE_TOTAL_AMOUNT === gpuTaskAmount) + assert(ResourceAmountUtils.toFractionalResource(resource("gpu")("2")) === gpuTaskAmount) } // assign will not update the real value. @@ -335,7 +331,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val availableExecResAmounts = ExecutorResourcesAmounts(executorsInfo) - assert(availableExecResAmounts.resourceAmount === Map("gpu" -> 3, "fpga" -> 2)) + assert(availableExecResAmounts.resourceAddressAmount === Map("gpu" -> 3, "fpga" -> 2)) val gpuTaskAmount = 0.1 val fpgaTaskAmount = 0.3 @@ -353,11 +349,13 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { assert(resourceAmounts("gpu").size === 1) assert(resourceAmounts("gpu").keys.toSeq === Seq("2")) - assert(resourceAmounts("gpu")("2").toDouble / RESOURCE_TOTAL_AMOUNT === gpuTaskAmount) + assert(ResourceAmountUtils.toFractionalResource(resourceAmounts("gpu")("2")) === + gpuTaskAmount) assert(resourceAmounts("fpga").size === 1) assert(resourceAmounts("fpga").keys.toSeq === Seq("aa")) - assert(resourceAmounts("fpga")("aa").toDouble / RESOURCE_TOTAL_AMOUNT === fpgaTaskAmount) + assert(ResourceAmountUtils.toFractionalResource(resourceAmounts("fpga")("aa")) === + fpgaTaskAmount) } // assign will not update the real value. @@ -411,7 +409,8 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val assigned = availableExecResAmounts.assignResources(rp) assert(!assigned.isEmpty) assigned.foreach { case resources => - assert(resources("gpu").values.toArray.sorted.map(_.toDouble / RESOURCE_TOTAL_AMOUNT) + assert( + resources("gpu").values.toArray.sorted.map(ResourceAmountUtils.toFractionalResource(_)) === expectedAssignedAmount.sorted) availableExecResAmounts.acquire(resources) @@ -511,13 +510,13 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // Acquire an address from a resource that doesn't exist val e = intercept[SparkException] { - availableExecResAmounts.acquire(Map("fpga" -> convertMapDoubleToLong(Map("1" -> 1.0)))) + availableExecResAmounts.acquire(toInternalResourceMap(Map("fpga" -> Map("1" -> 1.0)))) } assert(e.getMessage.contains("Try to acquire an address from fpga that doesn't exist")) // Acquire an address that is not available val e1 = intercept[SparkException] { - availableExecResAmounts.acquire(Map("gpu" -> convertMapDoubleToLong(Map("6" -> 1.0)))) + availableExecResAmounts.acquire(toInternalResourceMap(Map("gpu" -> Map("6" -> 1.0)))) } assert(e1.getMessage.contains("Try to acquire an address that doesn't exist")) } @@ -529,13 +528,13 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { // Acquire an address from a resource that doesn't exist val e = intercept[SparkException] { - availableExecResAmounts.release(Map("fpga" -> convertMapDoubleToLong(Map("1" -> 0.1)))) + availableExecResAmounts.release(toInternalResourceMap(Map("fpga" -> Map("1" -> 0.1)))) } assert(e.getMessage.contains("Try to release an address from fpga that doesn't exist")) // Acquire an address that is not available val e1 = intercept[SparkException] { - availableExecResAmounts.release(Map("gpu" -> convertMapDoubleToLong(Map("6" -> 0.1)))) + availableExecResAmounts.release(toInternalResourceMap(Map("gpu" -> Map("6" -> 0.1)))) } assert(e1.getMessage.contains("Try to release an address that is not assigned")) } 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 a2a7377899af..5aaacd2ec1c3 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.{JobArtifactSet, SparkFunSuite} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceAmountUtils import org.apache.spark.resource.ResourceUtils.GPU class TaskDescriptionSuite extends SparkFunSuite { @@ -59,9 +59,10 @@ class TaskDescriptionSuite extends SparkFunSuite { } } - val originalResources = Map(GPU -> Map("1" -> (0.2 * RESOURCE_TOTAL_AMOUNT).toLong, - "2" -> (0.5 * RESOURCE_TOTAL_AMOUNT).toLong, - "3" -> (0.1 * RESOURCE_TOTAL_AMOUNT).toLong)) + val originalResources = Map(GPU -> + Map("1" -> ResourceAmountUtils.toInternalResource(0.2), + "2" -> ResourceAmountUtils.toInternalResource(0.5), + "3" -> ResourceAmountUtils.toInternalResource(0.1))) // 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/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 29c3a19fada7..34d47e2edc32 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -35,8 +35,8 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.config -import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceProfile, TaskResourceRequests} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceAmountUtils, ResourceProfile, TaskResourceProfile, TaskResourceRequests} +import org.apache.spark.resource.ResourceAmountUtils.{ONE_ENTIRE_RESOURCE} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.status.api.v1.ThreadStackTrace @@ -149,11 +149,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } // Convert resources to ExecutorResourcesAmounts automatically - implicit def convertResources(resources: Map[String, mutable.Buffer[String]]): + implicit def toExecutorResourcesAmounts(resources: Map[String, mutable.Buffer[String]]): ExecutorResourcesAmounts = { // convert the old resources to ExecutorResourcesAmounts new ExecutorResourcesAmounts(resources.map { case (rName, addresses) => - rName -> addresses.map(address => address -> RESOURCE_TOTAL_AMOUNT).toMap + rName -> addresses.map(address => address -> ONE_ENTIRE_RESOURCE).toMap }) } @@ -1546,7 +1546,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val numFreeCores = 3 val workerOffers = IndexedSeq( new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), - convertResources(Map("gpu" -> Seq("0").toBuffer))), + Map("gpu" -> Seq("0").toBuffer)), new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), Map("gpu" -> Seq("0").toBuffer)), new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"), @@ -1937,7 +1937,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val availableResourcesAmount = workerOffers.map(_.resources).map { resAmounts => // available addresses already takes into account if there are fractional // task resource requests - resAmounts.resourceAmount + resAmounts.resourceAddressAmount } val taskSlotsForRp = TaskSchedulerImpl.calculateAvailableSlots( @@ -2295,15 +2295,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext taskScheduler.handleFailedTask(tsm, tid, state, reason) } - private implicit def convertMapDoubleToLong(resources: Map[String, Double]): Map[String, Long] = { - resources.map { case (k, v) => k -> (v * RESOURCE_TOTAL_AMOUNT).toLong } - } + private implicit def toInternalResource(resources: Map[String, Double]): Map[String, Long] = + ResourceAmountUtils.toInternalResource(resources) // 1 executor with 4 GPUS Seq(true, false).foreach { barrierMode => val barrier = if (barrierMode) "barrier" else "" (1 to 20).foreach { taskNum => - val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum) test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + s"restrict $taskNum $barrier tasks run in the same executor") { val taskCpus = 1 @@ -2323,7 +2322,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } val resources = new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) val workerOffers = IndexedSeq(WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources)) @@ -2349,7 +2348,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext Seq(true, false).foreach { barrierMode => val barrier = if (barrierMode) "barrier" else "" (1 to 20).foreach { taskNum => - val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum) test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " + s"restrict $taskNum $barrier tasks run on the different executor") { val taskCpus = 1 @@ -2371,13 +2370,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val workerOffers = IndexedSeq( WorkerOffer("executor0", "host0", executorCpus, Some("host0"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0))))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("0" -> 1.0))))), WorkerOffer("executor1", "host1", executorCpus, Some("host1"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("1" -> 1.0))))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("1" -> 1.0))))), WorkerOffer("executor2", "host2", executorCpus, Some("host2"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("2" -> 1.0))))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("2" -> 1.0))))), WorkerOffer("executor3", "host3", executorCpus, Some("host3"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("3" -> 1.0)))))) + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("3" -> 1.0)))))) taskScheduler.submitTasks(taskSet) // Launch tasks on executor that satisfies resource requirements @@ -2406,7 +2405,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext Seq(true, false).foreach { barrierMode => val barrier = if (barrierMode) "barrier" else "" (1 to 20).foreach { taskNum => - val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum) test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " + s"restrict $taskNum $barrier tasks run in the same executor") { val executorCpus = 100 // cpu will not limit the concurrent tasks number @@ -2427,7 +2426,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id) } val resources = new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) + Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))) val workerOffers = IndexedSeq( WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources, rp.id) @@ -2454,7 +2453,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext Seq(true, false).foreach { barrierMode => val barrier = if (barrierMode) "barrier" else "" (1 to 20).foreach { taskNum => - val gpuTaskAmount = (RESOURCE_TOTAL_AMOUNT / taskNum).toDouble / RESOURCE_TOTAL_AMOUNT + val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum) test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " + s"restrict $taskNum $barrier tasks run on the different executor") { val executorCpus = 100 // cpu will not limit the concurrent tasks number @@ -2478,16 +2477,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val workerOffers = IndexedSeq( WorkerOffer("executor0", "host0", executorCpus, Some("host1"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("0" -> 1.0)))), rp.id), WorkerOffer("executor1", "host1", executorCpus, Some("host2"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("1" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("1" -> 1.0)))), rp.id), WorkerOffer("executor2", "host2", executorCpus, Some("host3"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("2" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("2" -> 1.0)))), rp.id), WorkerOffer("executor3", "host3", executorCpus, Some("host4"), - new ExecutorResourcesAmounts(Map(GPU -> convertMapDoubleToLong(Map("3" -> 1.0)))), + new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("3" -> 1.0)))), rp.id) ) @@ -2543,7 +2542,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext IndexedSeq( // cpu won't be a problem WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))) + Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))) ) taskScheduler.submitTasks(lowerTaskSet) @@ -2559,11 +2558,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext assert(addresses.length == 1) if (index < 4) { // the first 4 tasks will grab 0.7 gpu assert(addresses(0) == index.toString) - assert(tDesc.resources.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) + assert(ResourceAmountUtils.toFractionalResource( + tDesc.resources.get(GPU).get(index.toString)) == 0.7) } else { assert(addresses(0) == (index - 4).toString) - assert(tDesc.resources.get(GPU).get((index - 4).toString).toDouble/RESOURCE_TOTAL_AMOUNT - == 0.3) + assert(ResourceAmountUtils.toFractionalResource( + tDesc.resources.get(GPU).get((index - 4).toString)) == 0.3) } index += 1 } @@ -2598,13 +2598,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext IndexedSeq( // cpu won't be a problem WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0))))), + Map(GPU -> toInternalResource(Map("0" -> 1.0))))), WorkerOffer("executor1", "host1", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("1" -> 1.0))))), + Map(GPU -> toInternalResource(Map("1" -> 1.0))))), WorkerOffer("executor2", "host2", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("2" -> 1.0))))), + Map(GPU -> toInternalResource(Map("2" -> 1.0))))), WorkerOffer("executor3", "host3", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("3" -> 1.0))))) + Map(GPU -> toInternalResource(Map("3" -> 1.0))))) ) taskScheduler.submitTasks(lowerTaskSet) @@ -2629,11 +2629,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext if (index % 2 == 0) { higherAssignedExecutorsGpus.append( (tDesc.executorId, address)) - assert(tDesc.resources.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) + assert(ResourceAmountUtils.toFractionalResource( + tDesc.resources.get(GPU).get(address)) == 0.7) } else { lowerAssignedExecutorsGpus.append( (tDesc.executorId, address)) - assert(tDesc.resources.get(GPU).get(address).toDouble/RESOURCE_TOTAL_AMOUNT == 0.3) + assert(ResourceAmountUtils.toFractionalResource( + tDesc.resources.get(GPU).get(address)) == 0.3) } index += 1 } @@ -2675,7 +2677,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext IndexedSeq( // cpu won't be a problem WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts( - Map(GPU -> convertMapDoubleToLong(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))) + Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))) ) taskScheduler.submitTasks(lowerTaskSet) @@ -2690,7 +2692,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext val addresses = tDesc.resources.get(GPU).get.keys.toArray.sorted assert(addresses.length == 1) assert(addresses(0) == index.toString) - assert(tDesc.resources.get(GPU).get(index.toString).toDouble/RESOURCE_TOTAL_AMOUNT == 0.7) + assert(ResourceAmountUtils.toFractionalResource( + tDesc.resources.get(GPU).get(index.toString)) == 0.7) index += 1 } } 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 09f80ec552b0..03f7a0b5ba41 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.{SKIP_VALIDATE_CORES_TESTING, TEST_DYNAMIC_ALLOCATION_SCHEDULE_ENABLED} -import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT +import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE import org.apache.spark.resource.ResourceProfile import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ @@ -1827,7 +1827,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) val taskResourceAssignments = Map( - GPU -> Map("0" -> RESOURCE_TOTAL_AMOUNT, "1" -> RESOURCE_TOTAL_AMOUNT)) + GPU -> Map("0" -> ONE_ENTIRE_RESOURCE, "1" -> ONE_ENTIRE_RESOURCE)) val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, 2, taskResourceAssignments)._1 assert(taskOption.isDefined) From 8657837815fe19feaa54aceb68498db9a3b5f540 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Tue, 21 Nov 2023 07:30:53 +0800 Subject: [PATCH 13/18] comments --- .../scala/org/apache/spark/resource/ResourceProfile.scala | 2 +- .../scala/org/apache/spark/resource/ResourceUtils.scala | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index ca69ad291308..b81121aacaa8 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -121,7 +121,7 @@ class ResourceProfile( /* * This function takes into account fractional amounts for the task resource requirement. * Spark only supports fractional amounts < 1 to basically allow for multiple tasks - * to use the same resource address. + * to use the same resource address or a whole number to use the multiple whole addresses. */ private[spark] def getSchedulerTaskResourceAmount(resource: String): Int = { val taskAmount = taskResources.getOrElse(resource, diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index bb53da089a79..7572e6519271 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -169,10 +169,9 @@ private[spark] object ResourceUtils extends Logging { // Used to take a fraction amount from a task resource requirement and split into a real // integer amount and the number of slots per address. For instance, if the amount is 0.5, - // the we get (1, 2) back out. This indicates that for each 1 address, it has 2 slots per - // address, which allows you to put 2 tasks on that address. Note if amount is greater - // than 1, then the number of parts per address has to be 1. This would indicate that a - // would have multiple addresses assigned per task. This can be used for calculating + // the we get (1, 2) back out. This indicates that for each 1 address, it allows you to + // put 2 tasks on that address. Note if amount is greater than 1, then the number of + // running tasks per address has to be 1. This can be used for calculating // the number of tasks per executor -> (executorAmount * numParts) / (integer amount). // Returns tuple of (integer amount, numParts) def calculateAmountAndPartsForFraction(doubleAmount: Double): (Int, Int) = { From 6ca3567d6ac4ced94b05535c002fdb5a08c87119 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Wed, 22 Nov 2023 10:08:39 +0800 Subject: [PATCH 14/18] comments --- .../main/scala/org/apache/spark/resource/ResourceProfile.scala | 2 +- .../main/scala/org/apache/spark/resource/ResourceUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index b81121aacaa8..f3d0e2d33d9d 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -120,7 +120,7 @@ class ResourceProfile( /* * This function takes into account fractional amounts for the task resource requirement. - * Spark only supports fractional amounts < 1 to basically allow for multiple tasks + * Spark only supports fractional amounts < 1 to basically allow for multiple tasks * to use the same resource address or a whole number to use the multiple whole addresses. */ private[spark] def getSchedulerTaskResourceAmount(resource: String): Int = { diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 7572e6519271..16bef9cf88c6 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -172,7 +172,7 @@ private[spark] object ResourceUtils extends Logging { // the we get (1, 2) back out. This indicates that for each 1 address, it allows you to // put 2 tasks on that address. Note if amount is greater than 1, then the number of // running tasks per address has to be 1. This can be used for calculating - // the number of tasks per executor -> (executorAmount * numParts) / (integer amount). + // the number of tasks per executor = (executorAmount * numParts) / (integer amount). // Returns tuple of (integer amount, numParts) def calculateAmountAndPartsForFraction(doubleAmount: Double): (Int, Int) = { val parts = if (doubleAmount <= 1.0) { From 3b08d1ea65a9110c9b2418641f055e491adc246e Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 27 Nov 2023 10:31:51 +0800 Subject: [PATCH 15/18] toIndexedSeq --- .../spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f8446036bc2e..9f1471f6088e 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 @@ -273,7 +273,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) val resourcesInfo = resources.map { case (rName, info) => - (info.name, new ExecutorResourceInfo(info.name, info.addresses)) + (info.name, new ExecutorResourceInfo(info.name, info.addresses.toIndexedSeq)) } // If we've requested the executor figure out when we did. val reqTs: Option[Long] = CoarseGrainedSchedulerBackend.this.synchronized { From c2da6e36b159616bf634f767d485e131bee575ae Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 4 Dec 2023 14:29:04 +0800 Subject: [PATCH 16/18] comments --- .../spark/resource/ResourceAllocator.scala | 4 +-- .../scheduler/ExecutorResourcesAmounts.scala | 31 ++++++++++++++----- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 6 ++-- .../ExecutorResourcesAmountsSuite.scala | 16 +++++----- 5 files changed, 36 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 0055712667d8..928e9e526738 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -123,7 +123,7 @@ private[spark] trait ResourceAllocator { throw new SparkException(s"Try to acquire an address that doesn't exist. $resourceName " + s"address $address doesn't exist.")) - val left = addressAvailabilityMap(address) - amount + val left = prevAmount - amount if (left < 0) { throw new SparkException(s"Try to acquire $resourceName address $address " + @@ -140,7 +140,7 @@ private[spark] trait ResourceAllocator { * addresses are released when a task has finished. * Throw an Exception if an address is not assigned or doesn't exist. */ - def release (addressesAmounts: Map[String, Long]): Unit = { + def release(addressesAmounts: Map[String, Long]): Unit = { addressesAmounts.foreach { case (address, amount) => val prevAmount = addressAvailabilityMap.getOrElse(address, throw new SparkException(s"Try to release an address that doesn't exist. $resourceName " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala index 44053d33fcfa..a93f2863ac2f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala @@ -75,7 +75,7 @@ private[spark] class ExecutorResourcesAmounts( } /** - * Acquire the resource and update the resource + * Acquire the resource. * @param assignedResource the assigned resource information */ def acquire(assignedResource: Map[String, Map[String, Long]]): Unit = { @@ -122,13 +122,26 @@ private[spark] class ExecutorResourcesAmounts( } /** - * Try to assign the addresses according to the task requirement. - * Please note that this function will not update the values. + * Try to assign the addresses according to the task requirement. This function always goes + * through the available resources starting from the "small" address. If the resources amount + * of the address is matching the task requirement, we will assign this address to this task. + * Eg, assuming the available resources are {"gpu" -> {"0"-> 0.7, "1" -> 1.0}) and the + * task requirement is 0.5, this function will return Some(Map("gpu" -> {"0" -> 0.5})). + * + * TODO: as we consistently allocate addresses beginning from the "small" address, it can + * potentially result in an undesired consequence where a portion of the resource is being wasted. + * Eg, assuming the available resources are {"gpu" -> {"0"-> 1.0, "1" -> 0.5}) and the + * task amount requirement is 0.5, this function will return + * Some(Map("gpu" -> {"0" -> 0.5})), and the left available resource will be + * {"gpu" -> {"0"-> 0.5, "1" -> 0.5}) which can't assign to the task that + * requires > 0.5 any more. * * @param taskSetProf assign resources based on which resource profile - * @return the optional resources amounts + * @return the optional assigned resources amounts. returns None if any + * of the task requests for resources aren't met. */ - def assignResources(taskSetProf: ResourceProfile): Option[Map[String, Map[String, Long]]] = { + def assignAddressesCustomResources(taskSetProf: ResourceProfile): + Option[Map[String, Map[String, Long]]] = { // only look at the resource other than cpus val tsResources = taskSetProf.getCustomTaskResources() if (tsResources.isEmpty) { @@ -137,7 +150,7 @@ private[spark] class ExecutorResourcesAmounts( val allocatedAddresses = HashMap[String, Map[String, Long]]() - // we go through all resources here so that we can make sure they match and also get what the + // Go through all resources here so that we can make sure they match and also get what the // assignments are for the next task for ((rName, taskReqs) <- tsResources) { // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number @@ -147,7 +160,7 @@ private[spark] class ExecutorResourcesAmounts( case Some(addressesAmountMap) => val allocatedAddressesMap = HashMap[String, Long]() - // always sort the addresses + // Always sort the addresses val addresses = addressesAmountMap.keys.toSeq.sorted // task.amount is a whole number @@ -173,7 +186,9 @@ private[spark] class ExecutorResourcesAmounts( if (taskAmount == 0 && allocatedAddressesMap.size > 0) { allocatedAddresses.put(rName, allocatedAddressesMap.toMap) - } else return None + } else { + return None + } case None => return None } 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 04661273ac0f..21f62097a4bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -469,7 +469,7 @@ private[spark] class TaskSchedulerImpl( // check if the ResourceProfile has cpus first since that is common case if (availCpus < taskCpus) return None // only look at the resource other than cpus - availWorkerResources.assignResources(taskSetProf) + availWorkerResources.assignAddressesCustomResources(taskSetProf) } private def minTaskLocality( 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 2f35ada16ac0..72ac25b389d0 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 @@ -173,10 +173,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case Some(executorInfo) => executorInfo.freeCores += taskCpus resources.foreach { case (rName, addressAmount) => - addressAmount.foreach { case (address, amount) => - executorInfo.resourcesInfo.get(rName).foreach { r => - r.release(Map(address -> amount)) - } + executorInfo.resourcesInfo.get(rName).foreach { r => + r.release(addressAmount) } } makeOffers(executorId) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala index eb6129f84bb6..316b3ef81b20 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala @@ -56,7 +56,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val rp = new ResourceProfileBuilder().require(treqs).build() // assign nothing to rp without resource profile - val assigned = availableExecResAmounts.assignResources(rp) + val assigned = availableExecResAmounts.assignAddressesCustomResources(rp) assert(assigned.isDefined) assigned.foreach { case resource => assert(resource.isEmpty) } } @@ -142,7 +142,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val rp = new ResourceProfileBuilder().require(treqs).build() // taskMount = 0.1 < 1.0 which can be assigned. - val assigned = availableExecResAmounts.assignResources(rp) + val assigned = availableExecResAmounts.assignAddressesCustomResources(rp) // update the value availableExecResAmounts.acquire(assigned.get) @@ -175,7 +175,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { .resource("gpu", gpuTaskAmount) val rp = new ResourceProfileBuilder().require(treqs).build() - val assigned = availableExecResAmounts.assignResources(rp) + val assigned = availableExecResAmounts.assignAddressesCustomResources(rp) assert(assigned.isEmpty) } @@ -191,7 +191,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { .resource("fpga", fpgaTaskAmount) val rp = new ResourceProfileBuilder().require(treqs).build() - var assigned = availableExecResAmounts.assignResources(rp) + var assigned = availableExecResAmounts.assignAddressesCustomResources(rp) assert(!assigned.isEmpty) assigned.foreach { case resource => assert(!resource.isEmpty)} @@ -200,7 +200,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { .resource("fpga", 0.9) // couldn't allocate fpga val rp1 = new ResourceProfileBuilder().require(treqs1).build() - assigned = availableExecResAmounts.assignResources(rp1) + assigned = availableExecResAmounts.assignAddressesCustomResources(rp1) assert(assigned.isEmpty) } @@ -274,7 +274,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val rp = new ResourceProfileBuilder().require(treqs).build() // taskMount = 0.1 < 1.0 which can be assigned. - val assigned = availableExecResAmounts.assignResources(rp) + val assigned = availableExecResAmounts.assignAddressesCustomResources(rp) assert(!assigned.isEmpty) assigned.foreach { case resource => assert(resource.size === 1) @@ -341,7 +341,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { val rp = new ResourceProfileBuilder().require(treqs).build() // taskMount = 0.1 < 1.0 which can be assigned. - val assigned = availableExecResAmounts.assignResources(rp) + val assigned = availableExecResAmounts.assignAddressesCustomResources(rp) assert(!assigned.isEmpty) assigned.foreach { case resourceAmounts => assert(resourceAmounts.size === 2) @@ -406,7 +406,7 @@ class ExecutorResourcesAmountsSuite extends SparkFunSuite { ): Unit = { val treqs = new TaskResourceRequests().resource("gpu", taskAmount) val rp = new ResourceProfileBuilder().require(treqs).build() - val assigned = availableExecResAmounts.assignResources(rp) + val assigned = availableExecResAmounts.assignAddressesCustomResources(rp) assert(!assigned.isEmpty) assigned.foreach { case resources => assert( From 196e11bd861d4915e5d51bfab87a134d5ee5ea1a Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Mon, 4 Dec 2023 15:12:30 +0800 Subject: [PATCH 17/18] comment --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 4 +--- 1 file changed, 1 insertion(+), 3 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 72ac25b389d0..b32544ff5858 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 @@ -440,9 +440,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // finishes. executorData.freeCores -= task.cpus task.resources.foreach { case (rName, addressAmounts) => - addressAmounts.foreach { case (address, amount) => - executorData.resourcesInfo(rName).acquire(Map(address -> amount)) - } + executorData.resourcesInfo(rName).acquire(addressAmounts) } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + s"${executorData.executorHost}.") From 347196f90149cc49fcafb4271971e8d71b95ee98 Mon Sep 17 00:00:00 2001 From: Bobby Wang Date: Wed, 20 Dec 2023 10:18:14 +0800 Subject: [PATCH 18/18] resolve comments --- .../executor/CoarseGrainedExecutorBackend.scala | 3 --- .../apache/spark/resource/ResourceAllocator.scala | 12 ------------ .../scheduler/ExecutorResourcesAmountsSuite.scala | 11 +++++++---- .../spark/scheduler/TaskSchedulerImplSuite.scala | 2 +- 4 files changed, 8 insertions(+), 20 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 194eb744d7ef..ed0735502707 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -181,9 +181,6 @@ private[spark] class CoarseGrainedExecutorBackend( } else { val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) - // Convert resources amounts into ResourceInformation - val resources = taskDesc.resources.map { case (rName, addressesAmounts) => - rName -> new ResourceInformation(rName, addressesAmounts.keys.toSeq.sorted.toArray)} executor.launchTask(this, taskDesc) } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala index 928e9e526738..e9bb11721725 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala @@ -60,20 +60,8 @@ private[spark] object ResourceAmountUtils { def toInternalResource(amount: Double): Long = (amount * ONE_ENTIRE_RESOURCE).toLong - private[spark] def toInternalResource(resources: Map[String, Double]): Map[String, Long] = { - resources.map { case (k, v) => k -> toInternalResource(v) } - } - def toFractionalResource(amount: Long): Double = amount.toDouble / ONE_ENTIRE_RESOURCE - private[spark] def toFractionalResource(resources: Map[String, Long]): Map[String, Double] = { - resources.map { case (k, v) => k -> toFractionalResource(v) } - } - - private[spark] def toInternalResourceMapMap(resources: Map[String, Map[String, Double]]): - Map[String, Map[String, Long]] = { - resources.map { case (k, v) => k -> toInternalResource(v) } - } } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala index 316b3ef81b20..e512327fefa7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourcesAmountsSuite.scala @@ -27,14 +27,17 @@ import org.apache.spark.resource.ResourceUtils.GPU class ExecutorResourcesAmountsSuite extends SparkFunSuite { implicit def toFractionalResource(resources: Map[String, Long]): Map[String, Double] = - ResourceAmountUtils.toFractionalResource(resources) + resources.map { case (k, v) => k -> ResourceAmountUtils.toFractionalResource(v) } implicit def toInternalResource(resources: Map[String, Double]): Map[String, Long] = - ResourceAmountUtils.toInternalResource(resources) + resources.map { case (k, v) => k -> ResourceAmountUtils.toInternalResource(v) } implicit def toInternalResourceMap(resources: Map[String, Map[String, Double]]): - Map[String, Map[String, Long]] = ResourceAmountUtils.toInternalResourceMapMap(resources) - + Map[String, Map[String, Long]] = + resources.map { case (resName, addressesAmountMap) => + resName -> addressesAmountMap.map { case (k, v) => + k -> ResourceAmountUtils.toInternalResource(v) } + } def compareMaps(lhs: Map[String, Double], rhs: Map[String, Double], eps: Double = 0.00000001): Boolean = { 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 34d47e2edc32..72d0354c5577 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -2296,7 +2296,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext } private implicit def toInternalResource(resources: Map[String, Double]): Map[String, Long] = - ResourceAmountUtils.toInternalResource(resources) + resources.map { case (k, v) => k -> ResourceAmountUtils.toInternalResource(v) } // 1 executor with 4 GPUS Seq(true, false).foreach { barrierMode =>