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


##########
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:
   Hmm, this is to keep the same design as the original 
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala#L53
 



##########
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:
   Done



##########
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:
   Yeah, you're correct. Removed the "update"



##########
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:
   Done. Thx



##########
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:
   Wow, good catch. Done. Thx



##########
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:
   Done



##########
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:
   Updated the comment to the function description



##########
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:
   Thx, Done.



##########
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:
   yeah. you're correct. there will be "wasting" in some cases. Anyways, I've 
updated the comment to the function description and added a TODO for it.



##########
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:
   Done.



-- 
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