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


##########
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.RESOURCE_TOTAL_AMOUNT
+import org.apache.spark.resource.ResourceProfile
+
+/**
+ * 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" -> 0.2*RESOURCE_TOTAL_AMOUNT,
+ *                                   "1" -> 1.0*RESOURCE_TOTAL_AMOUNT),
+ *                  "fpga" -> Map("a" -> 0.3*RESOURCE_TOTAL_AMOUNT,
+ *                                "b" -> 0.9*RESOURCE_TOTAL_AMOUNT)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, 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" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                  "1" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                  "2" -> 0.5 * RESOURCE_TOTAL_AMOUNT),
+   *     "fpga" -> Map("a" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                   "b" -> 0.5 * RESOURCE_TOTAL_AMOUNT))
+   * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2)
+   */
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case 
(rName, addressMap) =>
+    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, 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 ${left.toDouble / 
RESOURCE_TOTAL_AMOUNT} " +
+            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 > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${total.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            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 (addressesAmountMap(address) == RESOURCE_TOTAL_AMOUNT) {
+                taskAmount -= 1.0
+                // Assign the full resource of the address
+                allocatedAddressesMap(address) = RESOURCE_TOTAL_AMOUNT
+              }
+            }
+          } else if (taskAmount > 0.0) { // 0 < task.amount < 1.0
+            val internalTaskAmount = (taskAmount * 
RESOURCE_TOTAL_AMOUNT).toLong

Review Comment:
   can you make a utility function that converts to the internal task amount.. 
ie does the * RESOURCE_TOTAL_AMOUNT



##########
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.RESOURCE_TOTAL_AMOUNT
+import org.apache.spark.resource.ResourceProfile
+
+/**
+ * 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" -> 0.2*RESOURCE_TOTAL_AMOUNT,

Review Comment:
   nit put spaces around the *



##########
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.RESOURCE_TOTAL_AMOUNT
+import org.apache.spark.resource.ResourceProfile
+
+/**
+ * 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" -> 0.2*RESOURCE_TOTAL_AMOUNT,
+ *                                   "1" -> 1.0*RESOURCE_TOTAL_AMOUNT),
+ *                  "fpga" -> Map("a" -> 0.3*RESOURCE_TOTAL_AMOUNT,
+ *                                "b" -> 0.9*RESOURCE_TOTAL_AMOUNT)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, 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" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                  "1" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                  "2" -> 0.5 * RESOURCE_TOTAL_AMOUNT),
+   *     "fpga" -> Map("a" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                   "b" -> 0.5 * RESOURCE_TOTAL_AMOUNT))
+   * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2)
+   */
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case 
(rName, addressMap) =>
+    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, 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 ${left.toDouble / 
RESOURCE_TOTAL_AMOUNT} " +
+            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 > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${total.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            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]]] = {
+

Review Comment:
   nit: remove extra newline



##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -191,7 +191,10 @@ private[spark] class CoarseGrainedExecutorBackend(
       } else {
         val taskDesc = TaskDescription.decode(data.value)
         logInfo("Got assigned task " + taskDesc.taskId)
-        taskResources.put(taskDesc.taskId, taskDesc.resources)
+        // Convert resources amounts into ResourceInformation
+        val resources = taskDesc.resources.map { case (rName, 
addressesAmounts) =>
+          rName -> new ResourceInformation(rName, 
addressesAmounts.keys.toSeq.sorted.toArray)}
+        taskResources.put(taskDesc.taskId, resources)

Review Comment:
   so taskResources originally was required and provided useful functionality 
because the extra resources from taskDesc.resources wasn't exposed as public 
here.  Since a previous change, taskResources is only used by the tests. But 
the tests were only validating it with taskResources because that is how it 
functionally works.  The fact it isn't actually using taskResources means we 
are testing something that could functionally be wrong. 
   
   We should update the tests to stop using taskResources (and remove 
taskResources) and instead use         
backend.executor.runningTasks.get(taskId).taskDescription.resources, which is 
what we are really using to track the extra resources.
   



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -20,6 +20,42 @@ package org.apache.spark.resource
 import scala.collection.mutable
 
 import org.apache.spark.SparkException
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+private[spark] object ResourceAmountUtils {
+  /**
+   * Using "double" to do the resource calculation may encounter a problem of 
precision loss. Eg
+   *
+   * scala&gt; val taskAmount = 1.0 / 9
+   * taskAmount: Double = 0.1111111111111111
+   *
+   * scala&gt; var total = 1.0
+   * total: Double = 1.0
+   *
+   * scala&gt; for (i &lt;- 1 to 9 ) {
+   * |   if (total &gt;= taskAmount) {
+   * |           total -= taskAmount
+   * |           println(s"assign $taskAmount for task $i, total left: $total")
+   * |   } else {
+   * |           println(s"ERROR Can't assign $taskAmount for task $i, total 
left: $total")
+   * |   }
+   * | }
+   * assign 0.1111111111111111 for task 1, total left: 0.8888888888888888
+   * assign 0.1111111111111111 for task 2, total left: 0.7777777777777777
+   * assign 0.1111111111111111 for task 3, total left: 0.6666666666666665
+   * assign 0.1111111111111111 for task 4, total left: 0.5555555555555554
+   * assign 0.1111111111111111 for task 5, total left: 0.44444444444444425
+   * assign 0.1111111111111111 for task 6, total left: 0.33333333333333315
+   * assign 0.1111111111111111 for task 7, total left: 0.22222222222222204
+   * assign 0.1111111111111111 for task 8, total left: 0.11111111111111094
+   * ERROR Can't assign 0.1111111111111111 for task 9, total left: 
0.11111111111111094
+   *
+   * So we multiply RESOURCE_TOTAL_AMOUNT to convert the double to long to 
avoid this limitation.
+   * Double can display up to 16 decimal places, so we set the factor to
+   * 10, 000, 000, 000, 000, 000L.
+   */
+  final val RESOURCE_TOTAL_AMOUNT: Long = 10000000000000000L

Review Comment:
   I think we should rename this  ONE_ENTIRE_RESOURCE  or something that 
indicates this is the entire amount of a single resource..



##########
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.RESOURCE_TOTAL_AMOUNT
+import org.apache.spark.resource.ResourceProfile
+
+/**
+ * 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" -> 0.2*RESOURCE_TOTAL_AMOUNT,
+ *                                   "1" -> 1.0*RESOURCE_TOTAL_AMOUNT),
+ *                  "fpga" -> Map("a" -> 0.3*RESOURCE_TOTAL_AMOUNT,
+ *                                "b" -> 0.9*RESOURCE_TOTAL_AMOUNT)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, 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" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                  "1" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                  "2" -> 0.5 * RESOURCE_TOTAL_AMOUNT),
+   *     "fpga" -> Map("a" -> 0.5 * RESOURCE_TOTAL_AMOUNT,
+   *                   "b" -> 0.5 * RESOURCE_TOTAL_AMOUNT))
+   * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2)
+   */
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case 
(rName, addressMap) =>

Review Comment:
   rename to be resourceAddressAmount



##########
core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala:
##########
@@ -170,16 +170,16 @@ private[spark] object ResourceUtils extends Logging {
   // integer amount and the number of slots per address. For instance, if the 
amount is 0.5,
   // the we get (1, 2) back out. This indicates that for each 1 address, it 
has 2 slots per
   // address, which allows you to put 2 tasks on that address. Note if amount 
is greater
-  // than 1, then the number of slots per address has to be 1. This would 
indicate that a
+  // than 1, then the number of parts per address has to be 1. This would 
indicate that a
   // would have multiple addresses assigned per task. This can be used for 
calculating
   // the number of tasks per executor -> (executorAmount * numParts) / 
(integer amount).
   // Returns tuple of (integer amount, numParts)
   def calculateAmountAndPartsForFraction(doubleAmount: Double): (Int, Int) = {
-    val parts = if (doubleAmount <= 0.5) {
+    val parts = if (doubleAmount <= 1.0) {

Review Comment:
   did you move this check somewhere else?



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