tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1377768603


##########
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])

Review Comment:
   I think resources here is essentially unused now, which means taskResources 
is likely not used. 



##########
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) =>

Review Comment:
   resources here is no longer used and seems like a lot of duplicate 
information now.  We should figure out a better way to do this.  This also 
likely means the way its stored on Executor side needs to change.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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 {

Review Comment:
   is there a reason we are taking these as Double vs just using the Long 
representation (double * RESOURCE_TOTAL_AMOUNT) .   Seems like that would just 
be more efficient to not convert back and forth.



##########
core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala:
##########
@@ -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,

Review Comment:
   If we don't have a good way to differentiate the two modes, I'm ok with 
leaving this check as you changed it.  Maybe we can check things in the 
ResourceProfile where we know wheterh its a TaskResourceProfile or not and 
accomplish the same thing.



##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -58,6 +58,9 @@ 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]],

Review Comment:
   this also still seems like we are keeping duplicate information, we have the 
resources and then the resource amounts that have the same info.  We may need 
like a ResourceInformationWithAmount and just combine these.  The resources on 
the executor side do get into the TaskContext so we need to keep that 
information.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * 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 {
+
+  /**
+   * 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: _*)
+    }
+  }
+
+  /**
+   * 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.
+   */
+  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
+      }.toMap
+    }
+  }
+
+  /**
+   * 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,
+        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
+      }
+    }
+  }
+
+  /**
+   * 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,
+        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
+      }
+    }
+  }
+
+  /**
+   * 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]])] = {
+
+    // 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) {
+      // 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) =>
+
+          var internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong

Review Comment:
   this might have issues with overflow if taskAmount if large.  You might need 
to handle > 1 differently then < 1.0



##########
core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala:
##########
@@ -28,12 +28,21 @@ 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)
+

Review Comment:
   nit remove extra newline



##########
core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala:
##########
@@ -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 = {

Review Comment:
   same thing in these classes as mentioned earlier, seems like we are 
duplicating a lot of information between this and taskResourceAssignments, I 
would like to see a new class that tracks both I think.   If there is some 
reason you don't think that will work let me know.



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,54 @@ 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): _*)
+    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.
+   * Get the resources and its amounts.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map {

Review Comment:
   leave these in the Long form, I think only place this is used is in 
ExecutorResourcesAmount which could store the same way.  I think this is a 
global comment, if we can store it in Long format and pass that everywhere and 
skip converting I'd rather do that.  Only convert back to double to display to 
user and possibly logs. 
   
   I guess we do need to be careful to make sure that these are 1.0 or less 
though, if we start getting into the requests where user could ask for 250000 
resources then we could hit overflow issues, so if we are passing those 
requests around might need to keep them in double format.  Hopefully those are 
limited to the requests in the resource profiles though and we pass around the 
GPU index -> amount which should be 1.0 or less.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to