Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
5905f51
[SPARK-27024] Executor interface for cluster managers to support GPU …
tgravescs Apr 17, 2019
916991e
cleanup
tgravescs Apr 17, 2019
6ff9953
fix style issues
tgravescs Apr 17, 2019
bee34a0
add newline to test file
tgravescs Apr 17, 2019
6170342
Change to use JSON format for resources
tgravescs Apr 18, 2019
4bbaf2a
Update test to use file based arg
tgravescs Apr 18, 2019
55a3bd5
update docs
tgravescs Apr 18, 2019
eb0a8ed
Add in logDebug
tgravescs Apr 18, 2019
abff33f
fix spacing
tgravescs Apr 18, 2019
484a086
review comments
tgravescs Apr 30, 2019
e90582a
Merge branch 'master' of https://github.com/apache/spark into gpu-sch…
tgravescs Apr 30, 2019
d2ed08f
Merge branch 'master' of https://github.com/apache/spark into gpu-sch…
tgravescs May 1, 2019
c07b405
remove explicit getters from ResourceInformation
tgravescs May 1, 2019
01f97c8
add newline end of file
tgravescs May 1, 2019
b5afbd0
Rework to add in more checks on executor startup to make sure matches
tgravescs May 3, 2019
dd5b765
rework
tgravescs May 6, 2019
a9642b8
fix extra string interpolation
tgravescs May 7, 2019
01a6061
rework - remove units and count field from ResourceInformation, no
tgravescs May 9, 2019
9187c6d
Fix import order
tgravescs May 9, 2019
636fef8
fix comment syntax
tgravescs May 9, 2019
3d8e5cd
remove extra convert units function
tgravescs May 9, 2019
4165c60
rework
tgravescs May 10, 2019
dbb61fb
Merge branch 'master' of https://github.com/apache/spark into gpu-sch…
tgravescs May 10, 2019
dd60e42
nit rework
tgravescs May 13, 2019
b9dacef
fix missing string interpolation
tgravescs May 14, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
93 changes: 93 additions & 0 deletions core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* 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

import java.io.File

import com.fasterxml.jackson.core.JsonParseException
import org.json4s.{DefaultFormats, MappingException}
import org.json4s.JsonAST.JValue
import org.json4s.jackson.JsonMethods._

import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.util.Utils.executeAndGetOutput

/**
* Discovers resources (GPUs/FPGAs/etc). It currently only supports resources that have
* addresses.
* This class finds resources by running and parsing the output of the user specified script
* from the config spark.{driver/executor}.resource.{resourceType}.discoveryScript.
* The output of the script it runs is expected to be JSON in the format of the
* ResourceInformation class.
*
* For example: {"name": "gpu", "addresses": ["0","1"]}
*/
private[spark] object ResourceDiscoverer extends Logging {

private implicit val formats = DefaultFormats

def findResources(sparkConf: SparkConf, isDriver: Boolean): Map[String, ResourceInformation] = {
val prefix = if (isDriver) {
SPARK_DRIVER_RESOURCE_PREFIX
} else {
SPARK_EXECUTOR_RESOURCE_PREFIX
}
// get unique resource types by grabbing first part config with multiple periods,
// ie resourceType.count, grab resourceType part
val resourceNames = sparkConf.getAllWithPrefix(prefix).map { case (k, _) =>
k.split('.').head
}.toSet
resourceNames.map { rName => {
val rInfo = getResourceInfoForType(sparkConf, prefix, rName)
(rName -> rInfo)
}}.toMap
}

private def getResourceInfoForType(
sparkConf: SparkConf,
prefix: String,
resourceType: String): ResourceInformation = {
val discoveryConf = prefix + resourceType + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX
val script = sparkConf.getOption(discoveryConf)
val result = if (script.nonEmpty) {
val scriptFile = new File(script.get)
// check that script exists and try to execute
if (scriptFile.exists()) {
try {
val output = executeAndGetOutput(Seq(script.get), new File("."))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

any access control concern here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just the normal ones that I think we need to document ( I just added an item to the documentation jira specifically about this.) Normally the script should be sent with the job and needs to have the correct permissions on it, if they point at a script on the system that has been compromised then there are.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In standalone mode, worker needs to run the script provided by users. Could you leave a TODO inline?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure what TODO you want me to leave here, standalone mode has its own jira, which I don't know how its going to be designed so don't see why I would leave a TODO in this jira. I would expect that scrip tto not be provided by the users, but by the cluster administrator when they start the workers.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nvm, I was thinking about users might be able to let manager/executor run arbitrary scripts.

val parsedJson = parse(output)
val name = (parsedJson \ "name").extract[String]
val addresses = (parsedJson \ "addresses").extract[Array[String]].toArray
new ResourceInformation(name, addresses)
} catch {
case e @ (_: SparkException | _: MappingException | _: JsonParseException) =>
throw new SparkException(s"Error running the resource discovery script: $scriptFile" +
s" for $resourceType", e)
}
} else {
throw new SparkException(s"Resource script: $scriptFile to discover $resourceType" +
s" doesn't exist!")
}
} else {
throw new SparkException(s"User is expecting to use $resourceType resources but " +
s"didn't specify a script via conf: $discoveryConf, to find them!")
}
result
}
}
37 changes: 37 additions & 0 deletions core/src/main/scala/org/apache/spark/ResourceInformation.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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

