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

[PR] [SPARK-45527][core] Use fraction to do the resource calculation [spark]

wbo4958 opened a new pull request, #43494:
URL: https://github.com/apache/spark/pull/43494

   ### What changes were proposed in this pull request?
   
   This (PR) introduces the utilization of fractions instead of slots, which is similar to the CPU strategy, 
   for determining whether a worker offer can provide the necessary resources to tasks.
   
   For instance, when an executor reports to the driver with [gpu, ["1,", "2"]], the driver constructs an executor data map. 
   The keys in the map represent the GPU addresses, and their default values are set to 1.0, indicating one whole GPU.
   
   Consequently, the available resource amounts for the executor are as follows: { "1" -> 1.0f, "2" -> 1.0f }.
   
   When offering resources to a task that requires 1 CPU and 0.08 GPU, the worker offer examines the available resource amounts. 
   It identifies that the capacity of GPU address "1.0" is greater than the task's GPU requirement (1.0 >= 0.08). 
   Therefore, Spark assigns the GPU address "1" to this task. After the assignment, the available resource amounts 
   for this executor are updated to { "1" -> 0.92, "2" -> 1.0}, ensuring that the remaining resources can be allocated to other tasks.
   
   In scenarios where other tasks, using different task resource profiles, request varying GPU amounts 
   when dynamic allocation is disabled, Spark applies the same comparison approach. It compares the task's GPU requirement with 
   the available resource amounts to determine if the resources can be assigned to the task.
   
   ### Why are the changes needed?
   
   The existing resources offering including gpu, fpga is based on "slots per address", which is defined by the default resource profile.
   and it's a fixed number for all different resource profiles when dynamic allcation is disabled.
   
   Consider the below test case,
   
   ``` scala
     withTempDir { dir =>
       val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript",
         """{"name": "gpu","addresses":["0"]}""")
   
       val conf = new SparkConf()
         .setAppName("test")
         .setMaster("local-cluster[1, 12, 1024]")
         .set("spark.executor.cores", "12")
         
       conf.set("spark.worker.resource.gpu.amount", "1")
       conf.set("spark.worker.resource.gpu.discoveryScript", scriptPath)
       conf.set("spark.executor.resource.gpu.amount", "1")
       conf.set("spark.task.resource.gpu.amount", "0.08")
       
       sc = new SparkContext(conf)
       val rdd = sc.range(0, 100, 1, 4)
       var rdd1 = rdd.repartition(3)
       val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 1.0)
       val rp = new ResourceProfileBuilder().require(treqs).build
       rdd1 = rdd1.withResources(rp)
       assert(rdd1.collect().size === 100)
     }
   ```
   
   During the initial stages, Spark generates a default resource profile based on the configurations. The calculation 
   for determining the slots per GPU address is performed as "spark.executor.resource.gpu.amount / spark.task.resource.gpu.amount", 
   resulting in a value of 12 (1/0.08 = 12). This means that Spark can accommodate up to 12 tasks running on each GPU address simultaneously.
   
   The job is then divided into two stages. The first stage, which consists of 4 tasks, runs concurrently based on 
   the default resource profile. However, the second stage, comprising 3 tasks, runs sequentially using a new task 
   resource profile. This new profile specifies that each task requires 1 CPU and 1.0 full GPU.
   
   In reality, the tasks in the second stage are running in parallel, which is the underlying issue.
   
   The problem lies in the line `new TaskResourceRequests().cpus(1).resource("gpu", 1.0)`. The value of 1.0 
   for the GPU, or any value below 1.0 (specifically, (0, 0.5] which is rounded up to 1.0, spark throws an exception if the value is in (0.5, 1)), 
   is merely requesting the number of slots. In this case, it is requesting only 1 slot. Consequently, each task 
   necessitates 1 CPU core and 1 GPU slot, resulting in all tasks running simultaneously.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No
   
   
   ### How was this patch tested?
   
   To ensure all tests got passed
   
   ### Was this patch authored or co-authored using generative AI tooling?
   
   No


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1390573109


##########
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:
   The taskResources is exposing for testing purpose, please refer to https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala#L71-L79
   
   It's used in the CoarseGrainedExecutorBackendSuite, like https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala#L342-L350
   



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1384004018


##########
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:
   I don't think taskResources is needed at all anymore.  Lets remove it unless you see it being used for something I'm missing.  It was used in the statusUpdate call below that you removed.  I actually think it wasn't needed even before (changed in Spark 3.4) that since the taskDescription and runningTasks has the same information and is now accessible.



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1877272394

   Merged to master branch.


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378495425


##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {

Review Comment:
   Good suggestion, Let me just use the Long represnetation.



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1877250572

   thanks for the detailed test results and all the work! this looks good.  


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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1399408216


##########
core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala:
##########
@@ -171,16 +171,16 @@ private[spark] object ResourceUtils extends Logging {
   // integer amount and the number of slots per address. For instance, if the amount is 0.5,

Review Comment:
   there are comments in function getSchedulerTaskResourceAmount that need to change based  on these new changes. Its also unclear to me from your response whether that function and warnOnWastedResources are still used



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1776401120

   Hi @tgravescs @Ngone51 @WeichenXu123 , Could you help to review this PR, thx very much.


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1432324686


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -20,6 +20,61 @@ package org.apache.spark.resource
 import scala.collection.mutable
 
 import org.apache.spark.SparkException
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+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 ONE_ENTIRE_RESOURCE to convert the double to long to avoid this limitation.
+   * Double can display up to 16 decimal places, so we set the factor to
+   * 10, 000, 000, 000, 000, 000L.
+   */
+  final val ONE_ENTIRE_RESOURCE: Long = 10000000000000000L
+
+  def isOneEntireResource(amount: Long): Boolean = amount == ONE_ENTIRE_RESOURCE
+
+  def toInternalResource(amount: Double): Long = (amount * ONE_ENTIRE_RESOURCE).toLong
+
+  private[spark] def toInternalResource(resources: Map[String, Double]): Map[String, Long] = {
+    resources.map { case (k, v) => k -> toInternalResource(v) }
+  }
+
+  def toFractionalResource(amount: Long): Double = amount.toDouble / ONE_ENTIRE_RESOURCE
+
+  private[spark] def toFractionalResource(resources: Map[String, Long]): Map[String, Double] = {
+    resources.map { case (k, v) => k -> toFractionalResource(v) }
+  }
+
+  private[spark] def toInternalResourceMapMap(resources: Map[String, Map[String, Double]]):

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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1861662755

   sorry for the delay on this, overall looks good, a few minor comments.  Can you confirm you tested with the dynamic allocation path and everything is working as expected there? 


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1503476690


##########
core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala:
##########
@@ -2283,4 +2295,425 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext
     taskScheduler.handleFailedTask(tsm, tid, state, reason)
   }
 
