You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "tgravescs (via GitHub)" <gi...@apache.org> on 2023/12/01 22:03:32 UTC

Re: [PR] [SPARK-45527][CORE] Use fraction to do the resource calculation [spark]

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


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +84,53 @@ 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 1.0 (we multiply ONE_ENTIRE_RESOURCE
+   * to avoid precision error), a value &gt; 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 -> ONE_ENTIRE_RESOURCE): _*)
   }
 
   /**
-   * 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 amounts of resources that have been multiplied by ONE_ENTIRE_RESOURCE.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Long] = addressAvailabilityMap.toMap
+
+  /**
+   * Sequence of currently available resource addresses which are 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 < ONE_ENTIRE_RESOURCE).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, 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."))
-      if (isAvailable > 0) {
-        addressAvailabilityMap(address) -= 1
+          s"address $address doesn't exist."))
+
+      val left = addressAvailabilityMap(address) - amount

Review Comment:
   here we should use prevAmount since we already fetched it



##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -446,11 +441,11 @@ 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.toImmutableArraySeq)
+          task.resources.foreach { case (rName, addressAmounts) =>
+            addressAmounts.foreach { case (address, amount) =>

Review Comment:
   same thing here why are we iteration addressAmounts vs just passing to acquire?



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.{ResourceAmountUtils, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+/**
+ * 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" -> 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(
+    private val resources: Map[String, Map[String, Long]]) extends Serializable {
+
+  /**
+   * convert the resources to be mutable HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * 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 resourceAddressAmount: 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 -> ResourceAmountUtils.toFractionalResource(amount)
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  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"))
+      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 left = prevInternalTotalAmount - amount
+        if (left < 0) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(left)} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = left
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  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"))
+      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 total = prevInternalTotalAmount + amount
+        if (total > ONE_ENTIRE_RESOURCE) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(total)} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = total
+      }
+    }
+  }
+
+  /**
+   * 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 optional resources amounts

Review Comment:
   please add a better description.  The critical part here is that this returns None if any of the task requests for resources aren't met.
   
   Also perhaps we should rename this since its not really updating, assignAddressesCustomResources



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.{ResourceAmountUtils, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+/**
+ * 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" -> 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(
+    private val resources: Map[String, Map[String, Long]]) extends Serializable {
+
+  /**
+   * convert the resources to be mutable HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * 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 resourceAddressAmount: 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 -> ResourceAmountUtils.toFractionalResource(amount)
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  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"))
+      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 left = prevInternalTotalAmount - amount
+        if (left < 0) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(left)} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = left
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  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"))
+      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 total = prevInternalTotalAmount + amount
+        if (total > ONE_ENTIRE_RESOURCE) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(total)} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = total
+      }
+    }
+  }
+
+  /**
+   * 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 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) {
+      return Some(Map.empty)
+    }
+
+    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
+    for ((rName, taskReqs) <- tsResources) {
+      // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number
+      var taskAmount = taskReqs.amount
+
+      internalResources.get(rName) match {
+        case Some(addressesAmountMap) =>
+          val allocatedAddressesMap = HashMap[String, Long]()
+
+          // always sort the addresses

Review Comment:
   why?  please update the comment



##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -172,9 +172,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
           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.toImmutableArraySeq)
+              resources.foreach { case (rName, addressAmount) =>
+                addressAmount.foreach { case (address, amount) =>

Review Comment:
   this reads  weird,  why are we getting address and amount and looping over them when its a map and release takes a map?
   
   shouldn't this just be:
   
   ```
         resources.foreach { case (rName, addressAmount) =>
           executorInfo.resourcesInfo.get(rName).foreach { r =>
             r.release(addressAmount)
           }
   ```



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -91,16 +140,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, Long]): Unit = {

Review Comment:
   nit remote space between release and (



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.{ResourceAmountUtils, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+/**
+ * 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" -> 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(
+    private val resources: Map[String, Map[String, Long]]) extends Serializable {
+
+  /**
+   * convert the resources to be mutable HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * 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 resourceAddressAmount: 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 -> ResourceAmountUtils.toFractionalResource(amount)
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource

Review Comment:
   nit, this update description, what is acquire and update mean... it should really just be acquire, if there is some other side affect please be more specific.  I think at this point they are assigned and assumption is we just mark as acquired, else throw.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.{ResourceAmountUtils, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+/**
+ * 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" -> 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(
+    private val resources: Map[String, Map[String, Long]]) extends Serializable {
+
+  /**
+   * convert the resources to be mutable HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * 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 resourceAddressAmount: 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 -> ResourceAmountUtils.toFractionalResource(amount)
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  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"))
+      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 left = prevInternalTotalAmount - amount
+        if (left < 0) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(left)} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = left
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  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"))
+      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 total = prevInternalTotalAmount + amount
+        if (total > ONE_ENTIRE_RESOURCE) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(total)} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = total
+      }
+    }
+  }
+
+  /**
+   * 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 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) {
+      return Some(Map.empty)
+    }
+
+    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
+    for ((rName, taskReqs) <- tsResources) {
+      // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number
+      var taskAmount = taskReqs.amount
+
+      internalResources.get(rName) match {
+        case Some(addressesAmountMap) =>
+          val allocatedAddressesMap = HashMap[String, Long]()
+
+          // always sort the addresses
+          val addresses = addressesAmountMap.keys.toSeq.sorted
+
+          // task.amount is a whole number
+          if (taskAmount >= 1.0) {
+            for (address <- addresses if taskAmount > 0) {
+              // The address is still a whole resource
+              if (ResourceAmountUtils.isOneEntireResource(addressesAmountMap(address))) {
+                taskAmount -= 1.0
+                // Assign the full resource of the address
+                allocatedAddressesMap(address) = ONE_ENTIRE_RESOURCE
+              }
+            }
+          } else if (taskAmount > 0.0) { // 0 < task.amount < 1.0
+            val internalTaskAmount = ResourceAmountUtils.toInternalResource(taskAmount)
+            for (address <- addresses if taskAmount > 0) {
+              if (addressesAmountMap(address) >= internalTaskAmount) {

Review Comment:
   so one interesting side affect here is that you will be doing packing because you sort above.  I think that is fine and generally that is what it did before, but  would be good to document in the function description.  It is also slightly different because you can have different values and "waste" part of the resource now.  



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.{ResourceAmountUtils, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+/**
+ * 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" -> 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(
+    private val resources: Map[String, Map[String, Long]]) extends Serializable {
+
+  /**
+   * convert the resources to be mutable HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * 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 resourceAddressAmount: 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 -> ResourceAmountUtils.toFractionalResource(amount)
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  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"))
+      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 left = prevInternalTotalAmount - amount
+        if (left < 0) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(left)} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = left
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  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"))
+      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 total = prevInternalTotalAmount + amount
+        if (total > ONE_ENTIRE_RESOURCE) {
+          throw new SparkException(s"The total amount " +
+            s"${ResourceAmountUtils.toFractionalResource(total)} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = total
+      }
+    }
+  }
+
+  /**
+   * 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 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) {
+      return Some(Map.empty)
+    }
+
+    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
+    for ((rName, taskReqs) <- tsResources) {
+      // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number
+      var taskAmount = taskReqs.amount
+
+      internalResources.get(rName) match {
+        case Some(addressesAmountMap) =>
+          val allocatedAddressesMap = HashMap[String, Long]()
+
+          // always sort the addresses
+          val addresses = addressesAmountMap.keys.toSeq.sorted
+
+          // task.amount is a whole number
+          if (taskAmount >= 1.0) {
+            for (address <- addresses if taskAmount > 0) {
+              // The address is still a whole resource
+              if (ResourceAmountUtils.isOneEntireResource(addressesAmountMap(address))) {
+                taskAmount -= 1.0
+                // Assign the full resource of the address
+                allocatedAddressesMap(address) = ONE_ENTIRE_RESOURCE
+              }
+            }
+          } else if (taskAmount > 0.0) { // 0 < task.amount < 1.0
+            val internalTaskAmount = ResourceAmountUtils.toInternalResource(taskAmount)
+            for (address <- addresses if taskAmount > 0) {
+              if (addressesAmountMap(address) >= internalTaskAmount) {
+                // Assign the part of the address.
+                allocatedAddressesMap(address) = internalTaskAmount
+                taskAmount = 0
+              }
+            }
+          }
+
+          if (taskAmount == 0 && allocatedAddressesMap.size > 0) {
+            allocatedAddresses.put(rName, allocatedAddressesMap.toMap)
+          } else return None

Review Comment:
   nit put in {}
   
   } else {
     return None
   }



-- 
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-unsubscribe@spark.apache.org

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


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