import org.apache.spark.annotation.Evolving

/**
* Class to hold information about a type of Resource. A resource could be a GPU, FPGA, etc.
* The array of addresses are resource specific and its up to the user to interpret the address.
*
* One example is GPUs, where the addresses would be the indices of the GPUs
*
* @param name the name of the resource
* @param addresses an array of strings describing the addresses of the resource
*/
@Evolving
class ResourceInformation(
val name: String,
val addresses: Array[String]) extends Serializable {

override def toString: String = s"[name: ${name}, addresses: ${addresses.mkString(",")}]"
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.executor

import java.io.{BufferedInputStream, FileInputStream}
import java.net.URL
import java.nio.ByteBuffer
import java.util.Locale
Expand All @@ -26,12 +27,18 @@ import scala.collection.mutable
import scala.util.{Failure, Success}
import scala.util.control.NonFatal

import com.fasterxml.jackson.databind.exc.MismatchedInputException
import org.json4s.DefaultFormats
import org.json4s.JsonAST.JArray
import org.json4s.MappingException
import org.json4s.jackson.JsonMethods._

import org.apache.spark._
import org.apache.spark.TaskState.TaskState
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.deploy.worker.WorkerWatcher
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.EXECUTOR_ID
import org.apache.spark.internal.config._
import org.apache.spark.rpc._
import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription}
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
Expand All @@ -45,9 +52,12 @@ private[spark] class CoarseGrainedExecutorBackend(
hostname: String,
cores: Int,
userClassPath: Seq[URL],
env: SparkEnv)
env: SparkEnv,
resourcesFile: Option[String])
extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging {

private implicit val formats = DefaultFormats

private[this] val stopping = new AtomicBoolean(false)
var executor: Executor = null
@volatile var driver: Option[RpcEndpointRef] = None
Expand All @@ -58,11 +68,12 @@ private[spark] class CoarseGrainedExecutorBackend(

override def onStart() {
logInfo("Connecting to driver: " + driverUrl)
val resources = parseOrFindResources(resourcesFile)
rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref =>
// This is a very fast action so we can use "ThreadUtils.sameThread"
driver = Some(ref)
ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls,
extractAttributes))
extractAttributes, resources))
}(ThreadUtils.sameThread).onComplete {
// This is a very fast action so we can use "ThreadUtils.sameThread"
case Success(msg) =>
Expand All @@ -72,6 +83,97 @@ private[spark] class CoarseGrainedExecutorBackend(
}(ThreadUtils.sameThread)
}