+  private implicit def toInternalResource(resources: Map[String, Double]): Map[String, Long] =
+    resources.map { case (k, v) => k -> ResourceAmountUtils.toInternalResource(v) }
+
+  // 1 executor with 4 GPUS
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run in the same executor") {
+        val taskCpus = 1
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+        val executorGpus = 1
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> taskCpus.toString,
+          TASK_GPU_ID.amountConf -> gpuTaskAmount.toString,
+          EXECUTOR_GPU_ID.amountConf -> executorGpus.toString,
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum)
+        }
+
+        val resources = new ExecutorResourcesAmounts(
+          Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))
+
+        val workerOffers =
+          IndexedSeq(WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources))
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements.
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        var gpuAddress = -1
+        for (taskId <- 0 until 4 * taskNum) {
+          if (taskId % taskNum == 0) {
+            gpuAddress += 1
+          }
+          assert(ArrayBuffer(gpuAddress.toString) ===
+            taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted)
+        }
+      }
+    }
+  }
+
+  // 4 executors, each of which has 1 GPU
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run on the different executor") {
+        val taskCpus = 1
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+        val executorGpus = 1
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> taskCpus.toString,
+          TASK_GPU_ID.amountConf -> gpuTaskAmount.toString,
+          EXECUTOR_GPU_ID.amountConf -> executorGpus.toString,
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum)
+        }
+
+        val workerOffers =
+          IndexedSeq(
+            WorkerOffer("executor0", "host0", executorCpus, Some("host0"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("0" -> 1.0))))),
+            WorkerOffer("executor1", "host1", executorCpus, Some("host1"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("1" -> 1.0))))),
+            WorkerOffer("executor2", "host2", executorCpus, Some("host2"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("2" -> 1.0))))),
+            WorkerOffer("executor3", "host3", executorCpus, Some("host3"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("3" -> 1.0))))))
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements
+
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        val assignedGpus: HashMap[String, Int] = HashMap.empty
+        for (taskId <- 0 until 4 * taskNum) {
+          val gpus = taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted
+          assert(gpus.length == 1)
+          val addr = gpus(0)
+          if (!assignedGpus.contains(addr)) {
+            assignedGpus(addr) = 1
+          } else {
+            assignedGpus(addr) += 1
+          }
+        }
+        assert(assignedGpus.toMap ===
+          Map("0" -> taskNum, "1" -> taskNum, "2" -> taskNum, "3" -> taskNum))
+      }
+    }
+  }
+
+  // 1 executor with 4 GPUS
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run in the same executor") {
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> "1",
+          TASK_GPU_ID.amountConf -> "0.1",
+          EXECUTOR_GPU_ID.amountConf -> "4",
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val treqs = new TaskResourceRequests().cpus(1).resource(GPU, gpuTaskAmount)
+        val rp = new TaskResourceProfile(treqs.requests)
+        taskScheduler.sc.resourceProfileManager.addResourceProfile(rp)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100, 0, 1, 1, rp.id)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id)
+        }
+        val resources = new ExecutorResourcesAmounts(
+          Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))
+
+        val workerOffers = IndexedSeq(
+          WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources, rp.id)
+        )
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements.
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        var gpuAddress = -1
+        for (taskId <- 0 until 4 * taskNum) {
+          if (taskId % taskNum == 0) {
+            gpuAddress += 1
+          }
+          assert(ArrayBuffer(gpuAddress.toString) ===
+            taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted)
+        }
+      }
+    }
+  }
+
+  // 4 executors, each of which has 1 GPU
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run on the different executor") {
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> "1",
+          TASK_GPU_ID.amountConf -> "0.1",
+          EXECUTOR_GPU_ID.amountConf -> "1",
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val treqs = new TaskResourceRequests().cpus(1).resource(GPU, gpuTaskAmount)
+        val rp = new TaskResourceProfile(treqs.requests)
+        taskScheduler.sc.resourceProfileManager.addResourceProfile(rp)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100, 0, 1, 1, rp.id)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id)
+        }
+
+        val workerOffers =
+          IndexedSeq(
+            WorkerOffer("executor0", "host0", executorCpus, Some("host1"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("0" -> 1.0)))),
+              rp.id),
+            WorkerOffer("executor1", "host1", executorCpus, Some("host2"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("1" -> 1.0)))),
+              rp.id),
+            WorkerOffer("executor2", "host2", executorCpus, Some("host3"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("2" -> 1.0)))),
+              rp.id),
+            WorkerOffer("executor3", "host3", executorCpus, Some("host4"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("3" -> 1.0)))),
+              rp.id)
+          )
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements
+
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        val assignedGpus: HashMap[String, Int] = HashMap.empty
+        for (taskId <- 0 until 4 * taskNum) {
+          val gpus = taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted
+          assert(gpus.length == 1)
+          val addr = gpus(0)
+          if (!assignedGpus.contains(addr)) {
+            assignedGpus(addr) = 1
+          } else {
+            assignedGpus(addr) += 1
+          }
+        }
+        assert(assignedGpus.toMap ===
+          Map("0" -> taskNum, "1" -> taskNum, "2" -> taskNum, "3" -> taskNum))
+      }
+    }
+  }
+
+  test("SPARK-45527 TaskResourceProfile: the left multiple gpu resources on 1 executor " +
+    "can assign to other taskset") {
+    val taskCpus = 1
+    val taskGpus = 0.3
+    val executorGpus = 4
+    val executorCpus = 1000
+
+    // each tasks require 0.3 gpu
+    val taskScheduler = setupScheduler(numCores = executorCpus,
+      config.CPUS_PER_TASK.key -> taskCpus.toString,
+      TASK_GPU_ID.amountConf -> taskGpus.toString,
+      EXECUTOR_GPU_ID.amountConf -> executorGpus.toString,
+      config.EXECUTOR_CORES.key -> executorCpus.toString
+    )
+    val lowerTaskSet = FakeTask.createTaskSet(100, 1, 0, 1,
+      ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
+
+    // each task require 0.7 gpu
+    val treqs = new TaskResourceRequests().cpus(1).resource(GPU, 0.7)
+    val rp = new TaskResourceProfile(treqs.requests)
+    taskScheduler.sc.resourceProfileManager.addResourceProfile(rp)
+
+    val higherRpTaskSet = FakeTask.createTaskSet(1000, stageId = 2, stageAttemptId = 0,
+      priority = 0, rpId = rp.id)
+
+    val workerOffers =
+      IndexedSeq(
+        // cpu won't be a problem
+        WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts(
+          Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))))
+      )
+
+    taskScheduler.submitTasks(lowerTaskSet)
+    taskScheduler.submitTasks(higherRpTaskSet)
+
+    // should have 3 for default profile and 2 for additional resource profile
+    val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+    assert(8 === taskDescriptions.length)
+    var index = 0
+    for (tDesc <- taskDescriptions) {
+      assert(tDesc.resources.contains(GPU))
+      val addresses = tDesc.resources.get(GPU).get.keys.toArray.sorted
+      assert(addresses.length == 1)
+      if (index < 4) { // the first 4 tasks will grab 0.7 gpu
+        assert(addresses(0) == index.toString)
+        assert(ResourceAmountUtils.toFractionalResource(
+          tDesc.resources.get(GPU).get(index.toString)) == 0.7)
+      } else {
+        assert(addresses(0) == (index - 4).toString)
+        assert(ResourceAmountUtils.toFractionalResource(
+          tDesc.resources.get(GPU).get((index - 4).toString)) == 0.3)
+      }
+      index += 1
+    }
+  }
+
+  test("SPARK-45527 TaskResourceProfile: the left gpu resources on multiple executors " +
+    "can assign to other taskset") {
+    val taskCpus = 1
+    val taskGpus = 0.3
+    val executorGpus = 4
+    val executorCpus = 1000

Review Comment:
   Thx for your improvement.



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378859293


##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -165,15 +165,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
     }
 
     override def receive: PartialFunction[Any, Unit] = {
-      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) =>
+      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources, resourcesAmounts) =>

