Skip to content

Commit cbad616

Browse files
Ngone51tgravescs
authored andcommitted
[SPARK-27371][CORE] Support GPU-aware resources scheduling in Standalone
## What changes were proposed in this pull request? In this PR, we implements a complete process of GPU-aware resources scheduling in Standalone. The whole process looks like: Worker sets up isolated resources when it starts up and registers to master along with its resources. And, Master picks up usable workers according to driver/executor's resource requirements to launch driver/executor on them. Then, Worker launches the driver/executor after preparing resources file, which is created under driver/executor's working directory, with specified resource addresses(told by master). When driver/executor finished, their resources could be recycled to worker. Finally, if a worker stops, it should always release its resources firstly. For the case of Workers and Drivers in **client** mode run on the same host, we introduce a config option named `spark.resources.coordinate.enable`(default true) to indicate whether Spark should coordinate resources for user. If `spark.resources.coordinate.enable=false`, user should be responsible for configuring different resources for Workers and Drivers when use resourcesFile or discovery script. If true, Spark would help user to assign different resources for Workers and Drivers. The solution for Spark to coordinate resources among Workers and Drivers is: Generally, use a shared file named *____allocated_resources____.json* to sync allocated resources info among Workers and Drivers on the same host. After a Worker or Driver found all resources using the configured resourcesFile and/or discovery script during launching, it should filter out available resources by excluding resources already allocated in *____allocated_resources____.json* and acquire resources from available resources according to its own requirement. After that, it should write its allocated resources along with its process id (pid) into *____allocated_resources____.json*. Pid (proposed by tgravescs) here used to check whether the allocated resources are still valid in case of Worker or Driver crashes and doesn't release resources properly. And when a Worker or Driver finished, normally, it would always clean up its own allocated resources in *____allocated_resources____.json*. Note that we'll always get a file lock before any access to file *____allocated_resources____.json* and release the lock finally. Futhermore, we appended resources info in `WorkerSchedulerStateResponse` to work around master change behaviour in HA mode. ## How was this patch tested? Added unit tests in WorkerSuite, MasterSuite, SparkContextSuite. Manually tested with client/cluster mode (e.g. multiple workers) in a single node Standalone. Closes apache#25047 from Ngone51/SPARK-27371. Authored-by: wuyi <[email protected]> Signed-off-by: Thomas Graves <[email protected]>
1 parent 5159876 commit cbad616

38 files changed

+1217
-228
lines changed

.gitignore

+1
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,7 @@ scalastyle-on-compile.generated.xml
7171
scalastyle-output.xml
7272
scalastyle.txt
7373
spark-*-bin-*.tgz
74+
spark-resources/
7475
spark-tests.log
7576
src_managed/
7677
streaming-tests.log

core/src/main/scala/org/apache/spark/SparkContext.scala