// Check that the actual resources discovered will satisfy the user specified
// requirements and that they match the configs specified by the user to catch
// mismatches between what the user requested and what resource manager gave or
// what the discovery script found.
private def checkResourcesMeetRequirements(
Copy link
Contributor

@mengxr mengxr May 14, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Had an offline discussion with @WeichenXu123 . He suggested refactoring this check to make it easier to read. Now the arguments are:

  • reqResourcesAndCounts: request per task (not per executor)
  • actualResources: resources allocated per executor

It is not easy to tell from the variable names and hence make the code harder to read. Basically we need the following:

  1. number allocated per executor cannot be smaller than requested count for each resource name
  2. requested count for executor cannot be smaller than requested count for task for each resource name. Note that this doesn't require resource discovery.
  3. the set of requested resource names for executors should match the set of requested resource names for tasks.

It would be nice to refactor the method into those three. We can also do it in a follow-up PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added a comment to the driver jira since it needs to be refactored for that anyway:
https://issues.apache.org/jira/browse/SPARK-27488

resourceConfigPrefix: String,
reqResourcesAndCounts: Array[(String, String)],
actualResources: Map[String, ResourceInformation]): Unit = {

reqResourcesAndCounts.foreach { case (rName, reqCount) =>
if (actualResources.contains(rName)) {
val resourceInfo = actualResources(rName)

if (resourceInfo.addresses.size < reqCount.toLong) {
throw new SparkException(s"Resource: $rName with addresses: " +
s"${resourceInfo.addresses.mkString(",")} doesn't meet the " +
s"requirements of needing $reqCount of them")
}
// also make sure the resource count on start matches the
// resource configs specified by user
val userCountConfigName =
resourceConfigPrefix + rName + SPARK_RESOURCE_COUNT_POSTFIX
val userConfigCount = env.conf.getOption(userCountConfigName).
getOrElse(throw new SparkException(s"Resource: $rName not specified " +
s"via config: $userCountConfigName, but required, " +
"please fix your configuration"))

if (userConfigCount.toLong > resourceInfo.addresses.size) {
throw new SparkException(s"Resource: $rName, with addresses: " +
s"${resourceInfo.addresses.mkString(",")} " +
s"is less than what the user requested for count: $userConfigCount, " +
s"via $userCountConfigName")
}
} else {
throw new SparkException(s"Executor resource config missing required task resource: $rName")
}
}
}

// visible for testing
def parseOrFindResources(resourcesFile: Option[String]): Map[String, ResourceInformation] = {
// only parse the resources if a task requires them
val taskResourceConfigs = env.conf.getAllWithPrefix(SPARK_TASK_RESOURCE_PREFIX)
val resourceInfo = if (taskResourceConfigs.nonEmpty) {
val execResources = resourcesFile.map { resourceFileStr => {
val source = new BufferedInputStream(new FileInputStream(resourceFileStr))
val resourceMap = try {
val parsedJson = parse(source).asInstanceOf[JArray].arr
parsedJson.map { json =>
val name = (json \ "name").extract[String]
val addresses = (json \ "addresses").extract[Array[String]]
new ResourceInformation(name, addresses)
}.map(x => (x.name -> x)).toMap
} catch {
case e @ (_: MappingException | _: MismatchedInputException) =>
throw new SparkException(
s"Exception parsing the resources in $resourceFileStr", e)
} finally {
source.close()
}
resourceMap
}}.getOrElse(ResourceDiscoverer.findResources(env.conf, isDriver = false))

if (execResources.isEmpty) {
throw new SparkException("User specified resources per task via: " +
s"$SPARK_TASK_RESOURCE_PREFIX, but can't find any resources available on the executor.")
}
// get just the map of resource name to count
val resourcesAndCounts = taskResourceConfigs.
withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)}.
map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.size), v)}

checkResourcesMeetRequirements(SPARK_EXECUTOR_RESOURCE_PREFIX, resourcesAndCounts,
execResources)

