[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-21 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r903254945


##
core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala:
##
@@ -111,15 +112,19 @@ private[spark] class StandaloneSchedulerBackend(
 // ExecutorAllocationManager will send the real initial limit to the 
Master later.
 val initialExecutorLimit =
   if (Utils.isDynamicAllocationEnabled(conf)) {
+if (coresPerExecutor.isEmpty) {
+  logWarning("Dynamic allocation without explicitly setting 
spark.executor.cores " +
+"detected, you may get more executors allocated than expected. 
It's recommended to " +

Review Comment:
   nit: "Dynamic allocation enabled without spark.executor.cores explicitly 
set, you may ..."



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-21 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r903254447


##
core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala:
##
@@ -111,15 +112,19 @@ private[spark] class StandaloneSchedulerBackend(
 // ExecutorAllocationManager will send the real initial limit to the 
Master later.
 val initialExecutorLimit =
   if (Utils.isDynamicAllocationEnabled(conf)) {
+if (coresPerExecutor.isEmpty) {
+  logWarning("Dynamic allocation without explicitly setting 
spark.executor.cores " +
+"detected, you may get more executors allocated than expected. 
It's recommended to " +
+"set spark.executor.cores explicitly. Check this issue for more 
details: " +
+"https://issues.apache.org/jira/browse/SPARK-30299;)

Review Comment:
   nit: "Please check 
[SPARK-30299](https://issues.apache.org/jira/browse/SPARK-30299) for more 
details."



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-21 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r903253826


##
core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala:
##
@@ -63,17 +64,28 @@ private[spark] class ResourceProfileManager(sparkConf: 
SparkConf,
*/
   private[spark] def isSupported(rp: ResourceProfile): Boolean = {
 val isNotDefaultProfile = rp.id != 
ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
-val notYarnOrK8sAndNotDefaultProfile = isNotDefaultProfile && !(isYarn || 
isK8s)
-val YarnOrK8sNotDynAllocAndNotDefaultProfile =
-  isNotDefaultProfile && (isYarn || isK8s) && !dynamicEnabled
+val notYarnOrK8sOrStandaloneAndNotDefaultProfile =
+  isNotDefaultProfile && !(isYarn || isK8s || isStandalone)
+val YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile =
+  isNotDefaultProfile && (isYarn || isK8s || isStandalone) && 
!dynamicEnabled
 // We want the exception to be thrown only when we are specifically 
testing for the
 // exception or in a real application. Otherwise in all other testing 
scenarios we want
 // to skip throwing the exception so that we can test in other modes to 
make testing easier.
 if ((notRunningUnitTests || testExceptionThrown) &&
-(notYarnOrK8sAndNotDefaultProfile || 
YarnOrK8sNotDynAllocAndNotDefaultProfile)) {
+(notYarnOrK8sOrStandaloneAndNotDefaultProfile ||
+  YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile)) {
   throw new SparkException("ResourceProfiles are only supported on YARN 
and Kubernetes " +
-"with dynamic allocation enabled.")
+"and Standalone with dynamic allocation enabled.")
 }
+
+if (isStandalone && rp.getExecutorCores.isEmpty &&
+  sparkConf.getOption(config.EXECUTOR_CORES.key).isEmpty) {
+  logWarning(s"Executor cores is not set for resource profile: ${rp.id}, 
and " +
+s"spark.executor.cores is also not specified, you may get more 
executors allocated than " +
+s"expected. It's recommended to set executor cores explicitly. Check 
this issue " +

Review Comment:
   nit: "Neither executor cores is set for resource profile, nor 
spark.executor.cores is explicitly set, you may ..."



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-21 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r903252766


##
core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala:
##
@@ -63,17 +64,28 @@ private[spark] class ResourceProfileManager(sparkConf: 
SparkConf,
*/
   private[spark] def isSupported(rp: ResourceProfile): Boolean = {
 val isNotDefaultProfile = rp.id != 
ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
-val notYarnOrK8sAndNotDefaultProfile = isNotDefaultProfile && !(isYarn || 
isK8s)
-val YarnOrK8sNotDynAllocAndNotDefaultProfile =
-  isNotDefaultProfile && (isYarn || isK8s) && !dynamicEnabled
+val notYarnOrK8sOrStandaloneAndNotDefaultProfile =
+  isNotDefaultProfile && !(isYarn || isK8s || isStandalone)
+val YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile =
+  isNotDefaultProfile && (isYarn || isK8s || isStandalone) && 
!dynamicEnabled
 // We want the exception to be thrown only when we are specifically 
testing for the
 // exception or in a real application. Otherwise in all other testing 
scenarios we want
 // to skip throwing the exception so that we can test in other modes to 
make testing easier.
 if ((notRunningUnitTests || testExceptionThrown) &&
-(notYarnOrK8sAndNotDefaultProfile || 
YarnOrK8sNotDynAllocAndNotDefaultProfile)) {
+(notYarnOrK8sOrStandaloneAndNotDefaultProfile ||
+  YarnOrK8sOrStandaloneNotDynAllocAndNotDefaultProfile)) {
   throw new SparkException("ResourceProfiles are only supported on YARN 
and Kubernetes " +
-"with dynamic allocation enabled.")
+"and Standalone with dynamic allocation enabled.")
 }
+
+if (isStandalone && rp.getExecutorCores.isEmpty &&
+  sparkConf.getOption(config.EXECUTOR_CORES.key).isEmpty) {
+  logWarning(s"Executor cores is not set for resource profile: ${rp.id}, 
and " +
+s"spark.executor.cores is also not specified, you may get more 
executors allocated than " +
+s"expected. It's recommended to set executor cores explicitly. Check 
this issue " +
+s"for more details: https://issues.apache.org/jira/browse/SPARK-30299;)

Review Comment:
   nit: "Please check SPARK-30299 for more details."



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-20 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r902140262


##
docs/job-scheduling.md:
##
@@ -83,6 +83,10 @@ This feature is disabled by default and available on all 
coarse-grained cluster
 [Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes) and [K8s 
mode](running-on-kubernetes.html).
 
 
+### Caveats
+
+- In [standalone mode](spark-standalone.html), without explicitly setting 
cores for each executor, executors will get all the cores of a worker. In this 
case, when dynamic allocation enabled, spark will possibly acquire much more 
executors than expected. When you want to use dynamic allocation in [standalone 
mode](spark-standalone.html), you are recommended to explicitly set cores for 
each executor before the issue 
[SPARK-30299](https://issues.apache.org/jira/browse/SPARK-30299) got fixed.

Review Comment:
   "...without explicitly setting `spark.executor.cores`..."



##
docs/job-scheduling.md:
##
@@ -83,6 +83,10 @@ This feature is disabled by default and available on all 
coarse-grained cluster
 [Mesos coarse-grained mode](running-on-mesos.html#mesos-run-modes) and [K8s 
mode](running-on-kubernetes.html).
 
 
+### Caveats
+
+- In [standalone mode](spark-standalone.html), without explicitly setting 
cores for each executor, executors will get all the cores of a worker. In this 
case, when dynamic allocation enabled, spark will possibly acquire much more 
executors than expected. When you want to use dynamic allocation in [standalone 
mode](spark-standalone.html), you are recommended to explicitly set cores for 
each executor before the issue 
[SPARK-30299](https://issues.apache.org/jira/browse/SPARK-30299) got fixed.

Review Comment:
   I didn't see where this case is warned in the code. Could you add 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-unsubscr...@spark.apache.org

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-09 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r893599247


##
core/src/main/scala/org/apache/spark/deploy/master/Master.scala:
##
@@ -725,26 +729,38 @@ private[deploy] class Master(
*/
   private def startExecutorsOnWorkers(): Unit = {
 // Right now this is a very simple FIFO scheduler. We keep trying to fit 
in the first app
-// in the queue, then the second app, etc.
+// in the queue, then the second app, etc. And for each app, we will 
schedule base on
+// resource profiles also with a simple FIFO scheduler, resource profile 
with smaller id
+// first.

Review Comment:
   > Currently, we don't have a good way to infer about the order of requests 
for different resource profiles.
   
   I actually means the order of receiving the request in Master, although I 
know it could be out of order compared to the request sender (driver) due to 
asynchronous RPC framework. But after a second thinking, requests come from the 
pending tasks, which are able to be scheduled in parallel as long as there're 
enough resources. So it doesn't really matter which resource profile should be 
used to launch executors. Schedule by ordered resource profile ids should be 
enough.



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-09 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r893583053


##
core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala:
##
@@ -19,23 +19,28 @@ package org.apache.spark.deploy
 
 import java.net.URI
 
-import org.apache.spark.resource.ResourceRequirement
+import org.apache.spark.resource.{ResourceProfile, ResourceRequirement, 
ResourceUtils}
+import org.apache.spark.resource.ResourceProfile.getCustomExecutorResources
 
 private[spark] case class ApplicationDescription(
 name: String,
 maxCores: Option[Int],
-memoryPerExecutorMB: Int,
 command: Command,
 appUiUrl: String,
+defaultProfile: ResourceProfile,
 eventLogDir: Option[URI] = None,
 // short name of compression codec used when writing event logs, if any 
(e.g. lzf)
 eventLogCodec: Option[String] = None,
-coresPerExecutor: Option[Int] = None,
 // number of executors this application wants to start with,
 // only used if dynamic allocation is enabled
 initialExecutorLimit: Option[Int] = None,
-user: String = System.getProperty("user.name", ""),
-resourceReqsPerExecutor: Seq[ResourceRequirement] = Seq.empty) {
+user: String = System.getProperty("user.name", "")) {
+
+  def memoryPerExecutorMB: Int = 
defaultProfile.getExecutorMemory.map(_.toInt).getOrElse(1024)
+  def coresPerExecutor: Option[Int] = defaultProfile.getExecutorCores
+  def resourceReqsPerExecutor: Seq[ResourceRequirement] =
+ResourceUtils.executorResourceRequestToRequirement(
+  
getCustomExecutorResources(defaultProfile).values.toSeq.sortBy(_.resourceName))

Review Comment:
   Make sense. So let's make all of them be consistent? Also sort it in 
`ApplicationInfo`?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r891178775


##
core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala:
##
@@ -530,6 +535,87 @@ class MasterSuite extends SparkFunSuite
 schedulingWithEverything(spreadOut = false)
   }
 
+  test("scheduling for app with multiple resource profiles") {
+scheduleExecutorsForAppWithMultiRPs(withMaxCores = false)
+  }
+
+  test("scheduling for app with multiple resource profiles with max cores") {
+scheduleExecutorsForAppWithMultiRPs(withMaxCores = true)
+  }
+
+  private def scheduleExecutorsForAppWithMultiRPs(withMaxCores: Boolean): Unit 
= {
+val appInfo: ApplicationInfo = if (withMaxCores) {
+  makeAppInfo(
+  1024, maxCores = Some(30), initialExecutorLimit = Some(0))
+} else {
+  makeAppInfo(
+1024, maxCores = None, initialExecutorLimit = Some(0))
+}
+
+val master = makeAliveMaster()
+val conf = new SparkConf()
+val workers = (1 to 4).map { idx =>
+  val worker = new MockWorker(master.self, conf)
+  worker.rpcEnv.setupEndpoint(s"worker-$idx", worker)
+  val workerReg = RegisterWorker(
+worker.id,
+"localhost",
+worker.self.address.port,
+worker.self,
+10,
+4096,
+"http://localhost:8080;,
+RpcAddress("localhost", 1))
+  master.self.send(workerReg)
+  worker
+}
+
+// Register app and schedule.
+master.registerApplication(appInfo)
+startExecutorsOnWorkers(master)
+assert(appInfo.executors.isEmpty)
+
+// Request executors with multiple resource profile.
+val rp1 = DeployTestUtils.createResourceProfile(Some(2048), Map.empty, 
Some(5))
+val rp2 = DeployTestUtils.createResourceProfile(Some(2048), Map.empty, 
Some(10))

Review Comment:
   Could you also test the case where no worker can satisfy the resource 
profile ? In this case, no executor for that rp should be launched.



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r891171201


##
core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala:
##
@@ -530,6 +535,87 @@ class MasterSuite extends SparkFunSuite
 schedulingWithEverything(spreadOut = false)
   }
 
+  test("scheduling for app with multiple resource profiles") {
+scheduleExecutorsForAppWithMultiRPs(withMaxCores = false)
+  }
+
+  test("scheduling for app with multiple resource profiles with max cores") {
+scheduleExecutorsForAppWithMultiRPs(withMaxCores = true)
+  }
+
+  private def scheduleExecutorsForAppWithMultiRPs(withMaxCores: Boolean): Unit 
= {
+val appInfo: ApplicationInfo = if (withMaxCores) {
+  makeAppInfo(
+  1024, maxCores = Some(30), initialExecutorLimit = Some(0))

Review Comment:
   nit: 2 indents?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r891166646


##
core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala:
##
@@ -299,9 +300,10 @@ private[spark] class StandaloneAppClient(
*
* @return whether the request is acknowledged.
*/
-  def requestTotalExecutors(requestedTotal: Int): Future[Boolean] = {

Review Comment:
   How about leaving this method as it is and delagtes to 
`requestTotalExecutors(Map(default resource profile -> requestedTotal))` 
instead? So that some tests remains unchanged.



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r891129088


##
core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala:
##
@@ -107,11 +107,11 @@ object JsonConstants {
   |{"id":"id","starttime":3,"name":"name",
   |"cores":0,"user":"%s",
   |"memoryperexecutor":1234,
-  |"resourcesperexecutor":[{"name":"gpu",
-  |"amount":3},{"name":"fpga","amount":3}],
+  |"resourcesperexecutor":[{"name":"fpga",
+  |"amount":3},{"name":"gpu","amount":3}],

Review Comment:
   Why the order changes?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r890982027


##
core/src/main/scala/org/apache/spark/deploy/master/Master.scala:
##
@@ -725,26 +729,38 @@ private[deploy] class Master(
*/
   private def startExecutorsOnWorkers(): Unit = {
 // Right now this is a very simple FIFO scheduler. We keep trying to fit 
in the first app
-// in the queue, then the second app, etc.
+// in the queue, then the second app, etc. And for each app, we will 
schedule base on
+// resource profiles also with a simple FIFO scheduler, resource profile 
with smaller id
+// first.

Review Comment:
   I'd suggest to schedule in the order of the resource profile reuqests 
instead of the smaller id first. In the case of the resource profile is resued 
for later on RDD computation, the samller id doesn't seem to has the priority 
over other resource profiles. WDYT?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r890973724


##
core/src/main/scala/org/apache/spark/deploy/master/ExecutorResourceDescription.scala:
##
@@ -0,0 +1,32 @@
+/*
+ * 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.deploy.master
+
+import org.apache.spark.resource.ResourceRequirement
+
+/**
+ * Describe resource requests for different resource profiles. Used for 
executor schedule.
+ *
+ * @param coresPerExecutor cores for each executor.
+ * @param memoryMbPerExecutor memory for each executor.
+ * @param customResourcesPerExecutor custom resource requests for each 
executor.

Review Comment:
   nit: "resource requests" -> "resource requirements"
   
   (I think we also have `ExecutorResourceRequest` so it's good to distugish 
them carefully.)



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r890949860


##
core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala:
##
@@ -65,7 +66,70 @@ private[spark] class ApplicationInfo(
 appSource = new ApplicationSource(this)
 nextExecutorId = 0
 removedExecutors = new ArrayBuffer[ExecutorDesc]
-executorLimit = desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE)
+val initialExecutorLimit = 
desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE)
+
+rpIdToResourceProfile = new mutable.HashMap[Int, ResourceProfile]()
+rpIdToResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) = desc.defaultProfile
+rpIdToResourceDesc = new mutable.HashMap[Int, 
ExecutorResourceDescription]()
+createResourceDescForResourceProfile(desc.defaultProfile)
+
+targetNumExecutorsPerResourceProfileId = new mutable.HashMap[Int, Int]()
+targetNumExecutorsPerResourceProfileId(DEFAULT_RESOURCE_PROFILE_ID) = 
initialExecutorLimit
+
+executorsPerResourceProfileId = new mutable.HashMap[Int, 
mutable.Set[Int]]()
+  }
+
+  private[deploy] def getOrUpdateExecutorsForRPId(rpId: Int): mutable.Set[Int] 
= {
+executorsPerResourceProfileId.getOrElseUpdate(rpId, mutable.HashSet[Int]())
+  }
+
+  private[deploy] def getTargetExecutorNumForRPId(rpId: Int): Int = {
+targetNumExecutorsPerResourceProfileId.getOrElse(rpId, 0)
+  }
+
+  private[deploy] def getRequestedRPIds(): Seq[Int] = {
+rpIdToResourceProfile.keys.toSeq.sorted
+  }
+
+  private def createResourceDescForResourceProfile(resourceProfile: 
ResourceProfile): Unit = {
+if (!rpIdToResourceDesc.contains(resourceProfile.id)) {
+  val defaultMemoryMbPerExecutor = desc.memoryPerExecutorMB
+  val defaultCoresPerExecutor = desc.coresPerExecutor
+  val coresPerExecutor = resourceProfile.getExecutorCores
+.orElse(defaultCoresPerExecutor)
+  val memoryMbPerExecutor = resourceProfile.getExecutorMemory
+.map(_.toInt)
+.getOrElse(defaultMemoryMbPerExecutor)
+  val customResources = ResourceUtils.executorResourceRequestToRequirement(
+getCustomExecutorResources(resourceProfile).values.toSeq)
+
+  rpIdToResourceDesc(resourceProfile.id) =
+ExecutorResourceDescription(coresPerExecutor, memoryMbPerExecutor, 
customResources)
+}
+  }
+
+  // Get resources required for schedule.
+  private[deploy] def getResourceDescriptionForRpId(rpId: Int): 
ExecutorResourceDescription = {
+rpIdToResourceDesc(rpId)
+  }
+
+  private[deploy] def requestExecutors(
+  resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Unit = {
+resourceProfileToTotalExecs.foreach { case (rp, num) =>
+  createResourceDescForResourceProfile(rp)
+
+  if (!rpIdToResourceProfile.contains(rp.id)) {
+rpIdToResourceProfile(rp.id) = rp
+  }
+
+  if (!targetNumExecutorsPerResourceProfileId.get(rp.id).contains(num)) {
+targetNumExecutorsPerResourceProfileId(rp.id) = num
+  }

Review Comment:
   How about:
   ```suggestion
 targetNumExecutorsPerResourceProfileId(rp.id) = num
   ```



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r890938856


##
core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala:
##
@@ -25,10 +25,13 @@ package org.apache.spark.deploy
 private[deploy] class ExecutorDescription(
 val appId: String,
 val execId: Int,
+val rpId: Int,
 val cores: Int,
+val memoryMb: Int,

Review Comment:
   > And in master, we can only reconstruct the resource profile information in 
ApplicationInfo after client send resource request RequestExecutors
   
   Does it mean we can't launch new executors with the specific `rpId` until 
the client sends the request with the corresponding resource profile? For 
example, if the number of executos with the specific `rpId` hasn't reached the 
targer number, it seems like we can't schedule new executors for it until we 
know resource profile by `RequestExecutors`, right?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-07 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r890912848


##
core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala:
##
@@ -19,23 +19,28 @@ package org.apache.spark.deploy
 
 import java.net.URI
 
-import org.apache.spark.resource.ResourceRequirement
+import org.apache.spark.resource.{ResourceProfile, ResourceRequirement, 
ResourceUtils}
+import org.apache.spark.resource.ResourceProfile.getCustomExecutorResources
 
 private[spark] case class ApplicationDescription(
 name: String,
 maxCores: Option[Int],
-memoryPerExecutorMB: Int,
 command: Command,
 appUiUrl: String,
+defaultProfile: ResourceProfile,
 eventLogDir: Option[URI] = None,
 // short name of compression codec used when writing event logs, if any 
(e.g. lzf)
 eventLogCodec: Option[String] = None,
-coresPerExecutor: Option[Int] = None,
 // number of executors this application wants to start with,
 // only used if dynamic allocation is enabled
 initialExecutorLimit: Option[Int] = None,
-user: String = System.getProperty("user.name", ""),
-resourceReqsPerExecutor: Seq[ResourceRequirement] = Seq.empty) {
+user: String = System.getProperty("user.name", "")) {
+
+  def memoryPerExecutorMB: Int = 
defaultProfile.getExecutorMemory.map(_.toInt).getOrElse(1024)
+  def coresPerExecutor: Option[Int] = defaultProfile.getExecutorCores
+  def resourceReqsPerExecutor: Seq[ResourceRequirement] =
+ResourceUtils.executorResourceRequestToRequirement(
+  
getCustomExecutorResources(defaultProfile).values.toSeq.sortBy(_.resourceName))

Review Comment:
   Is `sortBy(_.resourceName)` necessary? I didn't see we sort it in 
`ApplicationInfo`.



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-06-01 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r886345400


##
core/src/main/scala/org/apache/spark/deploy/master/ResourceDescription.scala:
##
@@ -0,0 +1,32 @@
+/*
+ * 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.deploy.master
+
+import org.apache.spark.resource.ResourceRequirement
+
+/**
+ * Describe resource requests for different resource profiles. Used for 
executor schedule.
+ *
+ * @param coresPerExecutor cores for each executor.
+ * @param memoryMbPerExecutor memory for each executor.
+ * @param customResourcesPerExecutor custom resource requests for each 
executor.
+ */
+private[spark] case class ResourceDescription(

Review Comment:
   Had another look around `ExecutorResourcesOrDefaults`, it looks like it's a 
general abstraction of executor resources that is shared by various cluster 
managers. So I think it makes sense to extract a specific resource description 
separately for the Standalone itself. 



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-05-31 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r885557272


##
core/src/main/scala/org/apache/spark/deploy/master/ResourceDescription.scala:
##
@@ -0,0 +1,32 @@
+/*
+ * 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.deploy.master
+
+import org.apache.spark.resource.ResourceRequirement
+
+/**
+ * Describe resource requests for different resource profiles. Used for 
executor schedule.
+ *
+ * @param coresPerExecutor cores for each executor.
+ * @param memoryMbPerExecutor memory for each executor.
+ * @param customResourcesPerExecutor custom resource requests for each 
executor.
+ */
+private[spark] case class ResourceDescription(

Review Comment:
   Shall we reuse `ExecutorResourcesOrDefaults` to replace 
`ResourceDescription`?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-05-31 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r885377381


##
core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala:
##
@@ -43,8 +43,8 @@ private[ui] class ApplicationPage(parent: MasterWebUI) 
extends WebUIPage("app")
   return UIUtils.basicSparkPage(request, msg, "Not Found")
 }
 
-val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", 
"Resources",
-  "State", "Logs")
+val executorHeaders = Seq("ExecutorID", "Worker", "Resource Profile Id", 
"Cores", "Memory",
+  "Resources", "State", "Logs")

Review Comment:
   Could you leave the positions of "Cores" and "Memory" unchanged and put 
"Resource Profile Id" together with "Resources"?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-05-31 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r885358948


##
core/src/main/scala/org/apache/spark/deploy/master/ResourceDescription.scala:
##
@@ -0,0 +1,32 @@
+/*
+ * 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.deploy.master
+
+import org.apache.spark.resource.ResourceRequirement
+
+/**
+ * Describe resource requests for different resource profiles. Used for 
executor schedule.
+ *
+ * @param coresPerExecutor cores for each executor.
+ * @param memoryMbPerExecutor memory for each executor.
+ * @param customResourcesPerExecutor custom resource requests for each 
executor.
+ */
+private[spark] case class ResourceDescription(

Review Comment:
   How about `ExecutorResourceDescription`?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-05-31 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r885313120


##
core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala:
##
@@ -65,7 +66,70 @@ private[spark] class ApplicationInfo(
 appSource = new ApplicationSource(this)
 nextExecutorId = 0
 removedExecutors = new ArrayBuffer[ExecutorDesc]
-executorLimit = desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE)
+val initialExecutorLimit = 
desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE)
+
+rpIdToResourceProfile = new mutable.HashMap[Int, ResourceProfile]()
+rpIdToResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) = desc.defaultProfile
+rpIdToResourceDesc = new mutable.HashMap[Int, ResourceDescription]()
+createResourceDescForResourceProfile(desc.defaultProfile)
+
+targetNumExecutorsPerResourceProfileId = new mutable.HashMap[Int, Int]()
+targetNumExecutorsPerResourceProfileId(DEFAULT_RESOURCE_PROFILE_ID) = 
initialExecutorLimit

Review Comment:
   I think the original `executorLimit` limits the total executor number of the 
whole application. But now it looks like `initialExecutorLimit` only limits the 
executor number of the `DEFAULT_RESOURCE_PROFILE_ID` type. How about other 
resource profile types?



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

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

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


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



[GitHub] [spark] Ngone51 commented on a diff in pull request #36716: [SPARK-39062][CORE] Add stage level resource scheduling support for standalone cluster

2022-05-31 Thread GitBox


Ngone51 commented on code in PR #36716:
URL: https://github.com/apache/spark/pull/36716#discussion_r885297634


##
core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala:
##
@@ -25,10 +25,13 @@ package org.apache.spark.deploy
 private[deploy] class ExecutorDescription(
 val appId: String,
 val execId: Int,
+val rpId: Int,
 val cores: Int,
+val memoryMb: Int,

Review Comment:
   Why do we need `memoryMb` now?



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

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

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


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