+30-4
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
4242
import org.apache.spark.annotation.DeveloperApi
4343
import org.apache.spark.broadcast.Broadcast
4444
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
45+
import org.apache.spark.deploy.StandaloneResourceUtils._
4546
import org.apache.spark.executor.ExecutorMetrics
4647
import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat}
4748
import org.apache.spark.internal.Logging
@@ -245,6 +246,15 @@ class SparkContext(config: SparkConf) extends Logging {
245246

246247
def isLocal: Boolean = Utils.isLocalMaster(_conf)
247248

249+
private def isClientStandalone: Boolean = {
250+
val isSparkCluster = master match {
251+
case SparkMasterRegex.SPARK_REGEX(_) => true
252+
case SparkMasterRegex.LOCAL_CLUSTER_REGEX(_, _, _) => true
253+
case _ => false
254+
}
255+
deployMode == "client" && isSparkCluster
256+
}
257+
248258
/**
249259
* @return true if context is stopped or in the midst of stopping.
250260
*/
@@ -380,7 +390,18 @@ class SparkContext(config: SparkConf) extends Logging {
380390
_driverLogger = DriverLogger(_conf)
381391

382392
val resourcesFileOpt = conf.get(DRIVER_RESOURCES_FILE)
383-
_resources = getOrDiscoverAllResources(_conf, SPARK_DRIVER_PREFIX, resourcesFileOpt)
393+
val allResources = getOrDiscoverAllResources(_conf, SPARK_DRIVER_PREFIX, resourcesFileOpt)
394+
_resources = {
395+
// driver submitted in client mode under Standalone may have conflicting resources with
396+
// other drivers/workers on this host. We should sync driver's resources info into
397+
// SPARK_RESOURCES/SPARK_RESOURCES_COORDINATE_DIR/ to avoid collision.
398+
if (isClientStandalone) {
399+
acquireResources(_conf, SPARK_DRIVER_PREFIX, allResources, Utils.getProcessId)
400+
} else {
401+
allResources
402+
}
403+
}
404+
logResourceInfo(SPARK_DRIVER_PREFIX, _resources)
384405

385406
// log out spark.app.name in the Spark driver logs
386407
logInfo(s"Submitted application: $appName")
@@ -1911,8 +1932,10 @@ class SparkContext(config: SparkConf) extends Logging {
19111932
ShutdownHookManager.removeShutdownHook(_shutdownHookRef)
19121933
}
19131934

1914-
Utils.tryLogNonFatalError {
1915-
postApplicationEnd()
1935+
if (listenerBus != null) {
1936+
Utils.tryLogNonFatalError {
1937+
postApplicationEnd()
1938+
}
19161939
}
19171940
Utils.tryLogNonFatalError {
19181941
_driverLogger.foreach(_.stop())
@@ -1960,6 +1983,9 @@ class SparkContext(config: SparkConf) extends Logging {
19601983
Utils.tryLogNonFatalError {
19611984
_progressBar.foreach(_.stop())
19621985
}
1986+
if (isClientStandalone) {
1987+
releaseResources(_conf, SPARK_DRIVER_PREFIX, _resources, Utils.getProcessId)
1988+
}
19631989
_taskScheduler = null
19641990
// TODO: Cache.stop()?
19651991
if (_env != null) {
@@ -2726,7 +2752,7 @@ object SparkContext extends Logging {
27262752

27272753
// Calculate the max slots each executor can provide based on resources available on each
27282754
// executor and resources required by each task.
2729-
val taskResourceRequirements = parseTaskResourceRequirements(sc.conf)
2755+
val taskResourceRequirements = parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX)
27302756
val executorResourcesAndAmounts =
27312757
parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX)
27322758
.map(request => (request.id.resourceName, request.amount)).toMap

core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala

+4-1
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@ package org.apache.spark.deploy
1919

2020
import java.net.URI
2121

22+
import org.apache.spark.resource.ResourceRequirement
23+
2224
private[spark] case class ApplicationDescription(
2325
name: String,
2426
maxCores: Option[Int],
@@ -32,7 +34,8 @@ private[spark] case class ApplicationDescription(
3234
// number of executors this application wants to start with,
3335
// only used if dynamic allocation is enabled
3436
initialExecutorLimit: Option[Int] = None,
35-
user: String = System.getProperty("user.name", "<unknown>")) {
37+
user: String = System.getProperty("user.name", "<unknown>"),
38+
resourceReqsPerExecutor: Seq[ResourceRequirement] = Seq.empty) {
3639

3740
override def toString: String = "ApplicationDescription(" + name + ")"
3841
}

core/src/main/scala/org/apache/spark/deploy/Client.scala

+5-2
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import org.apache.spark.deploy.DeployMessages._
2929
import org.apache.spark.deploy.master.{DriverState, Master}
3030
import org.apache.spark.internal.{config, Logging}
3131
import org.apache.spark.internal.config.Network.RPC_ASK_TIMEOUT
32+
import org.apache.spark.resource.ResourceUtils
3233
import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint}
3334
import org.apache.spark.util.{SparkExitCode, ThreadUtils, Utils}
3435

@@ -92,13 +93,15 @@ private class ClientEndpoint(
9293
val command = new Command(mainClass,
9394
Seq("{{WORKER_URL}}", "{{USER_JAR}}", driverArgs.mainClass) ++ driverArgs.driverOptions,
9495
sys.env, classPathEntries, libraryPathEntries, javaOpts)
95-
96+
val driverResourceReqs = ResourceUtils.parseResourceRequirements(conf,
97+
config.SPARK_DRIVER_PREFIX)
9698
val driverDescription = new DriverDescription(
9799
driverArgs.jarUrl,
98100
driverArgs.memory,
99101
driverArgs.cores,
100102
driverArgs.supervise,
101-
command)
103+
command,
104+
driverResourceReqs)
102105
asyncSendToMasterAndForwardReply[SubmitDriverResponse](
103106
RequestSubmitDriver(driverDescription))
104107

core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala

+22-6
Original file line numberDiff line numberDiff line change
@@ -24,14 +24,14 @@ import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
2424
import org.apache.spark.deploy.master.DriverState.DriverState
2525
import org.apache.spark.deploy.master.RecoveryState.MasterState
2626
import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
27+
import org.apache.spark.resource.ResourceInformation
2728
import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef}
2829
import org.apache.spark.util.Utils
2930

3031
private[deploy] sealed trait DeployMessage extends Serializable
3132

3233
/** Contains messages sent between Scheduler endpoint nodes. */
3334
private[deploy] object DeployMessages {
34-
3535
// Worker to Master
3636

3737
/**
@@ -43,6 +43,7 @@ private[deploy] object DeployMessages {
4343
* @param memory the memory size of worker
4444
* @param workerWebUiUrl the worker Web UI address
4545
* @param masterAddress the master address used by the worker to connect
46+
* @param resources the resources of worker
4647
*/
4748
case class RegisterWorker(
4849
id: String,
@@ -52,7 +53,8 @@ private[deploy] object DeployMessages {
5253
cores: Int,
5354
memory: Int,
5455
workerWebUiUrl: String,
55-
masterAddress: RpcAddress)
56+
masterAddress: RpcAddress,
57+
resources: Map[String, ResourceInformation] = Map.empty)
5658
extends DeployMessage {
5759
Utils.checkHost(host)
5860
assert (port > 0)
@@ -72,8 +74,18 @@ private[deploy] object DeployMessages {
7274
exception: Option[Exception])
7375
extends DeployMessage
7476

75-
case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription],
76-
driverIds: Seq[String])
77+
case class WorkerExecutorStateResponse(
78+
desc: ExecutorDescription,
79+
resources: Map[String, ResourceInformation])
80+
81+
case class WorkerDriverStateResponse(
82+
driverId: String,
83+
resources: Map[String, ResourceInformation])
84+
85+
case class WorkerSchedulerStateResponse(
86+
id: String,
87+
execResponses: List[WorkerExecutorStateResponse],
88+
driverResponses: Seq[WorkerDriverStateResponse])
7789

7890
/**
7991
* A worker will send this message to the master when it registers with the master. Then the
@@ -118,10 +130,14 @@ private[deploy] object DeployMessages {
118130
execId: Int,
119131
appDesc: ApplicationDescription,
120132
cores: Int,
121-
memory: Int)
133+
memory: Int,
134+
resources: Map[String, ResourceInformation] = Map.empty)
122135
extends DeployMessage
123136

124-
case class LaunchDriver(driverId: String, driverDesc: DriverDescription) extends DeployMessage
137+
case class LaunchDriver(
138+
driverId: String,
139+
driverDesc: DriverDescription,
140+
resources: Map[String, ResourceInformation] = Map.empty) extends DeployMessage
125141

126142
case class KillDriver(driverId: String) extends DeployMessage
127143

core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala

+4-1
Original file line numberDiff line numberDiff line change
@@ -17,12 +17,15 @@
1717

1818
package org.apache.spark.deploy
1919

20+
import org.apache.spark.resource.ResourceRequirement
21+
2022
private[deploy] case class DriverDescription(
2123
jarUrl: String,
2224
mem: Int,
2325
cores: Int,
2426
supervise: Boolean,
25-
command: Command) {
27+
command: Command,
28+
resourceReqs: Seq[ResourceRequirement] = Seq.empty) {
2629

2730
override def toString: String = s"DriverDescription (${command.mainClass})"
2831
}

core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -64,7 +64,8 @@ class LocalSparkCluster(
6464
/* Start the Workers */
6565
for (workerNum <- 1 to numWorkers) {
6666
val workerEnv = Worker.startRpcEnvAndEndpoint(localHostname, 0, 0, coresPerWorker,
67-
memoryPerWorker, masters, null, Some(workerNum), _conf)
67+
memoryPerWorker, masters, null, Some(workerNum), _conf,
68+
conf.get(config.Worker.SPARK_WORKER_RESOURCE_FILE))
6869
workerRpcEnvs += workerEnv
6970
}
7071

0 commit comments

Comments
 (0)