logInfo("===============================================================================")
logInfo(s"Executor $executorId Resources:")
execResources.foreach { case (k, v) => logInfo(s"$k -> $v") }
logInfo("===============================================================================")

execResources
} else {
if (resourcesFile.nonEmpty) {
logWarning(s"A resources file was specified but the application is not configured " +
s"to use any resources, see the configs with prefix: ${SPARK_TASK_RESOURCE_PREFIX}")
}
Map.empty[String, ResourceInformation]
}
resourceInfo
}

def extractLogUrls: Map[String, String] = {
val prefix = "SPARK_LOG_URL_"
sys.env.filterKeys(_.startsWith(prefix))
Expand Down Expand Up @@ -189,13 +291,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
cores: Int,
appId: String,
workerUrl: Option[String],
userClassPath: mutable.ListBuffer[URL])
userClassPath: mutable.ListBuffer[URL],
resourcesFile: Option[String])

def main(args: Array[String]): Unit = {
val createFn: (RpcEnv, Arguments, SparkEnv) =>
CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) =>
new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId,
arguments.hostname, arguments.cores, arguments.userClassPath, env)
arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFile)
}
run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn)
System.exit(0)
Expand Down Expand Up @@ -257,6 +360,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
var executorId: String = null
var hostname: String = null
var cores: Int = 0
var resourcesFile: Option[String] = None
var appId: String = null
var workerUrl: Option[String] = None
val userClassPath = new mutable.ListBuffer[URL]()
Expand All @@ -276,6 +380,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
case ("--cores") :: value :: tail =>
cores = value.toInt
argv = tail
case ("--resourcesFile") :: value :: tail =>
resourcesFile = Some(value)
argv = tail
case ("--app-id") :: value :: tail =>
appId = value
argv = tail
Expand All @@ -301,7 +408,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
}

Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl,
userClassPath)
userClassPath, resourcesFile)
}

private def printUsageAndExit(classNameForEntry: String): Unit = {
Expand All @@ -315,6 +422,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
| --executor-id <executorId>
| --hostname <hostname>
| --cores <cores>
| --resourcesFile <fileWithJSONResourceInformation>
| --app-id <appid>
| --worker-url <workerUrl>
| --user-class-path <url>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,13 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader.MAX_

package object config {

private[spark] val SPARK_DRIVER_RESOURCE_PREFIX = "spark.driver.resource."
private[spark] val SPARK_EXECUTOR_RESOURCE_PREFIX = "spark.executor.resource."
private[spark] val SPARK_TASK_RESOURCE_PREFIX = "spark.task.resource."

private[spark] val SPARK_RESOURCE_COUNT_POSTFIX = ".count"
private[spark] val SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX = ".discoveryScript"

private[spark] val DRIVER_CLASS_PATH =
ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster

import java.nio.ByteBuffer

import org.apache.spark.ResourceInformation
import org.apache.spark.TaskState.TaskState
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.scheduler.ExecutorLossReason
Expand Down Expand Up @@ -64,7 +65,8 @@ private[spark] object CoarseGrainedClusterMessages {
hostname: String,
cores: Int,
logUrls: Map[String, String],
attributes: Map[String, String])
attributes: Map[String, String],
resources: Map[String, ResourceInformation])
extends CoarseGrainedClusterMessage

case class StatusUpdate(executorId: String, taskId: Long, state: TaskState,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp

override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {

case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls, attributes) =>
case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls,
attributes, resources) =>
if (executorDataMap.contains(executorId)) {
executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId))
context.reply(true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -174,9 +174,11 @@ class HeartbeatReceiverSuite
val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1)
val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2)
fakeSchedulerBackend.driverEndpoint.askSync[Boolean](
RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty, Map.empty))
RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty, Map.empty,
Map.empty))
fakeSchedulerBackend.driverEndpoint.askSync[Boolean](
RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty, Map.empty))
RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty, Map.empty,
Map.empty))
heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet)
addExecutorAndVerify(executorId1)
addExecutorAndVerify(executorId2)
Expand Down
Loading