Review Comment:
   no, it doesn't make sense to leave something not used here and we should be as efficient as we can about storage and passing things around.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1873931486

   # Manual test on Spark Standalone Cluster
   
   ## Environment
   
   The Spark Standalone cluster consists of a single worker node equipped with 8 CPU cores but lacks physical GPUs. However, Spark is capable of managing GPU resources by utilizing GPU IDs instead of actual GPUs. To achieve this, you can configure the `spark.worker.resource.gpu.discoveryScript` setting with a script that can retrieve the GPU IDs. For instance,
   
   ```bash
   cat <<EOF
   {"name": "gpu","addresses":["0", "1"]}
   EOF
   ```
   
   So we can change the above script to get 1 GPU/ 2 GPUs or any kind of GPUs.
   
   ## with dynamic allocation off
   
   ### 1 GPU
   
   - configurations
   
   add below configurations in the `SPARK_HOME/conf/spark-defaults.conf`
   
   ``` xml
   spark.worker.resource.gpu.amount 1
   spark.worker.resource.gpu.discoveryScript /tmp/gpu_discovery.sh
   ```
   
   - spark-submit configurations
   
   ```bash
   spark-shell --master spark://192.168.0.103:7077 --conf spark.executor.cores=8 --conf spark.task.cpus=1 \
      --conf spark.executor.resource.gpu.amount=1 --conf spark.task.resource.gpu.amount=0.125 \
      --conf spark.dynamicAllocation.enabled=false
   ```
   
   The aforementioned spark-submit configurations will launch a single executor with 8 CPU cores and 1 GPU. The tasks requires 1 CPU core and 0.125 GPUs each, allowing for the concurrent execution of 8 tasks.
   
   - test code
   
   ``` scala
   import org.apache.spark.TaskContext
   import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests}
   
   val rdd = sc.range(0, 100, 1, 12).mapPartitions { iter => {
     val tc = TaskContext.get()
     val tid = tc.partitionId()
     assert(tc.resources()("gpu").addresses sameElements Array("0"))
     iter
   }}
   
   val rdd1 = rdd.repartition(2)
   val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.6)
   val rp = new ResourceProfileBuilder().require(treqs).build
   val rdd2 = rdd1.withResources(rp).mapPartitions { iter => {
     val tc = TaskContext.get()
     val tid = tc.partitionId()
     assert(tc.resources()("gpu").addresses sameElements Array("0"))
     iter
   }
   }
   rdd2.collect()
   ```
   
   The provided Spark job will be split into two stages. The first stage comprises 12 tasks, each requiring 1 CPU core and 0.125 GPUs. As a result, the first 8 tasks can run concurrently, and then run the remaining 4 tasks. 
   
   ![index_0-dyn-off-1-gpu-shuffle-stages](https://github.com/apache/spark/assets/1320706/36a887f3-75a1-440f-a9f6-dfdf046a34b1)
   
   
   In contrast, the second stage consists of 2 tasks, each necessitating 1 CPU core and 0.6 GPUs. Consequently, only one task will run at any given time, while the remaining 2 tasks will execute sequentially.
   
   
   ![index_1-dyn-off-1-gpu-result-stages](https://github.com/apache/spark/assets/1320706/f1fed300-a86a-4925-9a38-997934493d9e)
   
   
   ### 2 GPUs
   
   - configurations
   
   add below configurations in the `SPARK_HOME/conf/spark-defaults.conf`
   
   ``` xml
   spark.worker.resource.gpu.amount 2
   spark.worker.resource.gpu.discoveryScript /tmp/gpu_discovery.sh
   ```
   
   - spark-submit configurations
   
   ```bash
   spark-shell --master spark://192.168.0.103:7077 --conf spark.executor.cores=8 --conf spark.task.cpus=1 \
      --conf spark.executor.resource.gpu.amount=2 --conf spark.task.resource.gpu.amount=0.25 \
      --conf spark.dynamicAllocation.enabled=false
   ```
   
   The aforementioned spark-submit configurations will launch a single executor with 8 CPU cores and 2 GPU. The tasks requires 1 CPU core and 0.25 GPUs each, allowing for the concurrent execution of 8 tasks. the first 4 tasks will grab GPU 0, while the remaining 4 tasks grabs the GPU 1 due to the round-robin manner when offering the resources.
   
   - test code
   
   ```scala
   import org.apache.spark.TaskContext
   import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests}
   
   val rdd = sc.range(0, 100, 1, 8).mapPartitions { iter => {
     val tc = TaskContext.get()
     val tid = tc.partitionId()
     if (tid >= 4) {
       assert(tc.resources()("gpu").addresses sameElements Array("1"))
     } else {
       assert(tc.resources()("gpu").addresses sameElements Array("0"))
     }
     iter
   }}
   
   val rdd1 = rdd.repartition(2)
   val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.6)
   val rp = new ResourceProfileBuilder().require(treqs).build
   val rdd2 = rdd1.withResources(rp).mapPartitions { iter => {
     val tc = TaskContext.get()
     val tid = tc.partitionId()
     if (tid > 0) {
       assert(tc.resources()("gpu").addresses sameElements Array("1"))
     } else {
       assert(tc.resources()("gpu").addresses sameElements Array("0"))
     }
     iter
   }
   }
   rdd2.collect()
   ```
   
   The provided Spark job will be split into two stages. The first stage comprises 8 tasks, each requiring 1 CPU core and 0.25 GPUs. As a result, the total 8 tasks can run concurrently. The first 4 tasks will grab GPU 0, while the remaining 4 tasks grabs the GPU 1 due to the round-robin manner when offering the resources. The assert line can ensure this policy.
    
   ![index_2-dyn-off-2-gpus-shuffle-stage](https://github.com/apache/spark/assets/1320706/9ed3c43d-57e3-446c-8c8d-ebdc0d7f9bb6)
   
   
   In contrast, the second stage consists of 2 tasks, each necessitating 1 CPU core and 0.6 GPUs, since there're 2 GPUs availabe, so the total 2 tasks can run concurrently, each grabs 1 different GPU, the assert line can ensure that.
   
   ![index_3-dyn-off-2-gpus-result-stage](https://github.com/apache/spark/assets/1320706/fbaac8c2-15be-4f66-9945-af03caebf3e5)
   
   ### concurrent spark jobs
   
   This test case is to ensure the other spark job can still grab the left gpu resources and run alongside the other spark job.
   
   - configurations
   
   add below configurations in the `SPARK_HOME/conf/spark-defaults.conf`
   
   ``` xml
   spark.worker.resource.gpu.amount 2
   spark.worker.resource.gpu.discoveryScript /tmp/gpu_discovery.sh
   ```
   
   - spark-submit configurations
   
   ```bash
   spark-shell --master spark://192.168.0.103:7077 --conf spark.executor.cores=8 --conf spark.task.cpus=1 \
      --conf spark.executor.resource.gpu.amount=2 --conf spark.task.resource.gpu.amount=0.25 \
      --conf spark.dynamicAllocation.enabled=false
   ```
   
   The aforementioned spark-submit configurations will launch a single executor with 8 CPU cores and 2 GPU. The tasks requires 1 CPU core and 0.25 GPUs each, allowing for the concurrent execution of 8 tasks. the first 4 tasks will grab GPU 0, while the remaining 4 tasks grabs the GPU 1 due to the round-robin manner when offering the resources.
   
   - test code
   
   ```scala
       import org.apache.spark.TaskContext
       import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests}
   
       // Submit Spark Job 0 in thread1.
       val thread1 = new Thread(() => {
         val rdd = sc.range(0, 8, 1, 8).mapPartitions { iter => {
           val tc = TaskContext.get()
           val tid = tc.partitionId()
           if (tid >= 4) {
             assert(tc.resources()("gpu").addresses sameElements Array("1"))
           } else {
             assert(tc.resources()("gpu").addresses sameElements Array("0"))
           }
           iter
         }
         }
   
         val rdd1 = rdd.repartition(2)
   
         val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.6)
         val rp = new ResourceProfileBuilder().require(treqs).build
   
         val rdd2 = rdd1.withResources(rp).mapPartitions { iter => {
           val tc = TaskContext.get()
           val tid = tc.partitionId()
           assert(tc.resources()("gpu").addresses sameElements Array(tid.toString))
           println("sleeping 20s")
           Thread.sleep(20000)
           iter
         }
         }
         rdd2.collect()
       })
   
       thread1.start()
       // sleep 5s in main thread to make sure the spark result tasks launched in thread1 are running
       Thread.sleep(5000)
   
       // Submit Spark Job 1 in main thread.
       // Each spark result task in thread1 takes 0.6 gpus, so there is only 0.4 gpus (for each gpu) left.
       // since the default task gpu amount = 0.25, the concurrent spark tasks in Spark Job 1
       // will be 1(0.4/0.25) * 2 (2 gpus)
       val rdd = sc.range(0, 4, 1, 2).mapPartitions(iter => {
         Thread.sleep(10000)
         val tc = TaskContext.get()
         val tid = tc.partitionId()
         if (tid % 2 == 1) {
           assert(tc.resources()("gpu").addresses sameElements Array("1"))
         } else {
           assert(tc.resources()("gpu").addresses sameElements Array("0"))
         }
         iter
       })
       rdd.collect()
   
       thread1.join()
   ```
   
   The given Spark application consists of two spark jobs. The first spark job 0 is submitted in thread1, while the second spark job 1 is submitted in the main thread. To guarantee that the spark job 0 runs prior to the spark job 1, a `sleep 5s` is included in the main thread. As a result, the result tasks in spark job 0 will pause for 20 seconds to await the completion of spark job 1. This is done to test whether spark job 1 can utilize the remaining GPU and execute concurrently with spark job 0.
   
   **Event timeline**
   
   ![index_6-dyn-off-concurrent-total-events](https://github.com/apache/spark/assets/1320706/e18fba0a-3aa7-4f5f-bbda-552677271e52)
   
   from the picture, we can see, the Spark Job 1 was running alongside Spark Job 0 and finished before spark job 0.
   
   
   **spark job 0**
   ![index_7-dyn-off-concurrent-job-0-shuffle](https://github.com/apache/spark/assets/1320706/b24db20c-8691-4268-862f-bfe37795750a)
   ![index_8-dyn-off-concurrent-job-0-result](https://github.com/apache/spark/assets/1320706/bb115d22-3b6f-4cb5-bcdf-f3dab5646b7f)
   
   
   **spark job 1**
   
   ![index_9-dyn-off-concurrent-job1-result](https://github.com/apache/spark/assets/1320706/79e8b828-213c-4f40-8b87-40dbcf161254)
   
   
   
   ---
   
   If we change `val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 1)` to require 1 each GPU for each task, then the spark job 1 will not grab any gpus because the left available GPUs is 0 after spark job is running.
   
   ```scala
       import org.apache.spark.TaskContext
       import org.apache.spark.resource.{ResourceProfileBuilder, TaskResourceRequests}
   
       // Submit Spark Job 0 in thread1.
       val thread1 = new Thread(() => {
         val rdd = sc.range(0, 8, 1, 8).mapPartitions { iter => {
           val tc = TaskContext.get()
           val tid = tc.partitionId()
           if (tid >= 4) {
             assert(tc.resources()("gpu").addresses sameElements Array("1"))
           } else {
             assert(tc.resources()("gpu").addresses sameElements Array("0"))
           }
           iter
         }
         }
   
         val rdd1 = rdd.repartition(2)
   
         val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 1)
         val rp = new ResourceProfileBuilder().require(treqs).build
   
         val rdd2 = rdd1.withResources(rp).mapPartitions { iter => {
           val tc = TaskContext.get()
           val tid = tc.partitionId()
           assert(tc.resources()("gpu").addresses sameElements Array(tid.toString))
           println("sleeping 20s")
           Thread.sleep(20000)
           iter
         }
         }
         rdd2.collect()
       })
   
       thread1.start()
       // sleep 5s in main thread to make sure the spark result tasks launched in thread1 are running
       Thread.sleep(5000)
   
       // Submit Spark Job 1 in main thread.
       // Each spark result task in thread1 takes 1 gpus, so there is no available gpus left for spark job 1.
       // The spark job 1 will run after spark job 0 finished, but we can't ensure which gpu the task will grab. 
       val rdd = sc.range(0, 4, 1, 2).mapPartitions(iter => {
         Thread.sleep(10000)
         val tc = TaskContext.get()
         assert(tc.resources().contains("gpu"))
         iter
       })
       rdd.collect()
   
       thread1.join()
   ```
   ![index_zzz-dyn-off-concurrent-job-sequentially](https://github.com/apache/spark/assets/1320706/0b4dd9eb-91d2-4d78-abb6-15d37e43522a)
   
   From the picture,  we can see, spark job 1 was submitted when spark job 0 was running, but the tasks on spark job 1 didn't run because of a lack of GPU resources. After spark job 0 is finished and releases the GPU, then tasks on spark job 1 can grab the GPUs and run. 
   
   
   
   
   


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1375824754


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {

Review Comment:
   Fixed by changing the way to use it in WorkerInfo



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378499848


##########
core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala:
##########
@@ -513,7 +515,8 @@ private[spark] class TaskSetManager(
       speculative: Boolean,
       taskCpus: Int,
       taskResourceAssignments: Map[String, ResourceInformation],
-      launchTime: Long): TaskDescription = {
+      launchTime: Long,
+      resourcesAmounts: Map[String, Map[String, Double]]): TaskDescription = {

Review Comment:
   I think we can remove taskResourceAssignments and re-construct it according to resourcesAmounts. But I'd like to keep this PR minimum change, and fix them in the follow up



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1399869432


##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  // Acquire the resource and update the resource
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+        // scalastyle:off println
+        println(s"Acquired. left ${realLeft}")
+        // scalastyle:on println
+      }
+    }
+  }
+
+  // release the resources and update the values
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+        // scalastyle:off println
+        println(s"Released. amount ${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT}")
+        // scalastyle:on println
+      }
+    }
+  }
+
+  // Try to assign the address according to the task requirement.
+  // Please note that this function will not update the values.
+  def assignResources(taskSetProf: ResourceProfile):
+      Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = {
+
+    // only look at the resource other than cpus
+    val tsResources = taskSetProf.getCustomTaskResources()
+    if (tsResources.isEmpty) {
+      return Some(Map.empty, Map.empty)
+    }
+
+    val localTaskReqAssign = HashMap[String, ResourceInformation]()
+    val allocatedAddresses = HashMap[String, Map[String, Double]]()
+
+    // we go through all resources here so that we can make sure they match and also get what the
+    // assignments are for the next task
+    for ((rName, taskReqs) <- tsResources) {
+      // if taskAmount = 1.5, we assign 2.0 gpu for user or
+      // just throw an exception in a very begging?
+      // TODO, just remove it, since we enabled the check at the very beginning.
+      val taskAmount = if (taskReqs.amount < 1.0) taskReqs.amount else Math.ceil(taskReqs.amount)

Review Comment:
   The new way will use the original taskAmount directly to do the resource calculation, so we don't need to `taskSetProf.getSchedulerTaskResourceAmount(rName)`. 
   
   > I'm again worried about this changing the behavior with dynamic allocation on.
   
   Yeah, adding the check on the specific ResourceProfiles like TaskResourceProfile or ResourceProfile.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1369478240


##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  // Acquire the resource and update the resource
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+        // scalastyle:off println

Review Comment:
   I will remove the debug log in the next commit



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  // Acquire the resource and update the resource
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+        // scalastyle:off println
+        println(s"Acquired. left ${realLeft}")
+        // scalastyle:on println
+      }
+    }
+  }
+
+  // release the resources and update the values
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+        // scalastyle:off println

Review Comment:
   I will remove the debug log in the next commit



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1809672224

   > Need to check barrier scheduling algorithm, which checks for max slots to make sure this doesn't break that.
   > 
   > I'm also curious if you are getting warnings out from the warnOnWastedResources function when setting things like this? Maybe its bypassing those checks, but like getNumSlotsPerAddress which you removed usage in code is still used for those warnings.
   
   Yeah. Added the tests for barrier scheduling. please refer to [here](https://github.com/apache/spark/pull/43494/files#diff-9c73792f401d6969f0a4c33839f22d37354a4ee4e33020287bb4b3c2973fd4ecR2303) and [here](https://github.com/apache/spark/pull/43494/files#diff-9c73792f401d6969f0a4c33839f22d37354a4ee4e33020287bb4b3c2973fd4ecR2349) and [here](https://github.com/apache/spark/pull/43494/files#diff-9c73792f401d6969f0a4c33839f22d37354a4ee4e33020287bb4b3c2973fd4ecR2406) and [here](https://github.com/apache/spark/pull/43494/files#diff-9c73792f401d6969f0a4c33839f22d37354a4ee4e33020287bb4b3c2973fd4ecR2454)
   
   and tests for warnOnWastedResources, please refer to [here](https://github.com/apache/spark/pull/43494/files#diff-6e4265cf1ad3e318e98ae4148871f7953b21772d4dad1b4c733d17d91165f54bR341) and [here](https://github.com/apache/spark/pull/43494/files#diff-6e4265cf1ad3e318e98ae4148871f7953b21772d4dad1b4c733d17d91165f54bR441) and [here](https://github.com/apache/spark/pull/43494/files#diff-6e4265cf1ad3e318e98ae4148871f7953b21772d4dad1b4c733d17d91165f54bR511)


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1399864198


##########
core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala:
##########
@@ -171,16 +171,16 @@ private[spark] object ResourceUtils extends Logging {
   // integer amount and the number of slots per address. For instance, if the amount is 0.5,

Review Comment:
   Yes, `getSchedulerTaskResourceAmount` is only used by warnOnWastedResources to calculate the max task number per executor, please refer to [here](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala#L474-L476), while warnOnWastedResources will be used when checking resource profile, or calculateTasksAndLimitingResource, Please refer to [here](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala#L228) and [here](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L3155)



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378466375


##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -58,6 +58,9 @@ private[spark] class TaskDescription(
     val properties: Properties,
     val cpus: Int,
     val resources: immutable.Map[String, ResourceInformation],
+    // resourcesAmounts is the total resources assigned to the task
+    // Eg, Map("gpu" -> Map("0" -> 0.7)): assign 0.7 of the gpu address "0" to this task
+    val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]],

Review Comment:
   My original thinking is we can remove `val resources: immutable.Map[String, ResourceInformation]` and re-construct it in the TaskContext according to resourcesAmounts.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1807362816

   Hi Tom, the resource amount info will be displayed in the log like the below,
   
   ``` console
   23/11/13 09:40:22 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1) (10.51.70.102, executor 0, partition 1, 
   PROCESS_LOCAL, 7823 bytes) taskResourceAssignments Map(gpu -> Map(0 -> 2000000000000000))
   ```
   
   Compared to the original log info 
   
   ``` console
   23/11/13 09:38:47 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 5) (10.51.70.102, executor 0, partition 0, 
   NODE_LOCAL, 7713 bytes) taskResourceAssignments Map(gpu -> [name: gpu, addresses: 0])
   ```
   
   Should I change this behavior to align with the original one?


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1477018575


##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -57,7 +56,10 @@ private[spark] class TaskDescription(
     val artifacts: JobArtifactSet,
     val properties: Properties,
     val cpus: Int,
-    val resources: immutable.Map[String, ResourceInformation],
+    // resources is the total resources assigned to the task
+    // Eg, Map("gpu" -> Map("0" -> ResourceAmountUtils.toInternalResource(0.7))):
+    // assign 0.7 of the gpu address "0" to this task
+    val resources: immutable.Map[String, immutable.Map[String, Long]],

Review Comment:
   Do you think we should have a new PR for master branch for your comment? @Ngone51 



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1430385324


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -20,6 +20,61 @@ package org.apache.spark.resource
 import scala.collection.mutable
 
 import org.apache.spark.SparkException
+import org.apache.spark.resource.ResourceAmountUtils.ONE_ENTIRE_RESOURCE
+
+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 ONE_ENTIRE_RESOURCE to convert the double to long to avoid this limitation.
+   * Double can display up to 16 decimal places, so we set the factor to
+   * 10, 000, 000, 000, 000, 000L.
+   */
+  final val ONE_ENTIRE_RESOURCE: Long = 10000000000000000L
+
+  def isOneEntireResource(amount: Long): Boolean = amount == ONE_ENTIRE_RESOURCE
+
+  def toInternalResource(amount: Double): Long = (amount * ONE_ENTIRE_RESOURCE).toLong
+
+  private[spark] def toInternalResource(resources: Map[String, Double]): Map[String, Long] = {
+    resources.map { case (k, v) => k -> toInternalResource(v) }
+  }
+
+  def toFractionalResource(amount: Long): Double = amount.toDouble / ONE_ENTIRE_RESOURCE
+
+  private[spark] def toFractionalResource(resources: Map[String, Long]): Map[String, Double] = {
+    resources.map { case (k, v) => k -> toFractionalResource(v) }
+  }
+
+  private[spark] def toInternalResourceMapMap(resources: Map[String, Map[String, Double]]):

Review Comment:
   this seems to only be used for testing?  If so can we just move to utility function in that suite since these are private to spark.



##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -192,7 +181,9 @@ 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) =>

Review Comment:
   this is not used, remove it



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1395211436


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378502033


##########
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:
   Yeah, I finally understand your concern. Let me try to figure out a way not to change this part with dynamic allocation on.



##########
core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala:
##########
@@ -37,8 +37,8 @@ import org.apache.spark.annotation.{Evolving, Since}
 class TaskResourceRequest(val resourceName: String, val amount: Double)
   extends Serializable {
 
-  assert(amount <= 0.5 || amount % 1 == 0,
-    s"The resource amount ${amount} must be either <= 0.5, or a whole number.")
+  assert(amount <= 1.0 || amount % 1 == 0,

Review Comment:
   Good suggestion, seems doable.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  /**
+   * Multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+   * and convert the addressesAmounts to be mutable.HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * Map("gpu" -> Map("0" -> 0.5, "1" -> 0.5, "2" -> 0.5),
+   *     "fpga" -> Map("a" -> 0.5, "b" -> 0.5))
+   * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2)
+   */
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  /**
+   * For testing purpose. convert internal resources back to the "fraction" resources.
+   */
+  private[spark] def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+      }
+    }
+  }
+
+  /**
+   * Try to assign the address according to the task requirement.
+   * Please note that this function will not update the values.
+   *
+   * @param taskSetProf assign resources based on which resource profile
+   * @return the resource
+   */
+  def assignResources(taskSetProf: ResourceProfile):
+      Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = {
+
+    // only look at the resource other than cpus
+    val tsResources = taskSetProf.getCustomTaskResources()
+    if (tsResources.isEmpty) {
+      return Some(Map.empty, Map.empty)
+    }
+
+    val localTaskReqAssign = HashMap[String, ResourceInformation]()
+    val allocatedAddresses = HashMap[String, Map[String, Double]]()
+
+    // we go through all resources here so that we can make sure they match and also get what the
+    // assignments are for the next task
+    for ((rName, taskReqs) <- tsResources) {
+      // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number
+      val taskAmount = taskReqs.amount
+
+      internalResources.get(rName) match {
+        case Some(addressesAmountMap) =>
+
+          var internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong

Review Comment:
   yes, you're right. Thx for catching. I will fix it.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378465451


##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -273,7 +273,8 @@ private[spark] class CoarseGrainedExecutorBackend(
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = {
     val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation])

Review Comment:
   yeah, you're right. I'd like to keep the minimum change in this PR, could we clean them up in the followup?



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1379434828


##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -273,7 +273,8 @@ private[spark] class CoarseGrainedExecutorBackend(
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = {
     val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation])

Review Comment:
   got it. I will fix them in this PR.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1375877991


##########
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala:
##########
@@ -468,33 +463,15 @@ private[spark] class TaskSchedulerImpl(
   private def resourcesMeetTaskRequirements(

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -58,6 +58,7 @@ private[spark] class TaskDescription(
     val properties: Properties,
     val cpus: Int,
     val resources: immutable.Map[String, ResourceInformation],
+    val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]],

Review Comment:
   Good suggestion. 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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1376085306


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.

Review Comment:
   Done



##########
core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala:
##########
@@ -28,12 +29,24 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String
 
   override protected def resourceName = this.name
   override protected def resourceAddresses = this.addresses
-  override protected def slotsPerAddress: Int = 1
 
+  /**
+   * Acquire the resources.
+   * @param amount How many addresses are requesting.
+   * @return ResourceInformation
+   */
   def acquire(amount: Int): ResourceInformation = {
-    val allocated = availableAddrs.take(amount)
-    acquire(allocated)
-    new ResourceInformation(resourceName, allocated.toArray)
+
+    var count = amount
+    val allocated: mutable.HashMap[String, Double] = mutable.HashMap.empty
+    for (address <- availableAddrs if count > 0) {
+      if (addressAvailabilityMap(address) == RESOURCE_TOTAL_AMOUNT) {

Review Comment:
   Good suggestion. Done



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.
    */
   def availableAddrs: Seq[String] = addressAvailabilityMap
-    .flatMap { case (addr, available) =>
-      (0 until available).map(_ => addr)
-    }.toSeq.sorted
+      .filter(addresses => addresses._2 > 0).keys.toSeq.sorted

Review Comment:
   Done.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>

Review Comment:
   Good catch. Thx, I will remove it.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378464730


##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -165,15 +165,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
     }
 
     override def receive: PartialFunction[Any, Unit] = {
-      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) =>
+      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources, resourcesAmounts) =>

Review Comment:
   Yeah. I've noticed that. The reason that I didn't change this part is I'd like to keep the minimum change in this PR. is that ok?



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1813235365

   >  23/11/13 09:40:22 INFO TaskSetManager: Starting task 1.0 in stage 0.0 (TID 1) (10.51.70.102, executor 0, partition 1, 
   PROCESS_LOCAL, 7823 bytes) taskResourceAssignments Map(gpu -> Map(0 -> 2000000000000000))
   
   > Should I change this behavior to align with the original one?
   
   Yeah that log message isn't very readable, for now lets just change it back to be similar to before format something like "resource" -> [Addresses], ie (gpu -> [0]).


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1395200464


##########
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:
   Wow, really good suggestion, 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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1383225358


##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  /**
+   * Multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+   * and convert the addressesAmounts to be mutable.HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * Map("gpu" -> Map("0" -> 0.5, "1" -> 0.5, "2" -> 0.5),
+   *     "fpga" -> Map("a" -> 0.5, "b" -> 0.5))
+   * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2)
+   */
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  /**
+   * For testing purpose. convert internal resources back to the "fraction" resources.
+   */
+  private[spark] def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+      }
+    }
+  }
+
+  /**
+   * Try to assign the address according to the task requirement.
+   * Please note that this function will not update the values.
+   *
+   * @param taskSetProf assign resources based on which resource profile
+   * @return the resource
+   */
+  def assignResources(taskSetProf: ResourceProfile):
+      Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = {
+
+    // only look at the resource other than cpus
+    val tsResources = taskSetProf.getCustomTaskResources()
+    if (tsResources.isEmpty) {
+      return Some(Map.empty, Map.empty)
+    }
+
+    val localTaskReqAssign = HashMap[String, ResourceInformation]()
+    val allocatedAddresses = HashMap[String, Map[String, Double]]()
+
+    // we go through all resources here so that we can make sure they match and also get what the
+    // assignments are for the next task
+    for ((rName, taskReqs) <- tsResources) {
+      // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number
+      val taskAmount = taskReqs.amount
+
+      internalResources.get(rName) match {
+        case Some(addressesAmountMap) =>
+
+          var internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong

Review Comment:
   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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1383223802


##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -58,6 +58,9 @@ private[spark] class TaskDescription(
     val properties: Properties,
     val cpus: Int,
     val resources: immutable.Map[String, ResourceInformation],
+    // resourcesAmounts is the total resources assigned to the task
+    // Eg, Map("gpu" -> Map("0" -> 0.7)): assign 0.7 of the gpu address "0" to this task
+    val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]],

Review Comment:
   Done. Removed the resources and reconstructed in the TaskContext



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {

Review Comment:
   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-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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1381839965


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,54 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
   private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Get the resources and its amounts.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map {

Review Comment:
   
   I just saying if we are storing things in long format (10000000000000000L * number of resources requested). Be sure that isn't going to overflow.  Since I said its a generic comment all over, just make sure it isn't going to happen.  If the values we store are always < (10000000000000000L * 1) its not a problem.
   
   Overflow means you have a value larger then 2^64, which then isn't positive anymore
   
   ```
   scala> 10000000000000000L * 2500l
   res7: Long = 6553255926290448384
   
   scala> 10000000000000000L * 25000l
   res8: Long = -8254417031933722624
   
   ```
   



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1777803901

   I'll take a look, might not get to it today.


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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1372201798


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.

Review Comment:
   nit, should be: "which are not fully assigned"



##########
core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala:
##########
@@ -28,12 +29,24 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String
 
   override protected def resourceName = this.name
   override protected def resourceAddresses = this.addresses
-  override protected def slotsPerAddress: Int = 1
 
+  /**
+   * Acquire the resources.
+   * @param amount How many addresses are requesting.
+   * @return ResourceInformation
+   */
   def acquire(amount: Int): ResourceInformation = {
-    val allocated = availableAddrs.take(amount)
-    acquire(allocated)
-    new ResourceInformation(resourceName, allocated.toArray)
+
+    var count = amount
+    val allocated: mutable.HashMap[String, Double] = mutable.HashMap.empty
+    for (address <- availableAddrs if count > 0) {
+      if (addressAvailabilityMap(address) == RESOURCE_TOTAL_AMOUNT) {

Review Comment:
   why do we need this check if WorkerInfo is always dealing with full Resources/GPU's?  isn't this just availableAddrs still?



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>

Review Comment:
   this seems like a weird assert because if asserts off I think its still going to do the foreach.    Either rewrite this or remove it.



##########
core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala:
##########
@@ -37,8 +37,8 @@ import org.apache.spark.annotation.{Evolving, Since}
 class TaskResourceRequest(val resourceName: String, val amount: Double)
   extends Serializable {
 
-  assert(amount <= 0.5 || amount % 1 == 0,
-    s"The resource amount ${amount} must be either <= 0.5, or a whole number.")
+  assert(amount <= 1.0 || amount % 1 == 0,

Review Comment:
   same here, this shouldn't change behavior for dynamic allocation



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  // Acquire the resource and update the resource
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+        // scalastyle:off println
+        println(s"Acquired. left ${realLeft}")
+        // scalastyle:on println
+      }
+    }
+  }
+
+  // release the resources and update the values
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+        // scalastyle:off println
+        println(s"Released. amount ${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT}")
+        // scalastyle:on println
+      }
+    }
+  }
+
+  // Try to assign the address according to the task requirement.
+  // Please note that this function will not update the values.
+  def assignResources(taskSetProf: ResourceProfile):
+      Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = {
+
+    // only look at the resource other than cpus
+    val tsResources = taskSetProf.getCustomTaskResources()
+    if (tsResources.isEmpty) {
+      return Some(Map.empty, Map.empty)
+    }
+
+    val localTaskReqAssign = HashMap[String, ResourceInformation]()
+    val allocatedAddresses = HashMap[String, Map[String, Double]]()
+
+    // we go through all resources here so that we can make sure they match and also get what the
+    // assignments are for the next task
+    for ((rName, taskReqs) <- tsResources) {
+      // if taskAmount = 1.5, we assign 2.0 gpu for user or
+      // just throw an exception in a very begging?
+      // TODO, just remove it, since we enabled the check at the very beginning.
+      val taskAmount = if (taskReqs.amount < 1.0) taskReqs.amount else Math.ceil(taskReqs.amount)

Review Comment:
   why isn't this just using    taskSetProf.getSchedulerTaskResourceAmount(rName)  and that function updated to do the right thing ?
   
   Also there is a comment in Resourceprofile.scala in the getSchedulerTaskResourceAmount that assumes addresses are done old way.
   
   I'm again worried about this changing the behavior with dynamic allocation on.



##########
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala:
##########
@@ -389,7 +389,7 @@ private[spark] class TaskSchedulerImpl(
       maxLocality: TaskLocality,
       shuffledOffers: Seq[WorkerOffer],
       availableCpus: Array[Int],
-      availableResources: Array[Map[String, Buffer[String]]],
+      availableResources: Array[ExecutorResourcesAmounts],

Review Comment:
   did the caller of this get updated?  it doesn't look like it unless I'm missing it or my ide didn't update



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.
    */
   def availableAddrs: Seq[String] = addressAvailabilityMap
-    .flatMap { case (addr, available) =>
-      (0 until available).map(_ => addr)
-    }.toSeq.sorted
+      .filter(addresses => addresses._2 > 0).keys.toSeq.sorted
 
   /**
    * Sequence of currently assigned resource addresses.

Review Comment:
    seems like this now returns something quite a bit different.  I think this goes to the UI so I assume the UI is losing details now, right? ie will just say if one resource is used but not how much of it.  I think we should find a better way to display what is really used



##########
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:
   so this changes this condition for all the resourceprofiles, even with dynamic allocation, which doesn't support changing task resources within a profile.  I don't think we want that, or at least shouldn't be part of the scope of this change. 



##########
core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala:
##########
@@ -28,12 +29,24 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String
 
   override protected def resourceName = this.name
   override protected def resourceAddresses = this.addresses
-  override protected def slotsPerAddress: Int = 1
 
+  /**
+   * Acquire the resources.
+   * @param amount How many addresses are requesting.
+   * @return ResourceInformation
+   */
   def acquire(amount: Int): ResourceInformation = {
-    val allocated = availableAddrs.take(amount)
-    acquire(allocated)
-    new ResourceInformation(resourceName, allocated.toArray)
+
+    var count = amount
+    val allocated: mutable.HashMap[String, Double] = mutable.HashMap.empty
+    for (address <- availableAddrs if count > 0) {
+      if (addressAvailabilityMap(address) == RESOURCE_TOTAL_AMOUNT) {
+        allocated(address) = 1.0

Review Comment:
   This code is weird to read by itself. please add a comment explaining why doing this vs calling availableAddrs. I think its because this is worker and need to do full GPU to an executor and I guess we just reuse the same ResourceAllocator class



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala:
##########
@@ -18,23 +18,45 @@
 package org.apache.spark.scheduler
 
 import org.apache.spark.resource.{ResourceAllocator, ResourceInformation}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
 
 /**
  * Class to hold information about a type of Resource on an Executor. This information is managed
  * by SchedulerBackend, and TaskScheduler shall schedule tasks on idle Executors based on the
  * information.
  * @param name Resource name
  * @param addresses Resource addresses provided by the executor
- * @param numParts Number of ways each resource is subdivided when scheduling tasks
  */
 private[spark] class ExecutorResourceInfo(
     name: String,
-    addresses: Seq[String],
-    numParts: Int)
+    addresses: Seq[String])
   extends ResourceInformation(name, addresses.toArray) with ResourceAllocator {
 
   override protected def resourceName = this.name
+
   override protected def resourceAddresses = this.addresses
-  override protected def slotsPerAddress: Int = numParts
-  def totalAddressAmount: Int = resourceAddresses.length * slotsPerAddress
+
+  /**
+   * Calculate how many parts the executor can offer according to the task resource amount
+   * @param taskAmount how many resource amount the task required
+   * @return the total parts
+   */
+  def totalParts(taskAmount: Double): Int = {
+    assert(taskAmount > 0.0)
+    if (taskAmount >= 1.0) {
+      addresses.length / taskAmount.ceil.toInt
+    } else {
+      addresses.length * Math.floor(1.0 / taskAmount).toInt
+    }
+  }
+
+  /**
+   * Convert the internal address availability to the public resource format

Review Comment:
   what is a public resource format?



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {

Review Comment:
   why open up permissions here?  I see this is accessed from WorkerInfo but I question if that is the right way to do that.



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.
    */
   def availableAddrs: Seq[String] = addressAvailabilityMap
-    .flatMap { case (addr, available) =>
-      (0 until available).map(_ => addr)
-    }.toSeq.sorted
+      .filter(addresses => addresses._2 > 0).keys.toSeq.sorted

Review Comment:
   indentation looks off here



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.

Review Comment:
   define what public is



##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -58,6 +58,7 @@ private[spark] class TaskDescription(
     val properties: Properties,
     val cpus: Int,
     val resources: immutable.Map[String, ResourceInformation],
+    val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]],

Review Comment:
   this is resource amounts needed by the task, assigned to the task, other?  please clarify name and description if needed.    Ideally explain what the map is in the inner part.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {

Review Comment:
   if only available for testing, state so, otherwise this should be private



##########
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala:
##########
@@ -468,33 +463,15 @@ private[spark] class TaskSchedulerImpl(
   private def resourcesMeetTaskRequirements(

Review Comment:
   description of this should be updated.



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.

Review Comment:
   this might need more description, this seems to be from a resource name ("GPU") to the number of addresses that have some capacity.  It looks like its only used for testing? If so add a comment saying that



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1377768603


##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -273,7 +273,8 @@ private[spark] class CoarseGrainedExecutorBackend(
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = {
     val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation])

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



##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -165,15 +165,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
     }
 
     override def receive: PartialFunction[Any, Unit] = {
-      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) =>
+      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources, resourcesAmounts) =>

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



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {

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



##########
core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala:
##########
@@ -37,8 +37,8 @@ import org.apache.spark.annotation.{Evolving, Since}
 class TaskResourceRequest(val resourceName: String, val amount: Double)
   extends Serializable {
 
-  assert(amount <= 0.5 || amount % 1 == 0,
-    s"The resource amount ${amount} must be either <= 0.5, or a whole number.")
+  assert(amount <= 1.0 || amount % 1 == 0,

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



##########
core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala:
##########
@@ -58,6 +58,9 @@ private[spark] class TaskDescription(
     val properties: Properties,
     val cpus: Int,
     val resources: immutable.Map[String, ResourceInformation],
+    // resourcesAmounts is the total resources assigned to the task
+    // Eg, Map("gpu" -> Map("0" -> 0.7)): assign 0.7 of the gpu address "0" to this task
+    val resourcesAmounts: immutable.Map[String, immutable.Map[String, Double]],

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



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  /**
+   * Multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+   * and convert the addressesAmounts to be mutable.HashMap
+   */
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  /**
+   * The total address count of each resource. Eg,
+   * Map("gpu" -> Map("0" -> 0.5, "1" -> 0.5, "2" -> 0.5),
+   *     "fpga" -> Map("a" -> 0.5, "b" -> 0.5))
+   * the resourceAmount will be Map("gpu" -> 3, "fpga" -> 2)
+   */
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  /**
+   * For testing purpose. convert internal resources back to the "fraction" resources.
+   */
+  private[spark] def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  /**
+   * Acquire the resource and update the resource
+   * @param assignedResource the assigned resource information
+   */
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+      }
+    }
+  }
+
+  /**
+   * Release the assigned resources to the resource pool
+   * @param assignedResource resource to be released
+   */
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+      }
+    }
+  }
+
+  /**
+   * Try to assign the address according to the task requirement.
+   * Please note that this function will not update the values.
+   *
+   * @param taskSetProf assign resources based on which resource profile
+   * @return the resource
+   */
+  def assignResources(taskSetProf: ResourceProfile):
+      Option[(Map[String, ResourceInformation], Map[String, Map[String, Double]])] = {
+
+    // only look at the resource other than cpus
+    val tsResources = taskSetProf.getCustomTaskResources()
+    if (tsResources.isEmpty) {
+      return Some(Map.empty, Map.empty)
+    }
+
+    val localTaskReqAssign = HashMap[String, ResourceInformation]()
+    val allocatedAddresses = HashMap[String, Map[String, Double]]()
+
+    // we go through all resources here so that we can make sure they match and also get what the
+    // assignments are for the next task
+    for ((rName, taskReqs) <- tsResources) {
+      // TaskResourceRequest checks the task amount should be in (0, 1] or a whole number
+      val taskAmount = taskReqs.amount
+
+      internalResources.get(rName) match {
+        case Some(addressesAmountMap) =>
+
+          var internalTaskAmount = (taskAmount * RESOURCE_TOTAL_AMOUNT).toLong

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



##########
core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala:
##########
@@ -28,12 +28,21 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String
 
   override protected def resourceName = this.name
   override protected def resourceAddresses = this.addresses
-  override protected def slotsPerAddress: Int = 1
 
+  /**
+   * Acquire the resources.
+   * @param amount How many addresses are requesting.
+   * @return ResourceInformation
+   */
   def acquire(amount: Int): ResourceInformation = {
-    val allocated = availableAddrs.take(amount)
-    acquire(allocated)
-    new ResourceInformation(resourceName, allocated.toArray)
+

Review Comment:
   nit remove extra newline



##########
core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala:
##########
@@ -513,7 +515,8 @@ private[spark] class TaskSetManager(
       speculative: Boolean,
       taskCpus: Int,
       taskResourceAssignments: Map[String, ResourceInformation],
-      launchTime: Long): TaskDescription = {
+      launchTime: Long,
+      resourcesAmounts: Map[String, Map[String, Double]]): TaskDescription = {

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



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,54 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
   private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Get the resources and its amounts.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map {

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



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

To unsubscribe, e-mail: reviews-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1375821367


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.
    */
   def availableAddrs: Seq[String] = addressAvailabilityMap
-    .flatMap { case (addr, available) =>
-      (0 until available).map(_ => addr)
-    }.toSeq.sorted
+      .filter(addresses => addresses._2 > 0).keys.toSeq.sorted
 
   /**
    * Sequence of currently assigned resource addresses.

Review Comment:
   Yeah, You're right. I haven't considered the UI case. Yes, the UI won't have this detail yet.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1379714430


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,54 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
   private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Get the resources and its amounts.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map {

Review Comment:
   Hi Tom, for "if we start getting into the requests where user could ask for 250000 resources then we could hit overflow issues"
   
   I couldn't understand why hitting the overflow issue?



##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,54 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
   private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Get the resources and its amounts.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map {

Review Comment:
   Yes, Leave in the Long should be more effective.



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1384051957


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1395199162


##########
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:
   Sounds good. new commits have removed the taskResources



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


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

Posted by "asfgit (via GitHub)" <gi...@apache.org>.
asfgit closed pull request #43494: [SPARK-45527][CORE] Use fraction to do the resource calculation
URL: https://github.com/apache/spark/pull/43494


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1383225086


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,54 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
   private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Get the resources and its amounts.
+   * @return the resources amounts
+   */
+  def resourcesAmounts: Map[String, Double] = addressAvailabilityMap.map {

Review Comment:
   Thx for the explanation, Yeah, the values we store are always < 10000000000000000L * 1), so overflow is not going to happen.



##########
core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala:
##########
@@ -513,7 +515,8 @@ private[spark] class TaskSetManager(
       speculative: Boolean,
       taskCpus: Int,
       taskResourceAssignments: Map[String, ResourceInformation],
-      launchTime: Long): TaskDescription = {
+      launchTime: Long,
+      resourcesAmounts: Map[String, Map[String, Double]]): TaskDescription = {

Review Comment:
   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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1873939433

   Hi @tgravescs, I did the manual tests on dynamic allocation on/off on the spark standalone cluster. Please check the comments of [dynamic allocation off](https://github.com/apache/spark/pull/43494#issuecomment-1873931486) and [dynamic allocation on](https://github.com/apache/spark/pull/43494#issuecomment-1873934512)
   
   If you think I still need to do some other tests, please feel free to tell me. Thx very much.


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1873934512

   ## with dynamic allocation off
   
   ### 1 GPU
   
   - configurations
   
   add below configurations in the `SPARK_HOME/conf/spark-defaults.conf`
   
   ``` xml
   spark.worker.resource.gpu.amount 1
   spark.worker.resource.gpu.discoveryScript /tmp/gpu_discovery.sh
   ```
   
   - spark-submit configurations
   
   ```bash
   spark-shell --master spark://192.168.0.103:7077 --conf spark.executor.instances=1 --conf spark.executor.cores=4 --conf spark.task.cpus=1 \
    --conf spark.dynamicAllocation.enabled=true
   ```
   
   By utilizing the aforementioned spark-submit configurations, dynamic allocation is enabled, resulting in the launch of an initial executor equipped with 4 CPU cores. As each task requires 1 CPU core, this setup allows for the simultaneous execution of 4 tasks.
   
   - test code
   
   ``` scala
       import org.apache.spark.TaskContext
       import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfileBuilder, TaskResourceRequests}
   
       val rdd = sc.range(0, 100, 1, 6).mapPartitions { iter => {
         val tc = TaskContext.get()
         assert(!tc.resources().contains("gpu"))
         iter
       }
       }
   
       val ereqs = new ExecutorResourceRequests().cores(8).resource("gpu", 1)
       val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.125)
       val rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build
   
       val rdd1 = rdd.repartition(12).withResources(rp).mapPartitions { iter => {
         Thread.sleep(1000)
         val tc = TaskContext.get()
         assert(tc.resources()("gpu").addresses sameElements Array("0"))
         iter
       }
       }
       rdd1.collect()
   ```
   
   The provided Spark job will be split into two stages. The first stage comprises 6 tasks, each requiring 1 CPU core for the default profile file. As a result, the first 4 tasks can run concurrently, and then run the remaining 2 tasks. 
   
   ![index_0-dyn-on-1-gpu-shuffle-stage](https://github.com/apache/spark/assets/1320706/3b0fbdc9-4984-4a23-a1ce-f98c3433dc34)
   
   Within the second stage, there are a total of 12 tasks that demand a distinct resource profile. These tasks necessitate executors equipped with 8 cores and 1 GPU, with each individual task requiring 1 CPU core and 0.125 GPUs. As a result, the initial 8 tasks will execute concurrently, followed by the subsequent 4 tasks.
   
   ![index_1-dyn-on-1-gpu-result-stage](https://github.com/apache/spark/assets/1320706/cf5e2378-4690-4a50-ae05-28d50eb67a11)
   
   
   ### 2 GPUs
   
   - configurations
   
   add below configurations in the `SPARK_HOME/conf/spark-defaults.conf`
   
   ``` xml
   spark.worker.resource.gpu.amount 2
   spark.worker.resource.gpu.discoveryScript /tmp/gpu_discovery.sh
   ```
   
   - spark-submit configurations
   
   ```bash
   spark-shell --master spark://192.168.0.103:7077 --conf spark.executor.instances=1 --conf spark.executor.cores=4 --conf spark.task.cpus=1 \
      --conf spark.dynamicAllocation.enabled=true
   ```
   
   By utilizing the aforementioned spark-submit configurations, dynamic allocation is enabled, resulting in the launch of an initial executor equipped with 4 CPU cores. As each task requires 1 CPU core, this setup allows for the simultaneous execution of 4 tasks.
   
   - test code
   
   ```scala
       import org.apache.spark.TaskContext
       import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfileBuilder, TaskResourceRequests}
   
       val rdd = sc.range(0, 100, 1, 6).mapPartitions { iter => {
         val tc = TaskContext.get()
         assert(!tc.resources().contains("gpu"))
         iter
       }
       }
   
       val ereqs = new ExecutorResourceRequests().cores(8).resource("gpu", 2)
       val treqs = new TaskResourceRequests().cpus(1).resource("gpu", 0.25)
       val rp = new ResourceProfileBuilder().require(ereqs).require(treqs).build
   
       val rdd1 = rdd.repartition(8).withResources(rp).mapPartitions { iter => {
         Thread.sleep(1000)
         val tc = TaskContext.get()
         val tid = tc.partitionId()
         if (tid >= 4) {
           assert(tc.resources()("gpu").addresses sameElements Array("1"))
         } else {
           assert(tc.resources()("gpu").addresses sameElements Array("0"))
         }
         iter
       }
       }
       rdd1.collect()
   ```
   
   The provided Spark job will be split into two stages. The first stage comprises 6 tasks, each requiring 1 CPU core for the default profile file. As a result, the first 4 tasks can run concurrently, and then run the remaining 2 tasks. 
    
   ![index_5-dyn-on-2-gpus-shuffle](https://github.com/apache/spark/assets/1320706/b543d356-51f9-43e0-a8b5-adae085a2eba)
   
   
   Within the second stage, there are a total of 8 tasks that demand a distinct resource profile. These tasks necessitate executors equipped with 8 cores and 2 GPUs, with each individual task requiring 1 CPU core and 0.25 GPUs. As a result, the total 8 tasks will execute concurrently, the first 4 tasks will grab GPU ID 0 while the remaining 4 tasks will grab GPU ID 1. The assert line can ensure that.
   
   ![index_6-dyn-on-2-gpus-result-stage](https://github.com/apache/spark/assets/1320706/d9f7e185-e737-4617-b7a6-d90cae04ec9f)
   
   


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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1395199672


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1395204414


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1375804675


##########
core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala:
##########
@@ -28,12 +29,24 @@ private[spark] case class WorkerResourceInfo(name: String, addresses: Seq[String
 
   override protected def resourceName = this.name
   override protected def resourceAddresses = this.addresses
-  override protected def slotsPerAddress: Int = 1
 
+  /**
+   * Acquire the resources.
+   * @param amount How many addresses are requesting.
+   * @return ResourceInformation
+   */
   def acquire(amount: Int): ResourceInformation = {
-    val allocated = availableAddrs.take(amount)
-    acquire(allocated)
-    new ResourceInformation(resourceName, allocated.toArray)
+
+    var count = amount
+    val allocated: mutable.HashMap[String, Double] = mutable.HashMap.empty
+    for (address <- availableAddrs if count > 0) {
+      if (addressAvailabilityMap(address) == RESOURCE_TOTAL_AMOUNT) {
+        allocated(address) = 1.0

Review Comment:
   Good suggestion. I modified the code so that it no longer checks if it's a complete GPU, as any address from the availableAddrs implies it must be a full GPU. Besides that, I can prevent the need to grant access to the addressAvailabilityMap.



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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
tgravescs commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378869370


##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -273,7 +273,8 @@ private[spark] class CoarseGrainedExecutorBackend(
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = {
     val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation])

Review Comment:
   No



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


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

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1503368176


##########
core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala:
##########
@@ -2283,4 +2295,425 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext
     taskScheduler.handleFailedTask(tsm, tid, state, reason)
   }
 
+  private implicit def toInternalResource(resources: Map[String, Double]): Map[String, Long] =
+    resources.map { case (k, v) => k -> ResourceAmountUtils.toInternalResource(v) }
+
+  // 1 executor with 4 GPUS
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run in the same executor") {
+        val taskCpus = 1
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+        val executorGpus = 1
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> taskCpus.toString,
+          TASK_GPU_ID.amountConf -> gpuTaskAmount.toString,
+          EXECUTOR_GPU_ID.amountConf -> executorGpus.toString,
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum)
+        }
+
+        val resources = new ExecutorResourcesAmounts(
+          Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))
+
+        val workerOffers =
+          IndexedSeq(WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources))
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements.
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        var gpuAddress = -1
+        for (taskId <- 0 until 4 * taskNum) {
+          if (taskId % taskNum == 0) {
+            gpuAddress += 1
+          }
+          assert(ArrayBuffer(gpuAddress.toString) ===
+            taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted)
+        }
+      }
+    }
+  }
+
+  // 4 executors, each of which has 1 GPU
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 default rp with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run on the different executor") {
+        val taskCpus = 1
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+        val executorGpus = 1
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> taskCpus.toString,
+          TASK_GPU_ID.amountConf -> gpuTaskAmount.toString,
+          EXECUTOR_GPU_ID.amountConf -> executorGpus.toString,
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum)
+        }
+
+        val workerOffers =
+          IndexedSeq(
+            WorkerOffer("executor0", "host0", executorCpus, Some("host0"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("0" -> 1.0))))),
+            WorkerOffer("executor1", "host1", executorCpus, Some("host1"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("1" -> 1.0))))),
+            WorkerOffer("executor2", "host2", executorCpus, Some("host2"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("2" -> 1.0))))),
+            WorkerOffer("executor3", "host3", executorCpus, Some("host3"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("3" -> 1.0))))))
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements
+
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        val assignedGpus: HashMap[String, Int] = HashMap.empty
+        for (taskId <- 0 until 4 * taskNum) {
+          val gpus = taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted
+          assert(gpus.length == 1)
+          val addr = gpus(0)
+          if (!assignedGpus.contains(addr)) {
+            assignedGpus(addr) = 1
+          } else {
+            assignedGpus(addr) += 1
+          }
+        }
+        assert(assignedGpus.toMap ===
+          Map("0" -> taskNum, "1" -> taskNum, "2" -> taskNum, "3" -> taskNum))
+      }
+    }
+  }
+
+  // 1 executor with 4 GPUS
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run in the same executor") {
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> "1",
+          TASK_GPU_ID.amountConf -> "0.1",
+          EXECUTOR_GPU_ID.amountConf -> "4",
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val treqs = new TaskResourceRequests().cpus(1).resource(GPU, gpuTaskAmount)
+        val rp = new TaskResourceProfile(treqs.requests)
+        taskScheduler.sc.resourceProfileManager.addResourceProfile(rp)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100, 0, 1, 1, rp.id)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id)
+        }
+        val resources = new ExecutorResourcesAmounts(
+          Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0))))
+
+        val workerOffers = IndexedSeq(
+          WorkerOffer("executor0", "host0", executorCpus, Some("host0"), resources, rp.id)
+        )
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements.
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        var gpuAddress = -1
+        for (taskId <- 0 until 4 * taskNum) {
+          if (taskId % taskNum == 0) {
+            gpuAddress += 1
+          }
+          assert(ArrayBuffer(gpuAddress.toString) ===
+            taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted)
+        }
+      }
+    }
+  }
+
+  // 4 executors, each of which has 1 GPU
+  Seq(true, false).foreach { barrierMode =>
+    val barrier = if (barrierMode) "barrier" else ""
+    (1 to 20).foreach { taskNum =>
+      val gpuTaskAmount = ResourceAmountUtils.toFractionalResource(ONE_ENTIRE_RESOURCE / taskNum)
+      test(s"SPARK-45527 TaskResourceProfile with task.gpu.amount=${gpuTaskAmount} can " +
+        s"restrict $taskNum $barrier tasks run on the different executor") {
+        val executorCpus = 100 // cpu will not limit the concurrent tasks number
+
+        val taskScheduler = setupScheduler(numCores = executorCpus,
+          config.CPUS_PER_TASK.key -> "1",
+          TASK_GPU_ID.amountConf -> "0.1",
+          EXECUTOR_GPU_ID.amountConf -> "1",
+          config.EXECUTOR_CORES.key -> executorCpus.toString)
+
+        val treqs = new TaskResourceRequests().cpus(1).resource(GPU, gpuTaskAmount)
+        val rp = new TaskResourceProfile(treqs.requests)
+        taskScheduler.sc.resourceProfileManager.addResourceProfile(rp)
+
+        val taskSet = if (barrierMode) {
+          FakeTask.createTaskSet(100, 0, 1, 1, rp.id)
+        } else {
+          FakeTask.createBarrierTaskSet(4 * taskNum, 0, 1, 1, rp.id)
+        }
+
+        val workerOffers =
+          IndexedSeq(
+            WorkerOffer("executor0", "host0", executorCpus, Some("host1"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("0" -> 1.0)))),
+              rp.id),
+            WorkerOffer("executor1", "host1", executorCpus, Some("host2"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("1" -> 1.0)))),
+              rp.id),
+            WorkerOffer("executor2", "host2", executorCpus, Some("host3"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("2" -> 1.0)))),
+              rp.id),
+            WorkerOffer("executor3", "host3", executorCpus, Some("host4"),
+              new ExecutorResourcesAmounts(Map(GPU -> toInternalResource(Map("3" -> 1.0)))),
+              rp.id)
+          )
+
+        taskScheduler.submitTasks(taskSet)
+        // Launch tasks on executor that satisfies resource requirements
+
+        val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+        assert(4 * taskNum === taskDescriptions.length)
+        assert(!failedTaskSet)
+        val assignedGpus: HashMap[String, Int] = HashMap.empty
+        for (taskId <- 0 until 4 * taskNum) {
+          val gpus = taskDescriptions(taskId).resources.get(GPU).get.keys.toArray.sorted
+          assert(gpus.length == 1)
+          val addr = gpus(0)
+          if (!assignedGpus.contains(addr)) {
+            assignedGpus(addr) = 1
+          } else {
+            assignedGpus(addr) += 1
+          }
+        }
+        assert(assignedGpus.toMap ===
+          Map("0" -> taskNum, "1" -> taskNum, "2" -> taskNum, "3" -> taskNum))
+      }
+    }
+  }
+
+  test("SPARK-45527 TaskResourceProfile: the left multiple gpu resources on 1 executor " +
+    "can assign to other taskset") {
+    val taskCpus = 1
+    val taskGpus = 0.3
+    val executorGpus = 4
+    val executorCpus = 1000
+
+    // each tasks require 0.3 gpu
+    val taskScheduler = setupScheduler(numCores = executorCpus,
+      config.CPUS_PER_TASK.key -> taskCpus.toString,
+      TASK_GPU_ID.amountConf -> taskGpus.toString,
+      EXECUTOR_GPU_ID.amountConf -> executorGpus.toString,
+      config.EXECUTOR_CORES.key -> executorCpus.toString
+    )
+    val lowerTaskSet = FakeTask.createTaskSet(100, 1, 0, 1,
+      ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
+
+    // each task require 0.7 gpu
+    val treqs = new TaskResourceRequests().cpus(1).resource(GPU, 0.7)
+    val rp = new TaskResourceProfile(treqs.requests)
+    taskScheduler.sc.resourceProfileManager.addResourceProfile(rp)
+
+    val higherRpTaskSet = FakeTask.createTaskSet(1000, stageId = 2, stageAttemptId = 0,
+      priority = 0, rpId = rp.id)
+
+    val workerOffers =
+      IndexedSeq(
+        // cpu won't be a problem
+        WorkerOffer("executor0", "host0", 1000, None, new ExecutorResourcesAmounts(
+          Map(GPU -> toInternalResource(Map("0" -> 1.0, "1" -> 1.0, "2" -> 1.0, "3" -> 1.0)))))
+      )
+
+    taskScheduler.submitTasks(lowerTaskSet)
+    taskScheduler.submitTasks(higherRpTaskSet)
+
+    // should have 3 for default profile and 2 for additional resource profile
+    val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+    assert(8 === taskDescriptions.length)
+    var index = 0
+    for (tDesc <- taskDescriptions) {
+      assert(tDesc.resources.contains(GPU))
+      val addresses = tDesc.resources.get(GPU).get.keys.toArray.sorted
+      assert(addresses.length == 1)
+      if (index < 4) { // the first 4 tasks will grab 0.7 gpu
+        assert(addresses(0) == index.toString)
+        assert(ResourceAmountUtils.toFractionalResource(
+          tDesc.resources.get(GPU).get(index.toString)) == 0.7)
+      } else {
+        assert(addresses(0) == (index - 4).toString)
+        assert(ResourceAmountUtils.toFractionalResource(
+          tDesc.resources.get(GPU).get((index - 4).toString)) == 0.3)
+      }
+      index += 1
+    }
+  }
+
+  test("SPARK-45527 TaskResourceProfile: the left gpu resources on multiple executors " +
+    "can assign to other taskset") {
+    val taskCpus = 1
+    val taskGpus = 0.3
+    val executorGpus = 4
+    val executorCpus = 1000

Review Comment:
   1000 threads seem to be too large for some CI systems with a limited resource.
   - https://github.com/apache/spark/actions/workflows/build_maven_java21_macos14.yml
     - https://github.com/apache/spark/actions/runs/8054862135/job/22000403549
   ```
   Warning: [766.327s][warning][os,thread] Failed to start thread "Unknown thread" - pthread_create failed (EAGAIN) for attributes: stacksize: 4096k, guardsize: 16k, detached.
   Warning: [766.327s][warning][os,thread] Failed to start the native thread for java.lang.Thread "dispatcher-event-loop-840"
   *** RUN ABORTED ***
   An exception or error caused a run to abort: unable to create native thread: possibly out of memory or process/resource limits reached 
     java.lang.OutOfMemoryError: unable to create native thread: possibly out of memory or process/resource limits reached
   ```
   
   I made a test-case follow-up.
   - https://github.com/apache/spark/pull/45264



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on PR #43494:
URL: https://github.com/apache/spark/pull/43494#issuecomment-1776398670

   The pipelines of both "Run docker integration test" and "Linters, License, dependencies" keep failing. But seems the code-related pipelines got passed. I will check it.


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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1375878289


##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.

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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1375821367


##########
core/src/main/scala/org/apache/spark/resource/ResourceAllocator.scala:
##########
@@ -29,59 +65,45 @@ private[spark] trait ResourceAllocator {
 
   protected def resourceName: String
   protected def resourceAddresses: Seq[String]
-  protected def slotsPerAddress: Int
 
   /**
-   * Map from an address to its availability, a value > 0 means the address is available,
-   * while value of 0 means the address is fully assigned.
-   *
-   * For task resources ([[org.apache.spark.scheduler.ExecutorResourceInfo]]), this value
-   * can be a multiple, such that each address can be allocated up to [[slotsPerAddress]]
-   * times.
+   * Map from an address to its availability default to RESOURCE_TOTAL_AMOUNT, a value > 0 means
+   * the address is available, while value of 0 means the address is fully assigned.
    */
-  private lazy val addressAvailabilityMap = {
-    mutable.HashMap(resourceAddresses.map(_ -> slotsPerAddress): _*)
+  protected lazy val addressAvailabilityMap = {
+    mutable.HashMap(resourceAddresses.map(address => address -> RESOURCE_TOTAL_AMOUNT): _*)
   }
 
   /**
-   * Sequence of currently available resource addresses.
-   *
-   * With [[slotsPerAddress]] greater than 1, [[availableAddrs]] can contain duplicate addresses
-   * e.g. with [[slotsPerAddress]] == 2, availableAddrs for addresses 0 and 1 can look like
-   * Seq("0", "0", "1"), where address 0 has two assignments available, and 1 has one.
+   * Sequence of currently available resource addresses which is not fully assigned.
    */
   def availableAddrs: Seq[String] = addressAvailabilityMap
-    .flatMap { case (addr, available) =>
-      (0 until available).map(_ => addr)
-    }.toSeq.sorted
+      .filter(addresses => addresses._2 > 0).keys.toSeq.sorted
 
   /**
    * Sequence of currently assigned resource addresses.

Review Comment:
   Yeah, You're right. The UI doesn't have this detail yet.



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1376121015


##########
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala:
##########
@@ -389,7 +389,7 @@ private[spark] class TaskSchedulerImpl(
       maxLocality: TaskLocality,
       shuffledOffers: Seq[WorkerOffer],
       availableCpus: Array[Int],
-      availableResources: Array[Map[String, Buffer[String]]],
+      availableResources: Array[ExecutorResourcesAmounts],

Review Comment:
   The resources in WorkerOffer https://github.com/apache/spark/pull/43494/files#diff-942455d6c015aaa273f66a9d8ea00297d7c05e45e83a0761783c68ad134b1e87R33 has been changed to ExecutorResourcesAmounts



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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1378463033


##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  // Acquire the resource and update the resource
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+        // scalastyle:off println

Review Comment:
   Removed. Done



##########
core/src/main/scala/org/apache/spark/scheduler/ExecutorResourcesAmounts.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
+import org.apache.spark.resource.ResourceAmountUtils.RESOURCE_TOTAL_AMOUNT
+
+/**
+ * Class to hold information about a series of resources belonging to an executor.
+ * A resource could be a GPU, FPGA, etc. And it is used as a temporary
+ * class to calculate the resources amounts when offering resources to
+ * the tasks in the [[TaskSchedulerImpl]]
+ *
+ * One example is GPUs, where the addresses would be the indices of the GPUs
+ *
+ * @param resources The executor available resources and amount. eg,
+ *                  Map("gpu" -> mutable.Map("0" -> 0.2, "1" -> 1.0),
+ *                  "fpga" -> mutable.Map("a" -> 0.3, "b" -> 0.9)
+ *                  )
+ */
+private[spark] class ExecutorResourcesAmounts(
+    private val resources: Map[String, Map[String, Double]]) extends Serializable {
+
+  resources.foreach { case (_, addressMount) =>
+    addressMount.foreach { case (_, amount) => assert(amount <= 1.0)}}
+
+  // multiply the RESOURCE_TOTAL_AMOUNT to avoid using double directly.
+  // and convert the addressesAmounts to be mutable.HashMap
+  private val internalResources: Map[String, HashMap[String, Long]] = {
+    resources.map { case (rName, addressAmounts) =>
+      rName -> HashMap(addressAmounts.map { case (address, amount) =>
+        address -> (amount * RESOURCE_TOTAL_AMOUNT).toLong
+      }.toSeq: _*)
+    }
+  }
+
+  // It maps from the resource name to its amount.
+  lazy val resourceAmount: Map[String, Int] = internalResources.map { case (rName, addressMap) =>
+    rName -> addressMap.size
+  }
+
+  // convert internal resources back to the public.
+  def availableResources: Map[String, Map[String, Double]] = {
+    internalResources.map { case (rName, addressMap) =>
+      rName -> addressMap.map { case (address, amount) =>
+        address -> amount.toDouble / RESOURCE_TOTAL_AMOUNT
+      }.toMap
+    }
+  }
+
+  // Acquire the resource and update the resource
+  def acquire(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to acquire an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to acquire an address that doesn't exist. $rName " +
+            s"address $address doesn't exist."))
+
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalLeft = prevInternalTotalAmount - internalTaskAmount
+        val realLeft = internalLeft.toDouble / RESOURCE_TOTAL_AMOUNT
+        if (realLeft < 0) {
+          throw new SparkException(s"The total amount ${realLeft} " +
+            s"after acquiring $rName address $address should be >= 0")
+        }
+        internalResources(rName)(address) = internalLeft
+        // scalastyle:off println
+        println(s"Acquired. left ${realLeft}")
+        // scalastyle:on println
+      }
+    }
+  }
+
+  // release the resources and update the values
+  def release(assignedResource: Map[String, Map[String, Double]]): Unit = {
+    assignedResource.foreach { case (rName, taskResAmounts) =>
+      val availableResourceAmounts = internalResources.getOrElse(rName,
+        throw new SparkException(s"Try to release an address from $rName that doesn't exist"))
+      taskResAmounts.foreach { case (address, amount) =>
+        val prevInternalTotalAmount = availableResourceAmounts.getOrElse(address,
+          throw new SparkException(s"Try to release an address that is not assigned. $rName " +
+            s"address $address is not assigned."))
+        val internalTaskAmount = (amount * RESOURCE_TOTAL_AMOUNT).toLong
+        val internalTotal = prevInternalTotalAmount + internalTaskAmount
+        if (internalTotal > RESOURCE_TOTAL_AMOUNT) {
+          throw new SparkException(s"The total amount " +
+            s"${internalTotal.toDouble / RESOURCE_TOTAL_AMOUNT} " +
+            s"after releasing $rName address $address should be <= 1.0")
+        }
+        internalResources(rName)(address) = internalTotal
+        // scalastyle:off println

Review Comment:
   Removed. 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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1383223189


##########
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala:
##########
@@ -165,15 +165,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
     }
 
     override def receive: PartialFunction[Any, Unit] = {
-      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) =>
+      case StatusUpdate(executorId, taskId, state, data, taskCpus, resources, resourcesAmounts) =>

Review Comment:
   Done. Removed all the duplicated un-used resource



##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -273,7 +273,8 @@ private[spark] class CoarseGrainedExecutorBackend(
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = {
     val resources = taskResources.getOrDefault(taskId, Map.empty[String, ResourceInformation])

Review Comment:
   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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1390665985


##########
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:
   I added the check in the ResourceProfile which requires the task.amount to be in (0, 0.5] or a whole number, while in the TaskResourceProfile which requires to be (0, 1.0] or a whole number



##########
core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala:
##########
@@ -37,8 +37,8 @@ import org.apache.spark.annotation.{Evolving, Since}
 class TaskResourceRequest(val resourceName: String, val amount: Double)
   extends Serializable {
 
-  assert(amount <= 0.5 || amount % 1 == 0,
-    s"The resource amount ${amount} must be either <= 0.5, or a whole number.")
+  assert(amount <= 1.0 || amount % 1 == 0,

Review Comment:
   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-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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1395199364


##########
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:
   Really good suggestion. 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-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


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

Posted by "tgravescs (via GitHub)" <gi...@apache.org>.
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


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

Posted by "wbo4958 (via GitHub)" <gi...@apache.org>.
wbo4958 commented on code in PR #43494:
URL: https://github.com/apache/spark/pull/43494#discussion_r1432087192


##########
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala:
##########
@@ -192,7 +181,9 @@ 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) =>

Review Comment:
   Good catch.



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