From 5905f5194b1a276c666fdc2ba393b75616cd0121 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 17 Apr 2019 11:23:21 -0500 Subject: [PATCH 01/22] [SPARK-27024] Executor interface for cluster managers to support GPU resources --- .../org/apache/spark/ResourceDiscoverer.scala | 98 ++++++++ .../apache/spark/ResourceInformation.scala | 46 ++++ .../CoarseGrainedExecutorBackend.scala | 58 ++++- .../spark/internal/config/package.scala | 14 +- .../cluster/CoarseGrainedClusterMessage.scala | 5 +- .../CoarseGrainedSchedulerBackend.scala | 3 +- .../apache/spark/HeartbeatReceiverSuite.scala | 6 +- .../spark/ResourceDiscovererSuite.scala | 234 ++++++++++++++++++ .../StandaloneDynamicAllocationSuite.scala | 6 +- .../CoarseGrainedExecutorBackendSuite.scala | 160 ++++++++++++ .../CoarseGrainedSchedulerBackendSuite.scala | 6 +- docs/configuration.md | 28 +++ .../src/main/resources/getGpuResources.sh | 35 +++ .../YarnCoarseGrainedExecutorBackend.scala | 8 +- 14 files changed, 689 insertions(+), 18 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala create mode 100644 core/src/main/scala/org/apache/spark/ResourceInformation.scala create mode 100644 core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala create mode 100755 examples/src/main/resources/getGpuResources.sh diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala new file mode 100644 index 000000000000..a10053df4a88 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -0,0 +1,98 @@ +/* + * 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 org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils.executeAndGetOutput + +/** + * Discovers resources (GPUs/FPGAs/etc). + * This class find resources by running and parses the output of the user specified script + * from the config spark.{driver/executor}.{resourceType}.discoveryScript. + * The output of the script it runs is expected to be a String that is in the format of + * count:unit:comma-separated list of addresses, where the list of addresses is + * specific for that resource type. The user is responsible for interpreting the address. + */ +private[spark] object ResourceDiscoverer extends Logging { + + 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 + val resourceTypes = sparkconf.getAllWithPrefix(prefix).map(x => x._1.split('.')(0)).toSet + resourceTypes.map{ rtype => { + val rInfo = getResourceAddrsForType(sparkconf, prefix, rtype) + (rtype -> rInfo) + }}.toMap + } + + private def getResourceAddrsForType(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(".")) + parseResourceTypeString(resourceType, output) + } catch { + case e @ (_: SparkException | _: NumberFormatException) => + 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 + } + + // this parses a resource information string in the format: + // count:unit:comma-separated list of addresses + // The units and addresses are optional. The idea being if the user has something like + // memory you don't have addresses to assign out. + def parseResourceTypeString(rtype: String, rInfoStr: String): ResourceInformation = { + // format should be: count:unit:addr1,addr2,addr3 + val singleResourceType = rInfoStr.split(':') + if (singleResourceType.size < 3) { + throw new SparkException("Format of the resourceAddrs parameter is invalid," + + " please specify all of count, unit, and addresses in the format:" + + " count:unit:addr1,addr2,addr3") + } + // format should be: addr1,addr2,addr3 + val splitAddrs = singleResourceType(2).split(',').map(_.trim()) + val retAddrs = if (splitAddrs.size == 1 && splitAddrs(0).isEmpty()) { + Array.empty[String] + } else { + splitAddrs + } + new ResourceInformation(rtype, singleResourceType(1), singleResourceType(0).toLong, retAddrs) + } +} diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala new file mode 100644 index 000000000000..647aa6de4985 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -0,0 +1,46 @@ +/* + * 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. + * The units and addresses could be empty if they doesn't apply to that resource. + * + * One example is GPUs, where the addresses would be the indices of the GPUs, the count would be the + * number of GPUs and the units would be an empty string. + * + * @param name the name of the resource + * @param units the units of the resources, can be an empty string if units don't apply + * @param count the number of resources available + * @param addresses an optional array of strings describing the addresses of the resource + */ +@Evolving +case class ResourceInformation( + private val name: String, + private val units: String, + private val count: Long, + private val addresses: Array[String] = Array.empty) { + + def getName(): String = name + def getUnits(): String = units + def getCount(): Long = count + def getAddresses(): Array[String] = addresses +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 645f58716de6..beea89a2f642 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,6 +31,7 @@ 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._ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -44,7 +45,8 @@ private[spark] class CoarseGrainedExecutorBackend( hostname: String, cores: Int, userClassPath: Seq[URL], - env: SparkEnv) + env: SparkEnv, + resourceAddrs: Option[String]) extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { private[this] val stopping = new AtomicBoolean(false) @@ -60,8 +62,9 @@ private[spark] class CoarseGrainedExecutorBackend( 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, parseResources(resourceAddrs))) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -71,6 +74,45 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } + // visible for testing + def parseResources(resourceAddrsArg: Option[String]): Map[String, ResourceInformation] = { + // only parse the resources if a task requires them + val taskConfPrefix = SPARK_TASK_RESOURCE_PREFIX + val resourceInfo = if (env.conf.getAllWithPrefix(taskConfPrefix).size > 0) { + val resources = resourceAddrsArg.map(resourceStr => { + // format here would be: + // resourceType=count:unit:addr1,addr2,addr3;resourceType2=count:unit:r2addr1,r2addr2, + // first separate out resource types + val allResourceTypes = resourceStr.split(';').map(_.trim()).map( eachResource => { + // format here should be: resourceType=count:unit:addr1,addr2,addr3 + val typeAndValue = eachResource.split('=').map(_.trim) + if (typeAndValue.size < 2) { + throw new SparkException("Format of the resourceAddrs parameter is invalid," + + " please specify both resource type and the count:unit:addresses: " + + "--resourceAddrs ") + } + val resType = typeAndValue(0) + // format should be: count:unit:addr1,addr2,addr3 + val singleResourceInfo = ResourceDiscoverer.parseResourceTypeString(resType, typeAndValue(1)) + (resType, singleResourceInfo) + }).toMap + allResourceTypes + }).getOrElse(ResourceDiscoverer.findResources(env.conf, false)) + + if (resources.size == 0) { + throw new SparkException(s"User specified resources per task via: $taskConfPrefix," + + s" but can't find any resources available on the executor.") + } + logInfo(s"Executor ${executorId} using resources: ${resources.values}") + // todo - add logDebug with full output? + resources + } else { + Map.empty[String, ResourceInformation] + } + resourceInfo + } + def extractLogUrls: Map[String, String] = { val prefix = "SPARK_LOG_URL_" sys.env.filterKeys(_.startsWith(prefix)) @@ -188,13 +230,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { cores: Int, appId: String, workerUrl: Option[String], - userClassPath: mutable.ListBuffer[URL]) + userClassPath: mutable.ListBuffer[URL], + resourceAddrs: 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.resourceAddrs) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) System.exit(0) @@ -255,6 +298,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { var executorId: String = null var hostname: String = null var cores: Int = 0 + var resourceAddrs: Option[String] = None var appId: String = null var workerUrl: Option[String] = None val userClassPath = new mutable.ListBuffer[URL]() @@ -274,6 +318,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--cores") :: value :: tail => cores = value.toInt argv = tail + case ("--resourceAddrs") :: value :: tail => + resourceAddrs = Some(value) + argv = tail case ("--app-id") :: value :: tail => appId = value argv = tail @@ -299,7 +346,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl, - userClassPath) + userClassPath, resourceAddrs) } private def printUsageAndExit(classNameForEntry: String): Unit = { @@ -313,6 +360,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | --executor-id | --hostname | --cores + | --resourceAddrs | --app-id | --worker-url | --user-class-path diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 0bd46bef35d2..5caba483fa4f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -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 @@ -170,6 +177,7 @@ package object config { private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) .intConf + .checkValue(_ > 0, "Each executor must contain at least 1 cpu core.") .createWithDefault(1) private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) @@ -314,7 +322,11 @@ package object config { private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) - private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) + private[spark] val CPUS_PER_TASK = + ConfigBuilder("spark.task.cpus") + .intConf + .checkValue(_ > 0, "Each task must require at least 1 cpu core.") + .createWithDefault(1) private[spark] val DYN_ALLOCATION_ENABLED = ConfigBuilder("spark.dynamicAllocation.enabled").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index afb48a31754f..c85ba0b456da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -19,6 +19,8 @@ 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 @@ -64,7 +66,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, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 4830d0e6f800..f7cf212d0bfe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -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) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 061aeb366cf5..dc4f4b4c66d9 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -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) diff --git a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala new file mode 100644 index 000000000000..569c415bf24f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala @@ -0,0 +1,234 @@ +/* + * 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 java.nio.charset.StandardCharsets +import java.nio.file.{Files => JavaFiles} +import java.nio.file.attribute.PosixFilePermission._ +import java.util.EnumSet + +import com.google.common.io.Files + +import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + + +class ResourceDiscovererSuite extends SparkFunSuite + with LocalSparkContext { + + test("Resource discoverer no resources") { + val sparkconf = new SparkConf + val resources = ResourceDiscoverer.findResources(sparkconf, false) + assert(resources.size === 0) + assert(resources.get("gpu").isEmpty, + "Should have a gpus entry that is empty") + } + + test("Resource discoverer multiple gpus") { + val sparkconf = new SparkConf + + assume(!(Utils.isWindows)) + + withTempDir { dir => + val file1 = new File(dir, "resourceDiscoverScript1") + Files.write("echo 2::0,1", file1, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file1.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + val resources = ResourceDiscoverer.findResources(sparkconf, false) + val gpuValue = resources.get("gpu") + assert(gpuValue.nonEmpty, "Should have a gpu entry") + assert(gpuValue.get.getCount() == 2, "Should have 2") + assert(gpuValue.get.getName() == "gpu", "name should be gpu") + assert(gpuValue.get.getUnits() == "", "units should be empty") + assert(gpuValue.get.getAddresses().size == 2, "Should have 2 indexes") + assert(gpuValue.get.getAddresses().deep == Array("0", "1").deep, "should have 0,1 entries") + } + } + + test("Resource discoverer no addresses") { + val sparkconf = new SparkConf + + assume(!(Utils.isWindows)) + + withTempDir { dir => + val file1 = new File(dir, "resourceDiscoverScript1") + Files.write("echo 2::", file1, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file1.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + val resources = ResourceDiscoverer.findResources(sparkconf, false) + val gpuValue = resources.get("gpu") + assert(gpuValue.nonEmpty, "Should have a gpu entry") + assert(gpuValue.get.getCount() == 2, "Should have 2") + assert(gpuValue.get.getName() == "gpu", "name should be gpu") + assert(gpuValue.get.getUnits() == "", "units should be empty") + assert(gpuValue.get.getAddresses().size == 0, "Should have 0 indexes") + } + } + + test("Resource discoverer no count") { + val sparkconf = new SparkConf + + assume(!(Utils.isWindows)) + + withTempDir { dir => + val file1 = new File(dir, "resourceDiscoverScript1") + Files.write("echo ::0,1", file1, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file1.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, false) + }.getMessage() + + assert(error.contains("Error running the resource discovery")) + } + } + + + test("Resource discoverer multiple resource types") { + val sparkconf = new SparkConf + + assume(!(Utils.isWindows)) + + withTempDir { dir => + val gpuDiscovery = new File(dir, "resourceDiscoverScriptgpu") + Files.write("echo 2::0,1", gpuDiscovery, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(gpuDiscovery.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + + val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") + Files.write("echo 3:mb:f1,f2,f3", fpgaDiscovery, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, gpuDiscovery.getPath()) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, fpgaDiscovery.getPath()) + val resources = ResourceDiscoverer.findResources(sparkconf, false) + assert(resources.size === 2) + val gpuValue = resources.get("gpu") + assert(gpuValue.nonEmpty, "Should have a gpu entry") + assert(gpuValue.get.getCount() == 2, "Should have 2") + assert(gpuValue.get.getName() == "gpu", "name should be gpu") + assert(gpuValue.get.getUnits() == "", "units should be empty") + assert(gpuValue.get.getAddresses().size == 2, "Should have 2 indexes") + assert(gpuValue.get.getAddresses().deep == Array("0", "1").deep, "should have 0,1 entries") + + val fpgaValue = resources.get("fpga") + assert(fpgaValue.nonEmpty, "Should have a gpu entry") + assert(fpgaValue.get.getCount() == 3, "Should have 3") + assert(fpgaValue.get.getName() == "fpga", "name should be fpga") + assert(fpgaValue.get.getUnits() == "mb", "units should be mb") + assert(fpgaValue.get.getAddresses().size == 3, "Should have 3 indexes") + assert(fpgaValue.get.getAddresses().deep == Array("f1", "f2", "f3").deep, + "should have f1,f2,f3 entries") + } + } + + test("Resource discoverer multiple gpus on driver") { + val sparkconf = new SparkConf + + assume(!(Utils.isWindows)) + + withTempDir { dir => + val file1 = new File(dir, "resourceDiscoverScript2") + Files.write("echo 2::0,1", file1, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file1.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + sparkconf.set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, "boguspath") + // make sure it reads from correct config, here it should use driver + val resources = ResourceDiscoverer.findResources(sparkconf, true) + val gpuValue = resources.get("gpu") + assert(gpuValue.nonEmpty, "Should have a gpu entry") + assert(gpuValue.get.getCount() == 2, "Should have 2") + assert(gpuValue.get.getName() == "gpu", "name should be gpu") + assert(gpuValue.get.getUnits() == "", "units should be empty") + assert(gpuValue.get.getAddresses().size == 2, "Should have 2 indexes") + assert(gpuValue.get.getAddresses().deep == Array("0", "1").deep, "should have 0,1 entries") + + } + } + + test("Resource discoverer script returns invalid format") { + val sparkconf = new SparkConf + assume(!(Utils.isWindows)) + + withTempDir { dir => + val file1 = new File(dir, "resourceDiscoverScript3") + Files.write("echo foo1", file1, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file1.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, false) + }.getMessage() + + assert(error.contains("Error running the resource discovery")) + } + } + + test("Resource discoverer script doesn't exist") { + val sparkconf = new SparkConf + + val file1 = new File("/tmp/bogus") + try { + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, false) + }.getMessage() + + assert(error.contains("doesn't exist")) + } finally { + JavaFiles.deleteIfExists(file1.toPath()) + } + } + + test("gpu's specified but not discovery script") { + val sparkconf = new SparkConf + + val file1 = new File("/tmp/bogus") + try { + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_COUNT_POSTFIX, "2") + + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, false) + }.getMessage() + + assert(error.contains("User is expecting to use")) + } finally { + JavaFiles.deleteIfExists(file1.toPath()) + } + } + +} diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 1ed2a1a2aeb4..39daf51ee613 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -497,7 +497,8 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty, Map.empty) + val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty, Map.empty, + Map.empty) // Get "localhost" on a blacklist. val taskScheduler = mock(classOf[TaskSchedulerImpl]) @@ -621,7 +622,8 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty) + val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty, + Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala new file mode 100644 index 000000000000..063de8cae56b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -0,0 +1,160 @@ +/* + * 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.executor + + +import java.io.File +import java.net.URL +import java.nio.charset.StandardCharsets +import java.nio.file.{Files => JavaFiles} +import java.nio.file.attribute.PosixFilePermission.{OWNER_EXECUTE, OWNER_READ, OWNER_WRITE} +import java.util.EnumSet + +import com.google.common.io.Files +import org.mockito.Mockito.when +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.util.Utils + + +class CoarseGrainedExecutorBackendSuite extends SparkFunSuite + with LocalSparkContext with MockitoSugar { + + test("parsing no resources") { + val conf = new SparkConf + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + var testResourceArgs = Some("") + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(testResourceArgs) + }.getMessage() + + assert(error.contains("Format of the resourceAddrs parameter is invalid")) + + testResourceArgs = Some("gpu=::") + error = intercept[SparkException] { + val parsedResources = backend.parseResources(testResourceArgs) + }.getMessage() + + assert(error.contains("Format of the resourceAddrs parameter is invalid")) + } + + + test("parsing one resources") { + val conf = new SparkConf + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + val testResourceArgs = Some("gpu=2::0,1") + val parsedResources = backend.parseResources(testResourceArgs) + + assert(parsedResources.size === 1) + assert(parsedResources.get("gpu").nonEmpty) + assert(parsedResources.get("gpu").get.getName() === "gpu") + assert(parsedResources.get("gpu").get.getUnits() === "") + assert(parsedResources.get("gpu").get.getCount() === 2) + assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + } + + test("parsing multiple resources") { + val conf = new SparkConf + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + val testResourceArgs = Some("gpu=2::0,1;fpga=3:mb:f1,f2,f3") + val parsedResources = backend.parseResources(testResourceArgs) + + assert(parsedResources.size === 2) + assert(parsedResources.get("gpu").nonEmpty) + assert(parsedResources.get("gpu").get.getName() === "gpu") + assert(parsedResources.get("gpu").get.getUnits() === "") + assert(parsedResources.get("gpu").get.getCount() === 2) + assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + assert(parsedResources.get("fpga").nonEmpty) + assert(parsedResources.get("fpga").get.getName() === "fpga") + assert(parsedResources.get("fpga").get.getUnits() === "mb") + assert(parsedResources.get("fpga").get.getCount() === 3) + assert(parsedResources.get("fpga").get.getAddresses().deep === Array("f1", "f2", "f3").deep) + } + + test("use discoverer") { + val conf = new SparkConf + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + + assume(!(Utils.isWindows)) + + withTempDir { dir => + val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") + Files.write("echo 3::f1,f2,f3", fpgaDiscovery, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, fpgaDiscovery.getPath()) + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + val parsedResources = backend.parseResources(None) + + assert(parsedResources.size === 1) + assert(parsedResources.get("fpga").nonEmpty) + assert(parsedResources.get("fpga").get.getName() === "fpga") + assert(parsedResources.get("fpga").get.getUnits() === "") + assert(parsedResources.get("fpga").get.getCount() === 3) + assert(parsedResources.get("fpga").get.getAddresses().deep === Array("f1", "f2", "f3").deep) + } + } + + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { + val mockEnv = mock[SparkEnv] + val mockRpcEnv = mock[RpcEnv] + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.serializer).thenReturn(serializer) + when(mockEnv.closureSerializer).thenReturn(serializer) + when(mockEnv.rpcEnv).thenReturn(mockRpcEnv) + SparkEnv.set(mockEnv) + mockEnv + } +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 33f48b858a73..4858d38cad40 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -164,11 +164,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo sc.addSparkListener(listener) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes)) + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes)) + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes)) + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) assert(executorAddedCount === 3) diff --git a/docs/configuration.md b/docs/configuration.md index 5325f8a352f7..9278f99244b9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -221,6 +221,25 @@ of the most common options to set are: This option is currently supported on YARN and Kubernetes. + + spark.executor.resource.{resourceType}.count + 0 + + The number of a particular resource type to use per executor process. + If this is used, you must also specify the + spark.executor.resource.{resourceType}.discoveryScript + for the executor to find the resource on startup. + + + + spark.executor.resource.{resourceType}.discoveryScript + None + + The script the executor should run to discover a particular resource type. This should + return a string in the format of: count:unit:comma-separated list of addresses. + unit and addresses can be empty if they don't apply to the resource type. + + spark.extraListeners (none) @@ -1793,6 +1812,15 @@ Apart from these, the following properties are also available, and may be useful Number of cores to allocate for each task. + + spark.task.resource.{resourceType}.count + 1 + + Number of a particular resource type to allocate for each task. If this is specified + you must also provide the executor config spark.executor.resource.{resourceType}.count + and any corresponding discovery configs so that your executors are created with that resource type. + + spark.task.maxFailures 4 diff --git a/examples/src/main/resources/getGpuResources.sh b/examples/src/main/resources/getGpuResources.sh new file mode 100755 index 000000000000..a32f804fcad9 --- /dev/null +++ b/examples/src/main/resources/getGpuResources.sh @@ -0,0 +1,35 @@ +# +# 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. +# + +# +# This is an example script that can be used to discover GPUs. It only works on NVIDIA GPUs since it +# uses the nvidia-smi command. This script will find all visible GPUs so if you aren't running +# in an environment that can isolate GPUs to an executor and where multiple executors can run on a +# single node you may not want to use this. See your cluster manager specific configs for other +# options. +# +# It can be passed into Spark via the configs spark.executor.resource.gpu.discoveryScript and/or +# spark.driver.resource.gpu.discoveryScript. +# The script will return the format count:unit:comma-separated string of the GPU indices available where it was executed. +# If used with the executors, Spark will assigned out the indices to tasks based on the config to +# control the number of GPUs per task. The driver +# +# +ADDRS=`nvidia-smi --query-gpu=index --format=csv,noheader | sed 'N;s/\n/,/'` +COUNT=`echo $ADDRS | tr -cd , | wc -c` +ALLCOUNT=`expr $COUNT + 1` +echo $ALLCOUNT::$ADDRS \ No newline at end of file diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index 53e99d992db8..757b7a65813f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -37,7 +37,8 @@ private[spark] class YarnCoarseGrainedExecutorBackend( hostname: String, cores: Int, userClassPath: Seq[URL], - env: SparkEnv) + env: SparkEnv, + resourceAddrs: Option[String]) extends CoarseGrainedExecutorBackend( rpcEnv, driverUrl, @@ -45,7 +46,8 @@ private[spark] class YarnCoarseGrainedExecutorBackend( hostname, cores, userClassPath, - env) with Logging { + env, + resourceAddrs) with Logging { private lazy val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(env.conf) @@ -66,7 +68,7 @@ private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.hostname, arguments.cores, arguments.userClassPath, env) + arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourceAddrs) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")) From 916991e302be68fc328ddbd2da81918f734258cb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 17 Apr 2019 11:33:20 -0500 Subject: [PATCH 02/22] cleanup --- .../main/scala/org/apache/spark/ResourceDiscoverer.scala | 6 ++++-- .../spark/executor/CoarseGrainedExecutorBackend.scala | 1 - .../scala/org/apache/spark/internal/config/package.scala | 7 +------ examples/src/main/resources/getGpuResources.sh | 8 +++----- 4 files changed, 8 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index a10053df4a88..f7343869744a 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -47,8 +47,10 @@ private[spark] object ResourceDiscoverer extends Logging { }}.toMap } - private def getResourceAddrsForType(sparkconf: SparkConf, - prefix: String, resourceType: String): ResourceInformation = { + private def getResourceAddrsForType( + 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) { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index beea89a2f642..1ff9aef71884 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -62,7 +62,6 @@ private[spark] class CoarseGrainedExecutorBackend( 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, parseResources(resourceAddrs))) }(ThreadUtils.sameThread).onComplete { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5caba483fa4f..93582eace4f2 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -177,7 +177,6 @@ package object config { private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) .intConf - .checkValue(_ > 0, "Each executor must contain at least 1 cpu core.") .createWithDefault(1) private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) @@ -322,11 +321,7 @@ package object config { private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) - private[spark] val CPUS_PER_TASK = - ConfigBuilder("spark.task.cpus") - .intConf - .checkValue(_ > 0, "Each task must require at least 1 cpu core.") - .createWithDefault(1) + private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) private[spark] val DYN_ALLOCATION_ENABLED = ConfigBuilder("spark.dynamicAllocation.enabled").booleanConf.createWithDefault(false) diff --git a/examples/src/main/resources/getGpuResources.sh b/examples/src/main/resources/getGpuResources.sh index a32f804fcad9..8cb8ec673663 100755 --- a/examples/src/main/resources/getGpuResources.sh +++ b/examples/src/main/resources/getGpuResources.sh @@ -17,18 +17,16 @@ # # This is an example script that can be used to discover GPUs. It only works on NVIDIA GPUs since it -# uses the nvidia-smi command. This script will find all visible GPUs so if you aren't running +# uses the nvidia-smi command. This script will find all visible GPUs, so if you aren't running # in an environment that can isolate GPUs to an executor and where multiple executors can run on a # single node you may not want to use this. See your cluster manager specific configs for other # options. # # It can be passed into Spark via the configs spark.executor.resource.gpu.discoveryScript and/or # spark.driver.resource.gpu.discoveryScript. -# The script will return the format count:unit:comma-separated string of the GPU indices available where it was executed. -# If used with the executors, Spark will assigned out the indices to tasks based on the config to -# control the number of GPUs per task. The driver -# +# The script will return a string in the format: count:unit:comma-separated list of the resource addresses # + ADDRS=`nvidia-smi --query-gpu=index --format=csv,noheader | sed 'N;s/\n/,/'` COUNT=`echo $ADDRS | tr -cd , | wc -c` ALLCOUNT=`expr $COUNT + 1` From 6ff9953b00df8723987f12303545104e97f00b3a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 17 Apr 2019 12:45:17 -0500 Subject: [PATCH 03/22] fix style issues --- .../main/scala/org/apache/spark/ResourceInformation.scala | 2 +- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 5 +++-- .../scheduler/cluster/CoarseGrainedClusterMessage.scala | 1 - 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index 647aa6de4985..3a749695a313 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -43,4 +43,4 @@ case class ResourceInformation( def getUnits(): String = units def getCount(): Long = count def getAddresses(): Array[String] = addresses -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 1ff9aef71884..860409588ba0 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -93,7 +93,8 @@ private[spark] class CoarseGrainedExecutorBackend( } val resType = typeAndValue(0) // format should be: count:unit:addr1,addr2,addr3 - val singleResourceInfo = ResourceDiscoverer.parseResourceTypeString(resType, typeAndValue(1)) + val singleResourceInfo = + ResourceDiscoverer.parseResourceTypeString(resType, typeAndValue(1)) (resType, singleResourceInfo) }).toMap allResourceTypes @@ -359,7 +360,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | --executor-id | --hostname | --cores - | --resourceAddrs + | --resourceAddrs | --app-id | --worker-url | --user-class-path diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index c85ba0b456da..89425e702677 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -20,7 +20,6 @@ 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 From bee34a060cb824f96fa9fffbcb120dca0db6a7a9 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 17 Apr 2019 13:24:20 -0500 Subject: [PATCH 04/22] add newline to test file --- .../spark/executor/CoarseGrainedExecutorBackendSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 063de8cae56b..ff9ac1b094a9 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -157,4 +157,4 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite SparkEnv.set(mockEnv) mockEnv } -} \ No newline at end of file +} From 61703427e74916bff48638da7def3560a9c5ba31 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 18 Apr 2019 08:08:23 -0500 Subject: [PATCH 05/22] Change to use JSON format for resources --- .../org/apache/spark/ResourceDiscoverer.scala | 39 ++++--------- .../CoarseGrainedExecutorBackend.scala | 52 +++++++++-------- .../spark/ResourceDiscovererSuite.scala | 56 ++++++++++++------- .../CoarseGrainedExecutorBackendSuite.scala | 22 ++++---- .../src/main/resources/getGpuResources.sh | 6 +- ...osCoarseGrainedSchedulerBackendSuite.scala | 2 +- 6 files changed, 86 insertions(+), 91 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index f7343869744a..91765ed6aa50 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -19,6 +19,10 @@ package org.apache.spark import java.io.File +import com.fasterxml.jackson.core.JsonParseException +import org.json4s.{DefaultFormats, MappingException} +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.util.Utils.executeAndGetOutput @@ -27,9 +31,10 @@ import org.apache.spark.util.Utils.executeAndGetOutput * Discovers resources (GPUs/FPGAs/etc). * This class find resources by running and parses the output of the user specified script * from the config spark.{driver/executor}.{resourceType}.discoveryScript. - * The output of the script it runs is expected to be a String that is in the format of - * count:unit:comma-separated list of addresses, where the list of addresses is - * specific for that resource type. The user is responsible for interpreting the address. + * The output of the script it runs is expected to be JSON in the format of the + * ResourceInformation class, with addresses being optional. + * + * For example: {"name": "gpu","count":2, "units":"", "addresses": ["0","1"] */ private[spark] object ResourceDiscoverer extends Logging { @@ -59,9 +64,11 @@ private[spark] object ResourceDiscoverer extends Logging { if (scriptFile.exists()) { try { val output = executeAndGetOutput(Seq(script.get), new File(".")) - parseResourceTypeString(resourceType, output) + val parsedJson = parse(output) + implicit val formats = DefaultFormats + parsedJson.extract[ResourceInformation] } catch { - case e @ (_: SparkException | _: NumberFormatException) => + case e @ (_: SparkException | _: MappingException | _: JsonParseException) => throw new SparkException(s"Error running the resource discovery script: $scriptFile" + s" for $resourceType", e) } @@ -75,26 +82,4 @@ private[spark] object ResourceDiscoverer extends Logging { } result } - - // this parses a resource information string in the format: - // count:unit:comma-separated list of addresses - // The units and addresses are optional. The idea being if the user has something like - // memory you don't have addresses to assign out. - def parseResourceTypeString(rtype: String, rInfoStr: String): ResourceInformation = { - // format should be: count:unit:addr1,addr2,addr3 - val singleResourceType = rInfoStr.split(':') - if (singleResourceType.size < 3) { - throw new SparkException("Format of the resourceAddrs parameter is invalid," + - " please specify all of count, unit, and addresses in the format:" + - " count:unit:addr1,addr2,addr3") - } - // format should be: addr1,addr2,addr3 - val splitAddrs = singleResourceType(2).split(',').map(_.trim()) - val retAddrs = if (splitAddrs.size == 1 && splitAddrs(0).isEmpty()) { - Array.empty[String] - } else { - splitAddrs - } - new ResourceInformation(rtype, singleResourceType(1), singleResourceType(0).toLong, retAddrs) - } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 860409588ba0..d16e86935266 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -26,6 +26,12 @@ 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 @@ -79,25 +85,18 @@ private[spark] class CoarseGrainedExecutorBackend( val taskConfPrefix = SPARK_TASK_RESOURCE_PREFIX val resourceInfo = if (env.conf.getAllWithPrefix(taskConfPrefix).size > 0) { val resources = resourceAddrsArg.map(resourceStr => { - // format here would be: - // resourceType=count:unit:addr1,addr2,addr3;resourceType2=count:unit:r2addr1,r2addr2, - // first separate out resource types - val allResourceTypes = resourceStr.split(';').map(_.trim()).map( eachResource => { - // format here should be: resourceType=count:unit:addr1,addr2,addr3 - val typeAndValue = eachResource.split('=').map(_.trim) - if (typeAndValue.size < 2) { - throw new SparkException("Format of the resourceAddrs parameter is invalid," + - " please specify both resource type and the count:unit:addresses: " + - "--resourceAddrs ") - } - val resType = typeAndValue(0) - // format should be: count:unit:addr1,addr2,addr3 - val singleResourceInfo = - ResourceDiscoverer.parseResourceTypeString(resType, typeAndValue(1)) - (resType, singleResourceInfo) - }).toMap - allResourceTypes + implicit val formats = DefaultFormats + val resourceMap = try { + val parsedJson = parse(resourceStr).asInstanceOf[JArray].arr + val allResources = parsedJson.map(_.extract[ResourceInformation]). + map(x => (x.getName() -> x)).toMap + allResources + } catch { + case e @ (_: MappingException | _: MismatchedInputException) => + throw new SparkException( + s"Exception parsing the resources passed in: $resourceAddrsArg", e) + } + resourceMap }).getOrElse(ResourceDiscoverer.findResources(env.conf, false)) if (resources.size == 0) { @@ -105,7 +104,6 @@ private[spark] class CoarseGrainedExecutorBackend( s" but can't find any resources available on the executor.") } logInfo(s"Executor ${executorId} using resources: ${resources.values}") - // todo - add logDebug with full output? resources } else { Map.empty[String, ResourceInformation] @@ -231,13 +229,13 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { appId: String, workerUrl: Option[String], userClassPath: mutable.ListBuffer[URL], - resourceAddrs: Option[String]) + 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.resourceAddrs) + arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFile) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) System.exit(0) @@ -298,7 +296,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { var executorId: String = null var hostname: String = null var cores: Int = 0 - var resourceAddrs: Option[String] = None + var resourcesFile: Option[String] = None var appId: String = null var workerUrl: Option[String] = None val userClassPath = new mutable.ListBuffer[URL]() @@ -318,8 +316,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--cores") :: value :: tail => cores = value.toInt argv = tail - case ("--resourceAddrs") :: value :: tail => - resourceAddrs = Some(value) + case ("--resourcesFile") :: value :: tail => + resourcesFile = Some(value) argv = tail case ("--app-id") :: value :: tail => appId = value @@ -346,7 +344,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl, - userClassPath, resourceAddrs) + userClassPath, resourcesFile) } private def printUsageAndExit(classNameForEntry: String): Unit = { @@ -360,7 +358,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | --executor-id | --hostname | --cores - | --resourceAddrs + | --resourcesFile | --app-id | --worker-url | --user-class-path diff --git a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala index 569c415bf24f..8e1ef5c9faad 100644 --- a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala +++ b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala @@ -25,7 +25,6 @@ import java.util.EnumSet import com.google.common.io.Files -import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.util.Utils @@ -43,12 +42,13 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer multiple gpus") { val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo 2::0,1", file1, StandardCharsets.UTF_8) + + // this is a bit ugly but do it the hardway to test out some formatting + Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + @@ -66,12 +66,11 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer no addresses") { val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo 2::", file1, StandardCharsets.UTF_8) + Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"}", + file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + @@ -88,12 +87,11 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer no count") { val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo ::0,1", file1, StandardCharsets.UTF_8) + Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + @@ -106,20 +104,39 @@ class ResourceDiscovererSuite extends SparkFunSuite } } + test("Resource discoverer no count value") { + val sparkconf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + val file1 = new File(dir, "resourceDiscoverScript1") + Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":, \\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file1.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, false) + }.getMessage() + + assert(error.contains("Error running the resource discovery")) + } + } test("Resource discoverer multiple resource types") { val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => val gpuDiscovery = new File(dir, "resourceDiscoverScriptgpu") - Files.write("echo 2::0,1", gpuDiscovery, StandardCharsets.UTF_8) + Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", gpuDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(gpuDiscovery.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") - Files.write("echo 3:mb:f1,f2,f3", fpgaDiscovery, StandardCharsets.UTF_8) + Files.write("echo {\\\"name\\\":\\\"fpga\\\", \\\"count\\\":3, \\\"units\\\":\\\"mb\\\"," + + " \\\"addresses\\\":[\\\"f1\\\",\\\"f2\\\",\\\"f3\\\"]}", + fpgaDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) @@ -150,12 +167,11 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer multiple gpus on driver") { val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript2") - Files.write("echo 2::0,1", file1, StandardCharsets.UTF_8) + Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) sparkconf.set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + @@ -178,10 +194,10 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer script returns invalid format") { val sparkconf = new SparkConf assume(!(Utils.isWindows)) - withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript3") - Files.write("echo foo1", file1, StandardCharsets.UTF_8) + Files.write("echo {\\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + @@ -197,7 +213,6 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer script doesn't exist") { val sparkconf = new SparkConf - val file1 = new File("/tmp/bogus") try { sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + @@ -215,7 +230,6 @@ class ResourceDiscovererSuite extends SparkFunSuite test("gpu's specified but not discovery script") { val sparkconf = new SparkConf - val file1 = new File("/tmp/bogus") try { sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index ff9ac1b094a9..8651c79db0f9 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -42,7 +42,6 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing no resources") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") - val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) @@ -55,14 +54,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseResources(testResourceArgs) }.getMessage() - assert(error.contains("Format of the resourceAddrs parameter is invalid")) - - testResourceArgs = Some("gpu=::") - error = intercept[SparkException] { - val parsedResources = backend.parseResources(testResourceArgs) - }.getMessage() - - assert(error.contains("Format of the resourceAddrs parameter is invalid")) + assert(error.contains("Exception parsing the resources passed")) } @@ -77,7 +69,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) - val testResourceArgs = Some("gpu=2::0,1") + val testResourceArgs = + Some("""[{"name": "gpu", "count":2, "units":"", "addresses":["0","1"]}]""") val parsedResources = backend.parseResources(testResourceArgs) assert(parsedResources.size === 1) @@ -99,7 +92,10 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) - val testResourceArgs = Some("gpu=2::0,1;fpga=3:mb:f1,f2,f3") + val testResourceArgs = + Some( + """[{"name": "gpu", "count":2, "units":"", "addresses":["0","1"]}, + |{"name": "fpga", "count":3, "units":"mb", "addresses":["f1","f2","f3"]}]""".stripMargin) val parsedResources = backend.parseResources(testResourceArgs) assert(parsedResources.size === 2) @@ -123,7 +119,9 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { dir => val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") - Files.write("echo 3::f1,f2,f3", fpgaDiscovery, StandardCharsets.UTF_8) + Files.write("echo {\\\"name\\\":\\\"fpga\\\", \\\"count\\\":3, \\\"units\\\":\\\"\\\"," + + " \\\"addresses\\\":[\\\"f1\\\",\\\"f2\\\",\\\"f3\\\"]}", + fpgaDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + diff --git a/examples/src/main/resources/getGpuResources.sh b/examples/src/main/resources/getGpuResources.sh index 8cb8ec673663..64bdf1aff7be 100755 --- a/examples/src/main/resources/getGpuResources.sh +++ b/examples/src/main/resources/getGpuResources.sh @@ -24,10 +24,10 @@ # # It can be passed into Spark via the configs spark.executor.resource.gpu.discoveryScript and/or # spark.driver.resource.gpu.discoveryScript. -# The script will return a string in the format: count:unit:comma-separated list of the resource addresses +# The script will return a JSON string in the format of the ResourceInformation class. # -ADDRS=`nvidia-smi --query-gpu=index --format=csv,noheader | sed 'N;s/\n/,/'` +ADDRS=`nvidia-smi --query-gpu=index --format=csv,noheader | sed 'N;s/\n/\",\"/'` COUNT=`echo $ADDRS | tr -cd , | wc -c` ALLCOUNT=`expr $COUNT + 1` -echo $ALLCOUNT::$ADDRS \ No newline at end of file +echo {\"name\": \"gpu\", \"count\":$ALLCOUNT, \"units\":\"\", \"addresses\":[\"$ADDRS\"]} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 37c0f5f45075..34b5f6cd19bb 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -692,7 +692,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty, - Map.empty) + Map.empty, Map.empty) backend.driverEndpoint.askSync[Boolean](message) } From 4bbaf2a3173b57d4a72fe0f5cba027d8998f1cc4 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 18 Apr 2019 09:23:43 -0500 Subject: [PATCH 06/22] Update test to use file based arg --- .../CoarseGrainedExecutorBackend.scala | 21 ++-- .../CoarseGrainedExecutorBackendSuite.scala | 106 +++++++++++------- 2 files changed, 81 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index d16e86935266..eb8e8a1816a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -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 @@ -52,9 +53,11 @@ private[spark] class CoarseGrainedExecutorBackend( cores: Int, userClassPath: Seq[URL], env: SparkEnv, - resourceAddrs: Option[String]) + 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 @@ -69,7 +72,7 @@ private[spark] class CoarseGrainedExecutorBackend( // 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, parseResources(resourceAddrs))) + extractAttributes, parseResources(resourcesFile))) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -80,21 +83,23 @@ private[spark] class CoarseGrainedExecutorBackend( } // visible for testing - def parseResources(resourceAddrsArg: Option[String]): Map[String, ResourceInformation] = { + def parseResources(resourcesFile: Option[String]): Map[String, ResourceInformation] = { // only parse the resources if a task requires them val taskConfPrefix = SPARK_TASK_RESOURCE_PREFIX val resourceInfo = if (env.conf.getAllWithPrefix(taskConfPrefix).size > 0) { - val resources = resourceAddrsArg.map(resourceStr => { - implicit val formats = DefaultFormats + val resources = resourcesFile.map(resourceFileStr => { + val source = new BufferedInputStream(new FileInputStream(resourceFileStr)) val resourceMap = try { - val parsedJson = parse(resourceStr).asInstanceOf[JArray].arr + val parsedJson = parse(source).asInstanceOf[JArray].arr val allResources = parsedJson.map(_.extract[ResourceInformation]). map(x => (x.getName() -> x)).toMap allResources } catch { - case e @ (_: MappingException | _: MismatchedInputException) => + case e @ (_: MappingException | _: MismatchedInputException | _: ClassCastException) => throw new SparkException( - s"Exception parsing the resources passed in: $resourceAddrsArg", e) + s"Exception parsing the resources passed in: $resourcesFile", e) + } finally { + source.close() } resourceMap }).getOrElse(ResourceDiscoverer.findResources(env.conf, false)) diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 8651c79db0f9..12486e0c90af 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.executor -import java.io.File +import java.io.{File, PrintWriter} import java.net.URL import java.nio.charset.StandardCharsets import java.nio.file.{Files => JavaFiles} @@ -26,19 +26,31 @@ import java.nio.file.attribute.PosixFilePermission.{OWNER_EXECUTE, OWNER_READ, O import java.util.EnumSet import com.google.common.io.Files +import org.json4s.JsonAST.{JArray, JObject, JString} +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.rpc.RpcEnv -import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.Utils - class CoarseGrainedExecutorBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { + // scalastyle:off println + private def writeFile(dir: File, strToWrite: JArray): String = { + val f1 = File.createTempFile("test-resource-parser1", "", dir) + val writer1 = new PrintWriter(f1) + writer1.println(compact(render(strToWrite))) + writer1.close() + f1.getPath() + } + // scalastyle:on println + test("parsing no resources") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") @@ -48,16 +60,18 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) - - var testResourceArgs = Some("") - var error = intercept[SparkException] { - val parsedResources = backend.parseResources(testResourceArgs) - }.getMessage() - - assert(error.contains("Exception parsing the resources passed")) + withTempDir { tmpDir => + val testResourceArgs: JObject = ("" -> "") + val ja = JArray(List(testResourceArgs)) + val f1 = writeFile(tmpDir, ja) + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("Exception parsing the resources passed")) + } } - test("parsing one resources") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") @@ -68,17 +82,23 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) + withTempDir { tmpDir => + val testResourceArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) + val ja = JArray(List(testResourceArgs)) + val f1 = writeFile(tmpDir, ja) + val parsedResources = backend.parseResources(Some(f1)) - val testResourceArgs = - Some("""[{"name": "gpu", "count":2, "units":"", "addresses":["0","1"]}]""") - val parsedResources = backend.parseResources(testResourceArgs) - - assert(parsedResources.size === 1) - assert(parsedResources.get("gpu").nonEmpty) - assert(parsedResources.get("gpu").get.getName() === "gpu") - assert(parsedResources.get("gpu").get.getUnits() === "") - assert(parsedResources.get("gpu").get.getCount() === 2) - assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + assert(parsedResources.size === 1) + assert(parsedResources.get("gpu").nonEmpty) + assert(parsedResources.get("gpu").get.getName() === "gpu") + assert(parsedResources.get("gpu").get.getUnits() === "") + assert(parsedResources.get("gpu").get.getCount() === 2) + assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + } } test("parsing multiple resources") { @@ -92,23 +112,33 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) - val testResourceArgs = - Some( - """[{"name": "gpu", "count":2, "units":"", "addresses":["0","1"]}, - |{"name": "fpga", "count":3, "units":"mb", "addresses":["f1","f2","f3"]}]""".stripMargin) - val parsedResources = backend.parseResources(testResourceArgs) - - assert(parsedResources.size === 2) - assert(parsedResources.get("gpu").nonEmpty) - assert(parsedResources.get("gpu").get.getName() === "gpu") - assert(parsedResources.get("gpu").get.getUnits() === "") - assert(parsedResources.get("gpu").get.getCount() === 2) - assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) - assert(parsedResources.get("fpga").nonEmpty) - assert(parsedResources.get("fpga").get.getName() === "fpga") - assert(parsedResources.get("fpga").get.getUnits() === "mb") - assert(parsedResources.get("fpga").get.getCount() === 3) - assert(parsedResources.get("fpga").get.getAddresses().deep === Array("f1", "f2", "f3").deep) + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) + val fpgaArgs = + ("name" -> "fpga") ~ + ("units" -> "mb") ~ + ("count" -> 3) ~ + ("addresses" -> JArray(Array("f1", "f2", "f3").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs, fpgaArgs)) + val f1 = writeFile(tmpDir, ja) + val parsedResources = backend.parseResources(Some(f1)) + + assert(parsedResources.size === 2) + assert(parsedResources.get("gpu").nonEmpty) + assert(parsedResources.get("gpu").get.getName() === "gpu") + assert(parsedResources.get("gpu").get.getUnits() === "") + assert(parsedResources.get("gpu").get.getCount() === 2) + assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + assert(parsedResources.get("fpga").nonEmpty) + assert(parsedResources.get("fpga").get.getName() === "fpga") + assert(parsedResources.get("fpga").get.getUnits() === "mb") + assert(parsedResources.get("fpga").get.getCount() === 3) + assert(parsedResources.get("fpga").get.getAddresses().deep === Array("f1", "f2", "f3").deep) + } } test("use discoverer") { From 55a3bd51d7d52c34b6822797d165b28233e5c7fd Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 18 Apr 2019 09:33:28 -0500 Subject: [PATCH 07/22] update docs --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 9278f99244b9..0cea6bf1a3cd 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -236,8 +236,8 @@ of the most common options to set are: None The script the executor should run to discover a particular resource type. This should - return a string in the format of: count:unit:comma-separated list of addresses. - unit and addresses can be empty if they don't apply to the resource type. + return a JSON string in the format of the ResourceInformation class. This has a + name, count, units, and array of addresses. From eb0a8ed693502468098dff6fa021f4a701f9f4c0 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 18 Apr 2019 09:57:58 -0500 Subject: [PATCH 08/22] Add in logDebug --- .../org/apache/spark/ResourceDiscoverer.scala | 9 +++++---- .../executor/CoarseGrainedExecutorBackend.scala | 14 ++++++++++---- .../org/apache/spark/ResourceDiscovererSuite.scala | 2 -- .../CoarseGrainedExecutorBackendSuite.scala | 14 ++++---------- .../YarnCoarseGrainedExecutorBackend.scala | 6 +++--- 5 files changed, 22 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index 91765ed6aa50..49cacb788d77 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -34,10 +34,12 @@ import org.apache.spark.util.Utils.executeAndGetOutput * The output of the script it runs is expected to be JSON in the format of the * ResourceInformation class, with addresses being optional. * - * For example: {"name": "gpu","count":2, "units":"", "addresses": ["0","1"] + * For example: {"name": "gpu","count":2, "units":"", "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 @@ -65,10 +67,9 @@ private[spark] object ResourceDiscoverer extends Logging { try { val output = executeAndGetOutput(Seq(script.get), new File(".")) val parsedJson = parse(output) - implicit val formats = DefaultFormats - parsedJson.extract[ResourceInformation] + parsedJson.extract[ResourceInformation] } catch { - case e @ (_: SparkException | _: MappingException | _: JsonParseException) => + case e@(_: SparkException | _: MappingException | _: JsonParseException) => throw new SparkException(s"Error running the resource discovery script: $scriptFile" + s" for $resourceType", e) } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index eb8e8a1816a3..be69dd24475d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -91,9 +91,7 @@ private[spark] class CoarseGrainedExecutorBackend( val source = new BufferedInputStream(new FileInputStream(resourceFileStr)) val resourceMap = try { val parsedJson = parse(source).asInstanceOf[JArray].arr - val allResources = parsedJson.map(_.extract[ResourceInformation]). - map(x => (x.getName() -> x)).toMap - allResources + parsedJson.map(_.extract[ResourceInformation]).map(x => (x.getName() -> x)).toMap } catch { case e @ (_: MappingException | _: MismatchedInputException | _: ClassCastException) => throw new SparkException( @@ -108,7 +106,15 @@ private[spark] class CoarseGrainedExecutorBackend( throw new SparkException(s"User specified resources per task via: $taskConfPrefix," + s" but can't find any resources available on the executor.") } - logInfo(s"Executor ${executorId} using resources: ${resources.values}") + logInfo(s"Executor ${executorId} using resources: ${resources.keys}") + if (log.isDebugEnabled) { + logDebug("===============================================================================") + logDebug("Executor Resources:") + resources.foreach{case (k, v) => + logDebug(s"$k -> [name: ${v.getName}, units: ${v.getUnits}, count: ${v.getCount}," + + s" addresses: ${v.getAddresses().deep}]")} + logDebug("===============================================================================") + } resources } else { Map.empty[String, ResourceInformation] diff --git a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala index 8e1ef5c9faad..3553256c5721 100644 --- a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala +++ b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala @@ -28,7 +28,6 @@ import com.google.common.io.Files import org.apache.spark.internal.config._ import org.apache.spark.util.Utils - class ResourceDiscovererSuite extends SparkFunSuite with LocalSparkContext { @@ -45,7 +44,6 @@ class ResourceDiscovererSuite extends SparkFunSuite assume(!(Utils.isWindows)) withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript1") - // this is a bit ugly but do it the hardway to test out some formatting Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 12486e0c90af..75aac122ebb4 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -42,7 +42,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { // scalastyle:off println - private def writeFile(dir: File, strToWrite: JArray): String = { + private def writeFileWithJson(dir: File, strToWrite: JArray): String = { val f1 = File.createTempFile("test-resource-parser1", "", dir) val writer1 = new PrintWriter(f1) writer1.println(compact(render(strToWrite))) @@ -63,7 +63,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val testResourceArgs: JObject = ("" -> "") val ja = JArray(List(testResourceArgs)) - val f1 = writeFile(tmpDir, ja) + val f1 = writeFileWithJson(tmpDir, ja) var error = intercept[SparkException] { val parsedResources = backend.parseResources(Some(f1)) }.getMessage() @@ -75,10 +75,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing one resources") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") - val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) - // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) @@ -89,7 +87,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite ("count" -> 2) ~ ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) val ja = JArray(List(testResourceArgs)) - val f1 = writeFile(tmpDir, ja) + val f1 = writeFileWithJson(tmpDir, ja) val parsedResources = backend.parseResources(Some(f1)) assert(parsedResources.size === 1) @@ -104,10 +102,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing multiple resources") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") - val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) - // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) @@ -124,7 +120,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite ("count" -> 3) ~ ("addresses" -> JArray(Array("f1", "f2", "f3").map(JString(_)).toList)) val ja = JArray(List(gpuArgs, fpgaArgs)) - val f1 = writeFile(tmpDir, ja) + val f1 = writeFileWithJson(tmpDir, ja) val parsedResources = backend.parseResources(Some(f1)) assert(parsedResources.size === 2) @@ -144,9 +140,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("use discoverer") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") - assume(!(Utils.isWindows)) - withTempDir { dir => val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") Files.write("echo {\\\"name\\\":\\\"fpga\\\", \\\"count\\\":3, \\\"units\\\":\\\"\\\"," + diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index 757b7a65813f..380da361e5fc 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -38,7 +38,7 @@ private[spark] class YarnCoarseGrainedExecutorBackend( cores: Int, userClassPath: Seq[URL], env: SparkEnv, - resourceAddrs: Option[String]) + resourcesFile: Option[String]) extends CoarseGrainedExecutorBackend( rpcEnv, driverUrl, @@ -47,7 +47,7 @@ private[spark] class YarnCoarseGrainedExecutorBackend( cores, userClassPath, env, - resourceAddrs) with Logging { + resourcesFile) with Logging { private lazy val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(env.conf) @@ -68,7 +68,7 @@ private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourceAddrs) + arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFile) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")) From abff33f3bfea98ba80251d8cd2868165e48a044c Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 18 Apr 2019 10:03:53 -0500 Subject: [PATCH 09/22] fix spacing --- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index be69dd24475d..57520bd80f7a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -110,7 +110,7 @@ private[spark] class CoarseGrainedExecutorBackend( if (log.isDebugEnabled) { logDebug("===============================================================================") logDebug("Executor Resources:") - resources.foreach{case (k, v) => + resources.foreach{ case (k, v) => logDebug(s"$k -> [name: ${v.getName}, units: ${v.getUnits}, count: ${v.getCount}," + s" addresses: ${v.getAddresses().deep}]")} logDebug("===============================================================================") From 484a0868023646354a292b1e3c97594ccbfa967a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 30 Apr 2019 10:19:05 -0500 Subject: [PATCH 10/22] review comments --- .../scala/org/apache/spark/ResourceDiscoverer.scala | 11 +++++++---- .../scala/org/apache/spark/ResourceInformation.scala | 3 ++- .../spark/executor/CoarseGrainedExecutorBackend.scala | 8 ++++---- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index 49cacb788d77..0de8005d5ffb 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -46,9 +46,12 @@ private[spark] object ResourceDiscoverer extends Logging { } else { SPARK_EXECUTOR_RESOURCE_PREFIX } - // get unique resource types - val resourceTypes = sparkconf.getAllWithPrefix(prefix).map(x => x._1.split('.')(0)).toSet - resourceTypes.map{ rtype => { + // get unique resource types by grabbing first part config with multiple periods, + // ie resourceType.count, grab resourceType part + val resourceTypes = sparkconf.getAllWithPrefix(prefix).map { case (k, _) => + k.split('.').head + }.toSet + resourceTypes.map { rtype => { val rInfo = getResourceAddrsForType(sparkconf, prefix, rtype) (rtype -> rInfo) }}.toMap @@ -69,7 +72,7 @@ private[spark] object ResourceDiscoverer extends Logging { val parsedJson = parse(output) parsedJson.extract[ResourceInformation] } catch { - case e@(_: SparkException | _: MappingException | _: JsonParseException) => + case e @ (_: SparkException | _: MappingException | _: JsonParseException) => throw new SparkException(s"Error running the resource discovery script: $scriptFile" + s" for $resourceType", e) } diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index 3a749695a313..4f9d8d942dba 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -21,8 +21,9 @@ import org.apache.spark.annotation.Evolving /** * Class to hold information about a type of Resource. A resource could be a GPU, FPGA, etc. + * The units are resource specific and could be something like MB or GB for memory. * The array of addresses are resource specific and its up to the user to interpret the address. - * The units and addresses could be empty if they doesn't apply to that resource. + * The units and addresses could be empty if they don't apply to that resource. * * One example is GPUs, where the addresses would be the indices of the GPUs, the count would be the * number of GPUs and the units would be an empty string. diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 57520bd80f7a..f3759190b852 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -86,8 +86,8 @@ private[spark] class CoarseGrainedExecutorBackend( def parseResources(resourcesFile: Option[String]): Map[String, ResourceInformation] = { // only parse the resources if a task requires them val taskConfPrefix = SPARK_TASK_RESOURCE_PREFIX - val resourceInfo = if (env.conf.getAllWithPrefix(taskConfPrefix).size > 0) { - val resources = resourcesFile.map(resourceFileStr => { + val resourceInfo = if (env.conf.getAllWithPrefix(taskConfPrefix).nonEmpty) { + val resources = resourcesFile.map { resourceFileStr => { val source = new BufferedInputStream(new FileInputStream(resourceFileStr)) val resourceMap = try { val parsedJson = parse(source).asInstanceOf[JArray].arr @@ -100,9 +100,9 @@ private[spark] class CoarseGrainedExecutorBackend( source.close() } resourceMap - }).getOrElse(ResourceDiscoverer.findResources(env.conf, false)) + }}.getOrElse(ResourceDiscoverer.findResources(env.conf, false)) - if (resources.size == 0) { + if (resources.isEmpty) { throw new SparkException(s"User specified resources per task via: $taskConfPrefix," + s" but can't find any resources available on the executor.") } From c07b40517ea56dddad42a186c77579e4fc17d410 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 1 May 2019 09:01:47 -0500 Subject: [PATCH 11/22] remove explicit getters from ResourceInformation --- .../apache/spark/ResourceInformation.scala | 14 ++---- .../CoarseGrainedExecutorBackend.scala | 6 +-- .../spark/ResourceDiscovererSuite.scala | 48 +++++++++---------- .../CoarseGrainedExecutorBackendSuite.scala | 32 ++++++------- 4 files changed, 47 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index 4f9d8d942dba..8cff7c23d549 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -35,13 +35,7 @@ import org.apache.spark.annotation.Evolving */ @Evolving case class ResourceInformation( - private val name: String, - private val units: String, - private val count: Long, - private val addresses: Array[String] = Array.empty) { - - def getName(): String = name - def getUnits(): String = units - def getCount(): Long = count - def getAddresses(): Array[String] = addresses -} + val name: String, + val units: String, + val count: Long, + val addresses: Array[String] = Array.empty) \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index f3759190b852..18d0b77a085b 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -91,7 +91,7 @@ private[spark] class CoarseGrainedExecutorBackend( val source = new BufferedInputStream(new FileInputStream(resourceFileStr)) val resourceMap = try { val parsedJson = parse(source).asInstanceOf[JArray].arr - parsedJson.map(_.extract[ResourceInformation]).map(x => (x.getName() -> x)).toMap + parsedJson.map(_.extract[ResourceInformation]).map(x => (x.name -> x)).toMap } catch { case e @ (_: MappingException | _: MismatchedInputException | _: ClassCastException) => throw new SparkException( @@ -111,8 +111,8 @@ private[spark] class CoarseGrainedExecutorBackend( logDebug("===============================================================================") logDebug("Executor Resources:") resources.foreach{ case (k, v) => - logDebug(s"$k -> [name: ${v.getName}, units: ${v.getUnits}, count: ${v.getCount}," + - s" addresses: ${v.getAddresses().deep}]")} + logDebug(s"$k -> [name: ${v.name}, units: ${v.units}, count: ${v.count}," + + s" addresses: ${v.addresses.deep}]")} logDebug("===============================================================================") } resources diff --git a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala index 3553256c5721..0d910c32c1eb 100644 --- a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala +++ b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala @@ -54,11 +54,11 @@ class ResourceDiscovererSuite extends SparkFunSuite val resources = ResourceDiscoverer.findResources(sparkconf, false) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.getCount() == 2, "Should have 2") - assert(gpuValue.get.getName() == "gpu", "name should be gpu") - assert(gpuValue.get.getUnits() == "", "units should be empty") - assert(gpuValue.get.getAddresses().size == 2, "Should have 2 indexes") - assert(gpuValue.get.getAddresses().deep == Array("0", "1").deep, "should have 0,1 entries") + assert(gpuValue.get.count == 2, "Should have 2") + assert(gpuValue.get.name == "gpu", "name should be gpu") + assert(gpuValue.get.units == "", "units should be empty") + assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") + assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") } } @@ -76,10 +76,10 @@ class ResourceDiscovererSuite extends SparkFunSuite val resources = ResourceDiscoverer.findResources(sparkconf, false) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.getCount() == 2, "Should have 2") - assert(gpuValue.get.getName() == "gpu", "name should be gpu") - assert(gpuValue.get.getUnits() == "", "units should be empty") - assert(gpuValue.get.getAddresses().size == 0, "Should have 0 indexes") + assert(gpuValue.get.count == 2, "Should have 2") + assert(gpuValue.get.name == "gpu", "name should be gpu") + assert(gpuValue.get.units == "", "units should be empty") + assert(gpuValue.get.addresses.size == 0, "Should have 0 indexes") } } @@ -146,19 +146,19 @@ class ResourceDiscovererSuite extends SparkFunSuite assert(resources.size === 2) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.getCount() == 2, "Should have 2") - assert(gpuValue.get.getName() == "gpu", "name should be gpu") - assert(gpuValue.get.getUnits() == "", "units should be empty") - assert(gpuValue.get.getAddresses().size == 2, "Should have 2 indexes") - assert(gpuValue.get.getAddresses().deep == Array("0", "1").deep, "should have 0,1 entries") + assert(gpuValue.get.count == 2, "Should have 2") + assert(gpuValue.get.name == "gpu", "name should be gpu") + assert(gpuValue.get.units == "", "units should be empty") + assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") + assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") val fpgaValue = resources.get("fpga") assert(fpgaValue.nonEmpty, "Should have a gpu entry") - assert(fpgaValue.get.getCount() == 3, "Should have 3") - assert(fpgaValue.get.getName() == "fpga", "name should be fpga") - assert(fpgaValue.get.getUnits() == "mb", "units should be mb") - assert(fpgaValue.get.getAddresses().size == 3, "Should have 3 indexes") - assert(fpgaValue.get.getAddresses().deep == Array("f1", "f2", "f3").deep, + assert(fpgaValue.get.count == 3, "Should have 3") + assert(fpgaValue.get.name == "fpga", "name should be fpga") + assert(fpgaValue.get.units == "mb", "units should be mb") + assert(fpgaValue.get.addresses.size == 3, "Should have 3 indexes") + assert(fpgaValue.get.addresses.deep == Array("f1", "f2", "f3").deep, "should have f1,f2,f3 entries") } } @@ -180,11 +180,11 @@ class ResourceDiscovererSuite extends SparkFunSuite val resources = ResourceDiscoverer.findResources(sparkconf, true) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.getCount() == 2, "Should have 2") - assert(gpuValue.get.getName() == "gpu", "name should be gpu") - assert(gpuValue.get.getUnits() == "", "units should be empty") - assert(gpuValue.get.getAddresses().size == 2, "Should have 2 indexes") - assert(gpuValue.get.getAddresses().deep == Array("0", "1").deep, "should have 0,1 entries") + assert(gpuValue.get.count == 2, "Should have 2") + assert(gpuValue.get.name == "gpu", "name should be gpu") + assert(gpuValue.get.units == "", "units should be empty") + assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") + assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") } } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 75aac122ebb4..d567253a1c32 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -92,10 +92,10 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(parsedResources.size === 1) assert(parsedResources.get("gpu").nonEmpty) - assert(parsedResources.get("gpu").get.getName() === "gpu") - assert(parsedResources.get("gpu").get.getUnits() === "") - assert(parsedResources.get("gpu").get.getCount() === 2) - assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + assert(parsedResources.get("gpu").get.name === "gpu") + assert(parsedResources.get("gpu").get.units === "") + assert(parsedResources.get("gpu").get.count === 2) + assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1").deep) } } @@ -125,15 +125,15 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(parsedResources.size === 2) assert(parsedResources.get("gpu").nonEmpty) - assert(parsedResources.get("gpu").get.getName() === "gpu") - assert(parsedResources.get("gpu").get.getUnits() === "") - assert(parsedResources.get("gpu").get.getCount() === 2) - assert(parsedResources.get("gpu").get.getAddresses().deep === Array("0", "1").deep) + assert(parsedResources.get("gpu").get.name === "gpu") + assert(parsedResources.get("gpu").get.units === "") + assert(parsedResources.get("gpu").get.count === 2) + assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1").deep) assert(parsedResources.get("fpga").nonEmpty) - assert(parsedResources.get("fpga").get.getName() === "fpga") - assert(parsedResources.get("fpga").get.getUnits() === "mb") - assert(parsedResources.get("fpga").get.getCount() === 3) - assert(parsedResources.get("fpga").get.getAddresses().deep === Array("f1", "f2", "f3").deep) + assert(parsedResources.get("fpga").get.name === "fpga") + assert(parsedResources.get("fpga").get.units === "mb") + assert(parsedResources.get("fpga").get.count === 3) + assert(parsedResources.get("fpga").get.addresses.deep === Array("f1", "f2", "f3").deep) } } @@ -162,10 +162,10 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(parsedResources.size === 1) assert(parsedResources.get("fpga").nonEmpty) - assert(parsedResources.get("fpga").get.getName() === "fpga") - assert(parsedResources.get("fpga").get.getUnits() === "") - assert(parsedResources.get("fpga").get.getCount() === 3) - assert(parsedResources.get("fpga").get.getAddresses().deep === Array("f1", "f2", "f3").deep) + assert(parsedResources.get("fpga").get.name === "fpga") + assert(parsedResources.get("fpga").get.units === "") + assert(parsedResources.get("fpga").get.count === 3) + assert(parsedResources.get("fpga").get.addresses.deep === Array("f1", "f2", "f3").deep) } } From 01f97c88a28fa89207ccc345937fc61b9dac47b3 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 1 May 2019 09:18:59 -0500 Subject: [PATCH 12/22] add newline end of file --- core/src/main/scala/org/apache/spark/ResourceInformation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index 8cff7c23d549..f8448c7547dd 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -38,4 +38,4 @@ case class ResourceInformation( val name: String, val units: String, val count: Long, - val addresses: Array[String] = Array.empty) \ No newline at end of file + val addresses: Array[String] = Array.empty) From b5afbd0ea8229709c8fdd0a2e2f21da2d90a8c9b Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 3 May 2019 16:30:24 -0500 Subject: [PATCH 13/22] Rework to add in more checks on executor startup to make sure matches task requirements --- .../org/apache/spark/ResourceDiscoverer.scala | 6 +- .../CoarseGrainedExecutorBackend.scala | 135 ++++++++++++- .../spark/internal/config/package.scala | 1 + .../CoarseGrainedExecutorBackendSuite.scala | 186 +++++++++++++++++- 4 files changed, 312 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index 0de8005d5ffb..d6f644655d70 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.Utils.executeAndGetOutput /** * Discovers resources (GPUs/FPGAs/etc). * This class find resources by running and parses the output of the user specified script - * from the config spark.{driver/executor}.{resourceType}.discoveryScript. + * 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, with addresses being optional. * @@ -52,12 +52,12 @@ private[spark] object ResourceDiscoverer extends Logging { k.split('.').head }.toSet resourceTypes.map { rtype => { - val rInfo = getResourceAddrsForType(sparkconf, prefix, rtype) + val rInfo = getResourceInfoForType(sparkconf, prefix, rtype) (rtype -> rInfo) }}.toMap } - private def getResourceAddrsForType( + private def getResourceInfoForType( sparkconf: SparkConf, prefix: String, resourceType: String): ResourceInformation = { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 18d0b77a085b..0562da35d46b 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -82,12 +82,108 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } + // Check that the executor resources at startup will satisfy the user specified task + // requirements (spark.taks.resource.*) and that they match the executor configs + // specified by the user (spark.executor.resource.*) to catch mismatches between what + // the user requested and what resource manager gave or what the discovery script found. + private def checkExecResourcesMeetTaskRequirements( + taskResourceConfigs: Array[(String, String)], + actualExecResources: Map[String, ResourceInformation]): Unit = { + + // get just the of resource name to count + val taskResourcesAndCounts = taskResourceConfigs. + withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)}. + map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.size), v)} + + case class ResourceRealCounts(execCount: Long, taskCount: Long) + + // SPARK will only base it off the counts and known byte units, if + // user is trying to use something else we will have to add a plugin later + taskResourcesAndCounts.foreach { case (rName, taskCount) => + if (actualExecResources.contains(rName)) { + val execResourceInfo = actualExecResources(rName) + val taskUnits = env.conf.getOption( + SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_UNITS_POSTFIX) + val userExecUnitsConfigName = + SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_UNITS_POSTFIX + val userExecConfigUnits = env.conf.getOption(userExecUnitsConfigName) + val realCounts = if (execResourceInfo.units.nonEmpty) { + if (taskUnits.nonEmpty && taskUnits.get.nonEmpty) { + if (userExecConfigUnits.isEmpty || userExecConfigUnits.get.isEmpty) { + throw new SparkException(s"Resource: $rName has units in task config " + + s"and executor startup config but the user specified executor resource " + + s"config is missing the units config - see ${userExecUnitsConfigName}.") + } + try { + val execCountWithUnits = + Utils.byteStringAsBytes(execResourceInfo.count.toString + execResourceInfo.units) + val taskCountWithUnits = Utils.byteStringAsBytes(taskCount + taskUnits.get) + ResourceRealCounts(execCountWithUnits, taskCountWithUnits) + } catch { + case e: NumberFormatException => + // Ignore units not of byte types and just use count + logWarning(s"Illegal resource unit type, spark only " + + s"supports conversion of byte types, units: $execResourceInfo.units, " + + s"ignoring the type and using the raw count.", e) + ResourceRealCounts(execResourceInfo.count, taskCount.toLong) + } + } else { + throw new SparkException( + s"Resource: $rName has an executor units config: ${execResourceInfo.units}, but " + + s"the task units config is missing.") + } + } else { + if (taskUnits.nonEmpty && taskUnits.get.nonEmpty) { + throw new SparkException( + s"Resource: $rName has a task units config: ${taskUnits.get}, but the executor " + + s"units config is missing.") + } + ResourceRealCounts(execResourceInfo.count, taskCount.toLong) + } + if (realCounts.execCount < realCounts.taskCount) { + throw new SparkException(s"Executor resource: $rName, count: ${realCounts.execCount} " + + s"isn't large enough to meet task requirements of: ${realCounts.taskCount}") + } + // also make sure the executor resource count on start matches the + // spark.executor.resource configs specified by user + val userExecCountConfigName = + SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_POSTFIX + val userExecConfigCount = env.conf.getOption(userExecCountConfigName). + getOrElse(throw new SparkException(s"Executor resource: $rName not specified " + + s"via config: $userExecCountConfigName, but required " + + s"by the task, please fix your configuration")) + val execConfigCountWithUnits = if (userExecConfigUnits.nonEmpty) { + val count = try { + Utils.byteStringAsBytes(userExecConfigCount + userExecConfigUnits.get) + } catch { + case e: NumberFormatException => + // Ignore units not of byte types and just use count + logWarning(s"Illegal resource unit type, spark only " + + s"supports conversion of byte types, units: $userExecConfigUnits, " + + s"ignoring the type and using the raw count.", e) + userExecConfigCount.toLong + } + count + } else { + userExecConfigCount.toLong + } + if (execConfigCountWithUnits != realCounts.execCount) { + throw new SparkException(s"Executor resource: $rName, count: ${realCounts.execCount} " + + s"doesn't match what user requests for executor count: $execConfigCountWithUnits, " + + s"via $userExecCountConfigName") + } + } else { + throw new SparkException(s"Executor resource config missing required task resource: $rName") + } + } + } + // visible for testing def parseResources(resourcesFile: Option[String]): Map[String, ResourceInformation] = { // only parse the resources if a task requires them - val taskConfPrefix = SPARK_TASK_RESOURCE_PREFIX - val resourceInfo = if (env.conf.getAllWithPrefix(taskConfPrefix).nonEmpty) { - val resources = resourcesFile.map { resourceFileStr => { + 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 @@ -95,28 +191,47 @@ private[spark] class CoarseGrainedExecutorBackend( } catch { case e @ (_: MappingException | _: MismatchedInputException | _: ClassCastException) => throw new SparkException( - s"Exception parsing the resources passed in: $resourcesFile", e) + s"Exception parsing the resources passed in: $SPARK_TASK_RESOURCE_PREFIX", e) } finally { source.close() } resourceMap }}.getOrElse(ResourceDiscoverer.findResources(env.conf, false)) - if (resources.isEmpty) { - throw new SparkException(s"User specified resources per task via: $taskConfPrefix," + - s" but can't find any resources available on the executor.") + if (execResources.isEmpty) { + throw new SparkException(s"User specified resources per task via: " + + s"$SPARK_TASK_RESOURCE_PREFIX, but can't find any resources available on the executor.") } - logInfo(s"Executor ${executorId} using resources: ${resources.keys}") + + // check that the executor has all the resources required by the application/task + checkExecResourcesMeetTaskRequirements(taskResourceConfigs, execResources) + + // make sure the addresses make sense with count + execResources.foreach { case (rName, rInfo) => + // check to make sure we have enough addresses when any specified, if we have + // more don't worry about it + if (rInfo.addresses.nonEmpty && rInfo.addresses.size < rInfo.count) { + throw new SparkException(s"The number of resource addresses is expected to either " + + s"be >= to the count or be empty if not applicable! Resource: $rName, " + + s"count: ${rInfo.count}, number of addresses: ${rInfo.addresses.size}") + } + } + + logInfo(s"Executor ${executorId} using resources: ${execResources.keys}") if (log.isDebugEnabled) { logDebug("===============================================================================") logDebug("Executor Resources:") - resources.foreach{ case (k, v) => + execResources.foreach{ case (k, v) => logDebug(s"$k -> [name: ${v.name}, units: ${v.units}, count: ${v.count}," + s" addresses: ${v.addresses.deep}]")} logDebug("===============================================================================") } - resources + 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 diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 0aed1af023f8..6ba1a8553763 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -35,6 +35,7 @@ package object config { private[spark] val SPARK_TASK_RESOURCE_PREFIX = "spark.task.resource." private[spark] val SPARK_RESOURCE_COUNT_POSTFIX = ".count" + private[spark] val SPARK_RESOURCE_UNITS_POSTFIX = ".units" private[spark] val SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX = ".discoveryScript" private[spark] val DRIVER_CLASS_PATH = diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index d567253a1c32..119e4743cc9c 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -74,6 +74,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing one resources") { val conf = new SparkConf + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) @@ -101,6 +102,11 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing multiple resources") { val conf = new SparkConf + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "3") + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_UNITS_POSTFIX, "gb") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "1024") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_UNITS_POSTFIX, "mb") + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) @@ -116,7 +122,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) val fpgaArgs = ("name" -> "fpga") ~ - ("units" -> "mb") ~ + ("units" -> "gb") ~ ("count" -> 3) ~ ("addresses" -> JArray(Array("f1", "f2", "f3").map(JString(_)).toList)) val ja = JArray(List(gpuArgs, fpgaArgs)) @@ -131,15 +137,189 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1").deep) assert(parsedResources.get("fpga").nonEmpty) assert(parsedResources.get("fpga").get.name === "fpga") - assert(parsedResources.get("fpga").get.units === "mb") + assert(parsedResources.get("fpga").get.units === "gb") assert(parsedResources.get("fpga").get.count === 3) assert(parsedResources.get("fpga").get.addresses.deep === Array("f1", "f2", "f3").deep) } } - test("use discoverer") { + test("error checking parsing resources and executor and task configs") { + val conf = new SparkConf + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + // not enough gpu's on the executor + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 1) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("isn't large enough to meet task requirements")) + } + + // missing resource on the executor + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "fpga") ~ + ("units" -> "") ~ + ("count" -> 1) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("Executor resource config missing required task resource")) + } + + // extra unit in executor config + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "m") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("the task units config is missing")) + } + + // number of addresses with count >, this is ok + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0", "1", "2").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + val parsedResources = backend.parseResources(Some(f1)) + assert(parsedResources.size === 1) + assert(parsedResources.get("gpu").nonEmpty) + assert(parsedResources.get("gpu").get.name === "gpu") + assert(parsedResources.get("gpu").get.units === "") + assert(parsedResources.get("gpu").get.count === 2) + assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1", "2").deep) + } + + // count of gpu's > then user executor config + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 3) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("gpu, count: 3 doesn't match what user requests for executor count: 2")) + } + + // number of addresses mismatch with count < + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("The number of resource addresses is expected to " + + "either be >= to the count or be empty if not applicable")) + } + } + + test("parsing resources task configs with units missing executor units") { + val conf = new SparkConf + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_UNITS_POSTFIX, "g") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + // executor config doesn't have units on gpu and task one does + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("executor units config is missing")) + } + } + + test("parsing resources task configs with missing executor count config") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + // executor config doesn't have units on gpu and task one does + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("units" -> "") ~ + ("count" -> 2) ~ + ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseResources(Some(f1)) + }.getMessage() + + assert(error.contains("Executor resource: gpu not specified via config: " + + "spark.executor.resource.gpu.count, but required by the task, please " + + "fix your configuration")) + } + } + + test("use discoverer") { + val conf = new SparkConf + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "3") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "3") assume(!(Utils.isWindows)) withTempDir { dir => val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") From dd5b7653f1b22a72b602fbc9619daafdb20d5094 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 May 2019 15:05:14 -0500 Subject: [PATCH 14/22] rework --- .../CoarseGrainedExecutorBackend.scala | 52 +++++++++---------- .../CoarseGrainedExecutorBackendSuite.scala | 2 +- 2 files changed, 26 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 0562da35d46b..e049aa1eecbe 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -82,15 +82,31 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } + // we only support converting from units that are byte based, this will + // either convert it to bytes or just return the count if the units can't be + // parsed as bytes. + private def tryConvertUnitsToBytes(count: String, units: String): Long = { + try { + Utils.byteStringAsBytes(count + units) + } catch { + case e: NumberFormatException => + // Ignore units not of byte types and just use count + logWarning(s"Illegal resource unit type, spark only " + + s"supports conversion of byte types, units: $units, " + + s"ignoring the type and using the raw count.", e) + count.toLong + } + } + // Check that the executor resources at startup will satisfy the user specified task - // requirements (spark.taks.resource.*) and that they match the executor configs + // requirements (spark.task.resource.*) and that they match the executor configs // specified by the user (spark.executor.resource.*) to catch mismatches between what // the user requested and what resource manager gave or what the discovery script found. private def checkExecResourcesMeetTaskRequirements( taskResourceConfigs: Array[(String, String)], actualExecResources: Map[String, ResourceInformation]): Unit = { - // get just the of resource name to count + // get just the map of resource name to count val taskResourcesAndCounts = taskResourceConfigs. withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)}. map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.size), v)} @@ -114,19 +130,11 @@ private[spark] class CoarseGrainedExecutorBackend( s"and executor startup config but the user specified executor resource " + s"config is missing the units config - see ${userExecUnitsConfigName}.") } - try { - val execCountWithUnits = - Utils.byteStringAsBytes(execResourceInfo.count.toString + execResourceInfo.units) - val taskCountWithUnits = Utils.byteStringAsBytes(taskCount + taskUnits.get) - ResourceRealCounts(execCountWithUnits, taskCountWithUnits) - } catch { - case e: NumberFormatException => - // Ignore units not of byte types and just use count - logWarning(s"Illegal resource unit type, spark only " + - s"supports conversion of byte types, units: $execResourceInfo.units, " + - s"ignoring the type and using the raw count.", e) - ResourceRealCounts(execResourceInfo.count, taskCount.toLong) - } + val execCountWithUnits = + tryConvertUnitsToBytes(execResourceInfo.count.toString, execResourceInfo.units) + val taskCountWithUnits = + tryConvertUnitsToBytes(taskCount, taskUnits.get) + ResourceRealCounts(execCountWithUnits, taskCountWithUnits) } else { throw new SparkException( s"Resource: $rName has an executor units config: ${execResourceInfo.units}, but " + @@ -153,17 +161,7 @@ private[spark] class CoarseGrainedExecutorBackend( s"via config: $userExecCountConfigName, but required " + s"by the task, please fix your configuration")) val execConfigCountWithUnits = if (userExecConfigUnits.nonEmpty) { - val count = try { - Utils.byteStringAsBytes(userExecConfigCount + userExecConfigUnits.get) - } catch { - case e: NumberFormatException => - // Ignore units not of byte types and just use count - logWarning(s"Illegal resource unit type, spark only " + - s"supports conversion of byte types, units: $userExecConfigUnits, " + - s"ignoring the type and using the raw count.", e) - userExecConfigCount.toLong - } - count + tryConvertUnitsToBytes(userExecConfigCount, userExecConfigUnits.get) } else { userExecConfigCount.toLong } @@ -191,7 +189,7 @@ private[spark] class CoarseGrainedExecutorBackend( } catch { case e @ (_: MappingException | _: MismatchedInputException | _: ClassCastException) => throw new SparkException( - s"Exception parsing the resources passed in: $SPARK_TASK_RESOURCE_PREFIX", e) + s"Exception parsing the resources in $resourceFileStr", e) } finally { source.close() } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 119e4743cc9c..023a1f154058 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -68,7 +68,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseResources(Some(f1)) }.getMessage() - assert(error.contains("Exception parsing the resources passed")) + assert(error.contains("Exception parsing the resources in")) } } From a9642b8b5f0de1d6a4727ff1a369322495f659c0 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 7 May 2019 09:24:25 -0500 Subject: [PATCH 15/22] fix extra string interpolation --- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e049aa1eecbe..3723de1f3a36 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -91,9 +91,9 @@ private[spark] class CoarseGrainedExecutorBackend( } catch { case e: NumberFormatException => // Ignore units not of byte types and just use count - logWarning(s"Illegal resource unit type, spark only " + + logWarning("Illegal resource unit type, spark only " + s"supports conversion of byte types, units: $units, " + - s"ignoring the type and using the raw count.", e) + "ignoring the type and using the raw count.", e) count.toLong } } From 01a6061b5d8d13f206fdcae03ed41efc85a9b137 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 May 2019 13:20:23 -0500 Subject: [PATCH 16/22] rework - remove units and count field from ResourceInformation, no longer a case class, remove sample script --- .../org/apache/spark/ResourceDiscoverer.scala | 27 ++-- .../apache/spark/ResourceInformation.scala | 23 ++- .../CoarseGrainedExecutorBackend.scala | 86 +++-------- .../spark/internal/config/package.scala | 1 - .../spark/ResourceDiscovererSuite.scala | 62 +------- .../CoarseGrainedExecutorBackendSuite.scala | 143 ++---------------- docs/configuration.md | 4 +- .../src/main/resources/getGpuResources.sh | 33 ---- 8 files changed, 67 insertions(+), 312 deletions(-) delete mode 100755 examples/src/main/resources/getGpuResources.sh diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index d6f644655d70..ed5297dae9fe 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -22,25 +22,26 @@ import java.io.File import com.fasterxml.jackson.core.JsonParseException import org.json4s.{DefaultFormats, MappingException} import org.json4s.jackson.JsonMethods._ - import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.util.Utils.executeAndGetOutput +import org.json4s.JsonAST.{JObject, JValue} /** - * Discovers resources (GPUs/FPGAs/etc). + * Discovers resources (GPUs/FPGAs/etc). It currently only supports resources that have + * addresses. * This class find resources by running and parses 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, with addresses being optional. + * ResourceInformation class. * - * For example: {"name": "gpu","count":2, "units":"", "addresses": ["0","1"]} + * 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] = { + def findResources(sparkConf: SparkConf, isDriver: Boolean): Map[String, ResourceInformation] = { val prefix = if (isDriver) { SPARK_DRIVER_RESOURCE_PREFIX } else { @@ -48,21 +49,21 @@ private[spark] object ResourceDiscoverer extends Logging { } // get unique resource types by grabbing first part config with multiple periods, // ie resourceType.count, grab resourceType part - val resourceTypes = sparkconf.getAllWithPrefix(prefix).map { case (k, _) => + val resourceNames = sparkConf.getAllWithPrefix(prefix).map { case (k, _) => k.split('.').head }.toSet - resourceTypes.map { rtype => { - val rInfo = getResourceInfoForType(sparkconf, prefix, rtype) - (rtype -> rInfo) + resourceNames.map { rName => { + val rInfo = getResourceInfoForType(sparkConf, prefix, rName) + (rName -> rInfo) }}.toMap } private def getResourceInfoForType( - sparkconf: SparkConf, + sparkConf: SparkConf, prefix: String, resourceType: String): ResourceInformation = { val discoveryConf = prefix + resourceType + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX - val script = sparkconf.getOption(discoveryConf) + val script = sparkConf.getOption(discoveryConf) val result = if (script.nonEmpty) { val scriptFile = new File(script.get) // check that script exists and try to execute @@ -70,7 +71,9 @@ private[spark] object ResourceDiscoverer extends Logging { try { val output = executeAndGetOutput(Seq(script.get), new File(".")) val parsedJson = parse(output) - parsedJson.extract[ResourceInformation] + val name = (parsedJson \ "name").extract[String] + val addresses = (parsedJson \ "addresses").extract[Array[JValue]].map(_.extract[String]) + new ResourceInformation(name, addresses) } catch { case e @ (_: SparkException | _: MappingException | _: JsonParseException) => throw new SparkException(s"Error running the resource discovery script: $scriptFile" + diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index f8448c7547dd..ec10cac2ff45 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -21,21 +21,20 @@ import org.apache.spark.annotation.Evolving /** * Class to hold information about a type of Resource. A resource could be a GPU, FPGA, etc. - * The units are resource specific and could be something like MB or GB for memory. * The array of addresses are resource specific and its up to the user to interpret the address. - * The units and addresses could be empty if they don't apply to that resource. * - * One example is GPUs, where the addresses would be the indices of the GPUs, the count would be the - * number of GPUs and the units would be an empty string. + * One example is GPUs, where the addresses would be the indices of the GPUs * * @param name the name of the resource - * @param units the units of the resources, can be an empty string if units don't apply - * @param count the number of resources available - * @param addresses an optional array of strings describing the addresses of the resource + * @param addresses an array of strings describing the addresses of the resource */ @Evolving -case class ResourceInformation( - val name: String, - val units: String, - val count: Long, - val addresses: Array[String] = Array.empty) +class ResourceInformation( + private val _name: String, + private val _addresses: Array[String]) extends Serializable { + + def name: String = _name + def addresses: Array[String] = _addresses + + override def toString: String = s"[name: ${name}, addresses: ${_addresses.mkString(",")}]" +} diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3723de1f3a36..d9f3d11a28e7 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import com.fasterxml.jackson.databind.exc.MismatchedInputException import org.json4s.DefaultFormats -import org.json4s.JsonAST.JArray +import org.json4s.JsonAST.{JArray, JValue} import org.json4s.MappingException import org.json4s.jackson.JsonMethods._ @@ -72,7 +72,7 @@ private[spark] class CoarseGrainedExecutorBackend( // 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, parseResources(resourcesFile))) + extractAttributes, parseOrFindResources(resourcesFile))) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -111,46 +111,14 @@ private[spark] class CoarseGrainedExecutorBackend( withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)}. map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.size), v)} - case class ResourceRealCounts(execCount: Long, taskCount: Long) - - // SPARK will only base it off the counts and known byte units, if - // user is trying to use something else we will have to add a plugin later - taskResourcesAndCounts.foreach { case (rName, taskCount) => + taskResourcesAndCounts.foreach { case (rName, taskReqCount) => if (actualExecResources.contains(rName)) { val execResourceInfo = actualExecResources(rName) - val taskUnits = env.conf.getOption( - SPARK_TASK_RESOURCE_PREFIX + rName + SPARK_RESOURCE_UNITS_POSTFIX) - val userExecUnitsConfigName = - SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_UNITS_POSTFIX - val userExecConfigUnits = env.conf.getOption(userExecUnitsConfigName) - val realCounts = if (execResourceInfo.units.nonEmpty) { - if (taskUnits.nonEmpty && taskUnits.get.nonEmpty) { - if (userExecConfigUnits.isEmpty || userExecConfigUnits.get.isEmpty) { - throw new SparkException(s"Resource: $rName has units in task config " + - s"and executor startup config but the user specified executor resource " + - s"config is missing the units config - see ${userExecUnitsConfigName}.") - } - val execCountWithUnits = - tryConvertUnitsToBytes(execResourceInfo.count.toString, execResourceInfo.units) - val taskCountWithUnits = - tryConvertUnitsToBytes(taskCount, taskUnits.get) - ResourceRealCounts(execCountWithUnits, taskCountWithUnits) - } else { - throw new SparkException( - s"Resource: $rName has an executor units config: ${execResourceInfo.units}, but " + - s"the task units config is missing.") - } - } else { - if (taskUnits.nonEmpty && taskUnits.get.nonEmpty) { - throw new SparkException( - s"Resource: $rName has a task units config: ${taskUnits.get}, but the executor " + - s"units config is missing.") - } - ResourceRealCounts(execResourceInfo.count, taskCount.toLong) - } - if (realCounts.execCount < realCounts.taskCount) { - throw new SparkException(s"Executor resource: $rName, count: ${realCounts.execCount} " + - s"isn't large enough to meet task requirements of: ${realCounts.taskCount}") + + if (execResourceInfo.addresses.size < taskReqCount.toLong) { + throw new SparkException(s"Executor resource: $rName with addresses: " + + s"${execResourceInfo.addresses.mkString(",")} doesn't meet the task " + + s"requirements of needing $taskReqCount of them") } // also make sure the executor resource count on start matches the // spark.executor.resource configs specified by user @@ -160,14 +128,11 @@ private[spark] class CoarseGrainedExecutorBackend( getOrElse(throw new SparkException(s"Executor resource: $rName not specified " + s"via config: $userExecCountConfigName, but required " + s"by the task, please fix your configuration")) - val execConfigCountWithUnits = if (userExecConfigUnits.nonEmpty) { - tryConvertUnitsToBytes(userExecConfigCount, userExecConfigUnits.get) - } else { - userExecConfigCount.toLong - } - if (execConfigCountWithUnits != realCounts.execCount) { - throw new SparkException(s"Executor resource: $rName, count: ${realCounts.execCount} " + - s"doesn't match what user requests for executor count: $execConfigCountWithUnits, " + + + if (userExecConfigCount.toLong != execResourceInfo.addresses.size) { + throw new SparkException(s"Executor resource: $rName, with addresses: " + + s"${execResourceInfo.addresses.mkString(",")} " + + s"doesn't match what the user requested for executor count: $userExecConfigCount, " + s"via $userExecCountConfigName") } } else { @@ -177,7 +142,7 @@ private[spark] class CoarseGrainedExecutorBackend( } // visible for testing - def parseResources(resourcesFile: Option[String]): Map[String, ResourceInformation] = { + 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) { @@ -185,9 +150,13 @@ private[spark] class CoarseGrainedExecutorBackend( val source = new BufferedInputStream(new FileInputStream(resourceFileStr)) val resourceMap = try { val parsedJson = parse(source).asInstanceOf[JArray].arr - parsedJson.map(_.extract[ResourceInformation]).map(x => (x.name -> x)).toMap + parsedJson.map { json => + val name = (json \ "name").extract[String] + val addresses = (json \ "addresses").extract[Array[JValue]].map(_.extract[String]) + new ResourceInformation(name, addresses) + }.map(x => (x.name -> x)).toMap } catch { - case e @ (_: MappingException | _: MismatchedInputException | _: ClassCastException) => + case e @ (_: MappingException | _: MismatchedInputException) => throw new SparkException( s"Exception parsing the resources in $resourceFileStr", e) } finally { @@ -204,24 +173,11 @@ private[spark] class CoarseGrainedExecutorBackend( // check that the executor has all the resources required by the application/task checkExecResourcesMeetTaskRequirements(taskResourceConfigs, execResources) - // make sure the addresses make sense with count - execResources.foreach { case (rName, rInfo) => - // check to make sure we have enough addresses when any specified, if we have - // more don't worry about it - if (rInfo.addresses.nonEmpty && rInfo.addresses.size < rInfo.count) { - throw new SparkException(s"The number of resource addresses is expected to either " + - s"be >= to the count or be empty if not applicable! Resource: $rName, " + - s"count: ${rInfo.count}, number of addresses: ${rInfo.addresses.size}") - } - } - logInfo(s"Executor ${executorId} using resources: ${execResources.keys}") if (log.isDebugEnabled) { logDebug("===============================================================================") logDebug("Executor Resources:") - execResources.foreach{ case (k, v) => - logDebug(s"$k -> [name: ${v.name}, units: ${v.units}, count: ${v.count}," + - s" addresses: ${v.addresses.deep}]")} + execResources.foreach { case (k, v) => logDebug(s"$k -> $v") } logDebug("===============================================================================") } execResources diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6ba1a8553763..0aed1af023f8 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -35,7 +35,6 @@ package object config { private[spark] val SPARK_TASK_RESOURCE_PREFIX = "spark.task.resource." private[spark] val SPARK_RESOURCE_COUNT_POSTFIX = ".count" - private[spark] val SPARK_RESOURCE_UNITS_POSTFIX = ".units" private[spark] val SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX = ".discoveryScript" private[spark] val DRIVER_CLASS_PATH = diff --git a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala index 0d910c32c1eb..a30ba66dd042 100644 --- a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala +++ b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala @@ -45,7 +45,7 @@ class ResourceDiscovererSuite extends SparkFunSuite withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript1") // this is a bit ugly but do it the hardway to test out some formatting - Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + + Files.write("echo {\\\"name\\\":\\\"gpu\\\"," + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) @@ -54,20 +54,19 @@ class ResourceDiscovererSuite extends SparkFunSuite val resources = ResourceDiscoverer.findResources(sparkconf, false) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.count == 2, "Should have 2") assert(gpuValue.get.name == "gpu", "name should be gpu") - assert(gpuValue.get.units == "", "units should be empty") assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") } } - test("Resource discoverer no addresses") { + // TODO + test("Resource discoverer no addresses errors") { val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"}", + Files.write("echo {\\\"name\\\":\\\"gpu\\\"}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) @@ -76,63 +75,23 @@ class ResourceDiscovererSuite extends SparkFunSuite val resources = ResourceDiscoverer.findResources(sparkconf, false) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.count == 2, "Should have 2") assert(gpuValue.get.name == "gpu", "name should be gpu") - assert(gpuValue.get.units == "", "units should be empty") assert(gpuValue.get.addresses.size == 0, "Should have 0 indexes") } } - test("Resource discoverer no count") { - val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => - val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"units\\\":\\\"\\\"," + - " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(file1.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) - sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - val error = intercept[SparkException] { - ResourceDiscoverer.findResources(sparkconf, false) - }.getMessage() - - assert(error.contains("Error running the resource discovery")) - } - } - - test("Resource discoverer no count value") { - val sparkconf = new SparkConf - assume(!(Utils.isWindows)) - withTempDir { dir => - val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":, \\\"units\\\":\\\"\\\"," + - " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(file1.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) - sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - val error = intercept[SparkException] { - ResourceDiscoverer.findResources(sparkconf, false) - }.getMessage() - - assert(error.contains("Error running the resource discovery")) - } - } - test("Resource discoverer multiple resource types") { val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => val gpuDiscovery = new File(dir, "resourceDiscoverScriptgpu") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + + Files.write("echo {\\\"name\\\":\\\"gpu\\\", " + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", gpuDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(gpuDiscovery.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") - Files.write("echo {\\\"name\\\":\\\"fpga\\\", \\\"count\\\":3, \\\"units\\\":\\\"mb\\\"," + + Files.write("echo {\\\"name\\\":\\\"fpga\\\"," + " \\\"addresses\\\":[\\\"f1\\\",\\\"f2\\\",\\\"f3\\\"]}", fpgaDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), @@ -146,17 +105,13 @@ class ResourceDiscovererSuite extends SparkFunSuite assert(resources.size === 2) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.count == 2, "Should have 2") assert(gpuValue.get.name == "gpu", "name should be gpu") - assert(gpuValue.get.units == "", "units should be empty") assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") val fpgaValue = resources.get("fpga") assert(fpgaValue.nonEmpty, "Should have a gpu entry") - assert(fpgaValue.get.count == 3, "Should have 3") assert(fpgaValue.get.name == "fpga", "name should be fpga") - assert(fpgaValue.get.units == "mb", "units should be mb") assert(fpgaValue.get.addresses.size == 3, "Should have 3 indexes") assert(fpgaValue.get.addresses.deep == Array("f1", "f2", "f3").deep, "should have f1,f2,f3 entries") @@ -168,7 +123,7 @@ class ResourceDiscovererSuite extends SparkFunSuite assume(!(Utils.isWindows)) withTempDir { dir => val file1 = new File(dir, "resourceDiscoverScript2") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", \\\"count\\\":2, \\\"units\\\":\\\"\\\"," + + Files.write("echo {\\\"name\\\":\\\"gpu\\\", " + " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(file1.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) @@ -180,12 +135,9 @@ class ResourceDiscovererSuite extends SparkFunSuite val resources = ResourceDiscoverer.findResources(sparkconf, true) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") - assert(gpuValue.get.count == 2, "Should have 2") assert(gpuValue.get.name == "gpu", "name should be gpu") - assert(gpuValue.get.units == "", "units should be empty") assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") - } } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 023a1f154058..01e1000548c7 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -65,7 +65,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val ja = JArray(List(testResourceArgs)) val f1 = writeFileWithJson(tmpDir, ja) var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) + val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() assert(error.contains("Exception parsing the resources in")) @@ -84,18 +84,14 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val testResourceArgs = ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 2) ~ ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) val ja = JArray(List(testResourceArgs)) val f1 = writeFileWithJson(tmpDir, ja) - val parsedResources = backend.parseResources(Some(f1)) + val parsedResources = backend.parseOrFindResources(Some(f1)) assert(parsedResources.size === 1) assert(parsedResources.get("gpu").nonEmpty) assert(parsedResources.get("gpu").get.name === "gpu") - assert(parsedResources.get("gpu").get.units === "") - assert(parsedResources.get("gpu").get.count === 2) assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1").deep) } } @@ -103,9 +99,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing multiple resources") { val conf = new SparkConf conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "3") - conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_UNITS_POSTFIX, "gb") - conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "1024") - conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_UNITS_POSTFIX, "mb") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "fpga" + SPARK_RESOURCE_COUNT_POSTFIX, "3") conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") val serializer = new JavaSerializer(conf) @@ -117,28 +111,20 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val gpuArgs = ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 2) ~ ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) val fpgaArgs = ("name" -> "fpga") ~ - ("units" -> "gb") ~ - ("count" -> 3) ~ ("addresses" -> JArray(Array("f1", "f2", "f3").map(JString(_)).toList)) val ja = JArray(List(gpuArgs, fpgaArgs)) val f1 = writeFileWithJson(tmpDir, ja) - val parsedResources = backend.parseResources(Some(f1)) + val parsedResources = backend.parseOrFindResources(Some(f1)) assert(parsedResources.size === 2) assert(parsedResources.get("gpu").nonEmpty) assert(parsedResources.get("gpu").get.name === "gpu") - assert(parsedResources.get("gpu").get.units === "") - assert(parsedResources.get("gpu").get.count === 2) assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1").deep) assert(parsedResources.get("fpga").nonEmpty) assert(parsedResources.get("fpga").get.name === "fpga") - assert(parsedResources.get("fpga").get.units === "gb") - assert(parsedResources.get("fpga").get.count === 3) assert(parsedResources.get("fpga").get.addresses.deep === Array("f1", "f2", "f3").deep) } } @@ -157,134 +143,31 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val gpuArgs = ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 1) ~ ("addresses" -> JArray(Array("0").map(JString(_)).toList)) val ja = JArray(List(gpuArgs)) val f1 = writeFileWithJson(tmpDir, ja) var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) + val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("isn't large enough to meet task requirements")) + assert(error.contains("doesn't meet the task requirements of needing")) } // missing resource on the executor withTempDir { tmpDir => val gpuArgs = ("name" -> "fpga") ~ - ("units" -> "") ~ - ("count" -> 1) ~ ("addresses" -> JArray(Array("0").map(JString(_)).toList)) val ja = JArray(List(gpuArgs)) val f1 = writeFileWithJson(tmpDir, ja) var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) + val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() assert(error.contains("Executor resource config missing required task resource")) } - - // extra unit in executor config - withTempDir { tmpDir => - val gpuArgs = - ("name" -> "gpu") ~ - ("units" -> "m") ~ - ("count" -> 2) ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) - val ja = JArray(List(gpuArgs)) - val f1 = writeFileWithJson(tmpDir, ja) - - var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) - }.getMessage() - - assert(error.contains("the task units config is missing")) - } - - // number of addresses with count >, this is ok - withTempDir { tmpDir => - val gpuArgs = - ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 2) ~ - ("addresses" -> JArray(Array("0", "1", "2").map(JString(_)).toList)) - val ja = JArray(List(gpuArgs)) - val f1 = writeFileWithJson(tmpDir, ja) - val parsedResources = backend.parseResources(Some(f1)) - assert(parsedResources.size === 1) - assert(parsedResources.get("gpu").nonEmpty) - assert(parsedResources.get("gpu").get.name === "gpu") - assert(parsedResources.get("gpu").get.units === "") - assert(parsedResources.get("gpu").get.count === 2) - assert(parsedResources.get("gpu").get.addresses.deep === Array("0", "1", "2").deep) - } - - // count of gpu's > then user executor config - withTempDir { tmpDir => - val gpuArgs = - ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 3) ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) - val ja = JArray(List(gpuArgs)) - val f1 = writeFileWithJson(tmpDir, ja) - - var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) - }.getMessage() - - assert(error.contains("gpu, count: 3 doesn't match what user requests for executor count: 2")) - } - - // number of addresses mismatch with count < - withTempDir { tmpDir => - val gpuArgs = - ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 2) ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) - val ja = JArray(List(gpuArgs)) - val f1 = writeFileWithJson(tmpDir, ja) - - var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) - }.getMessage() - - assert(error.contains("The number of resource addresses is expected to " + - "either be >= to the count or be empty if not applicable")) - } - } - - test("parsing resources task configs with units missing executor units") { - val conf = new SparkConf - conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") - conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") - conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_UNITS_POSTFIX, "g") - val serializer = new JavaSerializer(conf) - val env = createMockEnv(conf, serializer) - // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", - 4, Seq.empty[URL], env, None) - - // executor config doesn't have units on gpu and task one does - withTempDir { tmpDir => - val gpuArgs = - ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 2) ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) - val ja = JArray(List(gpuArgs)) - val f1 = writeFileWithJson(tmpDir, ja) - - var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) - }.getMessage() - - assert(error.contains("executor units config is missing")) - } } test("parsing resources task configs with missing executor count config") { @@ -300,14 +183,12 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val gpuArgs = ("name" -> "gpu") ~ - ("units" -> "") ~ - ("count" -> 2) ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) val ja = JArray(List(gpuArgs)) val f1 = writeFileWithJson(tmpDir, ja) var error = intercept[SparkException] { - val parsedResources = backend.parseResources(Some(f1)) + val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() assert(error.contains("Executor resource: gpu not specified via config: " + @@ -323,7 +204,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assume(!(Utils.isWindows)) withTempDir { dir => val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") - Files.write("echo {\\\"name\\\":\\\"fpga\\\", \\\"count\\\":3, \\\"units\\\":\\\"\\\"," + + Files.write("echo {\\\"name\\\":\\\"fpga\\\", " + " \\\"addresses\\\":[\\\"f1\\\",\\\"f2\\\",\\\"f3\\\"]}", fpgaDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), @@ -338,13 +219,11 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) - val parsedResources = backend.parseResources(None) + val parsedResources = backend.parseOrFindResources(None) assert(parsedResources.size === 1) assert(parsedResources.get("fpga").nonEmpty) assert(parsedResources.get("fpga").get.name === "fpga") - assert(parsedResources.get("fpga").get.units === "") - assert(parsedResources.get("fpga").get.count === 3) assert(parsedResources.get("fpga").get.addresses.deep === Array("f1", "f2", "f3").deep) } } diff --git a/docs/configuration.md b/docs/configuration.md index 4f409b5eaf65..5aeaccd0afaf 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -236,8 +236,8 @@ of the most common options to set are: None The script the executor should run to discover a particular resource type. This should - return a JSON string in the format of the ResourceInformation class. This has a - name, count, units, and array of addresses. + write to STDOUT a JSON string in the format of the ResourceInformation class. This has a + name and array of addresses. diff --git a/examples/src/main/resources/getGpuResources.sh b/examples/src/main/resources/getGpuResources.sh deleted file mode 100755 index 64bdf1aff7be..000000000000 --- a/examples/src/main/resources/getGpuResources.sh +++ /dev/null @@ -1,33 +0,0 @@ -# -# 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. -# - -# -# This is an example script that can be used to discover GPUs. It only works on NVIDIA GPUs since it -# uses the nvidia-smi command. This script will find all visible GPUs, so if you aren't running -# in an environment that can isolate GPUs to an executor and where multiple executors can run on a -# single node you may not want to use this. See your cluster manager specific configs for other -# options. -# -# It can be passed into Spark via the configs spark.executor.resource.gpu.discoveryScript and/or -# spark.driver.resource.gpu.discoveryScript. -# The script will return a JSON string in the format of the ResourceInformation class. -# - -ADDRS=`nvidia-smi --query-gpu=index --format=csv,noheader | sed 'N;s/\n/\",\"/'` -COUNT=`echo $ADDRS | tr -cd , | wc -c` -ALLCOUNT=`expr $COUNT + 1` -echo {\"name\": \"gpu\", \"count\":$ALLCOUNT, \"units\":\"\", \"addresses\":[\"$ADDRS\"]} From 9187c6d735877684bdd741622e3fc7eda0cd235d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 May 2019 13:26:57 -0500 Subject: [PATCH 17/22] Fix import order --- core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index ed5297dae9fe..a3e1b5c8f2e5 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -21,11 +21,12 @@ 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 -import org.json4s.JsonAST.{JObject, JValue} /** * Discovers resources (GPUs/FPGAs/etc). It currently only supports resources that have From 636fef8f9231dfd9586fceebb1e0f2fb5b5168e0 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 May 2019 13:29:01 -0500 Subject: [PATCH 18/22] fix comment syntax --- core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index a3e1b5c8f2e5..3f2ecba0f5c8 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils.executeAndGetOutput /** * Discovers resources (GPUs/FPGAs/etc). It currently only supports resources that have * addresses. - * This class find resources by running and parses the output of the user specified script + * 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. From 3d8e5cd3d846176d8d049f9b4511ffa6149ef837 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 9 May 2019 13:30:37 -0500 Subject: [PATCH 19/22] remove extra convert units function --- .../executor/CoarseGrainedExecutorBackend.scala | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index d9f3d11a28e7..3e4b19f33a60 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -82,22 +82,6 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } - // we only support converting from units that are byte based, this will - // either convert it to bytes or just return the count if the units can't be - // parsed as bytes. - private def tryConvertUnitsToBytes(count: String, units: String): Long = { - try { - Utils.byteStringAsBytes(count + units) - } catch { - case e: NumberFormatException => - // Ignore units not of byte types and just use count - logWarning("Illegal resource unit type, spark only " + - s"supports conversion of byte types, units: $units, " + - "ignoring the type and using the raw count.", e) - count.toLong - } - } - // Check that the executor resources at startup will satisfy the user specified task // requirements (spark.task.resource.*) and that they match the executor configs // specified by the user (spark.executor.resource.*) to catch mismatches between what From 4165c60955fbacc65399b723f81f3350740d7c4d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 10 May 2019 15:12:07 -0500 Subject: [PATCH 20/22] rework --- .../org/apache/spark/ResourceDiscoverer.scala | 2 +- .../apache/spark/ResourceInformation.scala | 9 +- .../CoarseGrainedExecutorBackend.scala | 95 +++++++------ .../spark/ResourceDiscovererSuite.scala | 132 ++++++++---------- .../CoarseGrainedExecutorBackendSuite.scala | 61 +++++--- 5 files changed, 152 insertions(+), 147 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala index 3f2ecba0f5c8..19639420b8b9 100644 --- a/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala +++ b/core/src/main/scala/org/apache/spark/ResourceDiscoverer.scala @@ -73,7 +73,7 @@ private[spark] object ResourceDiscoverer extends Logging { val output = executeAndGetOutput(Seq(script.get), new File(".")) val parsedJson = parse(output) val name = (parsedJson \ "name").extract[String] - val addresses = (parsedJson \ "addresses").extract[Array[JValue]].map(_.extract[String]) + val addresses = (parsedJson \ "addresses").extract[Array[String]].toArray new ResourceInformation(name, addresses) } catch { case e @ (_: SparkException | _: MappingException | _: JsonParseException) => diff --git a/core/src/main/scala/org/apache/spark/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/ResourceInformation.scala index ec10cac2ff45..6a5b725ac21d 100644 --- a/core/src/main/scala/org/apache/spark/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/ResourceInformation.scala @@ -30,11 +30,8 @@ import org.apache.spark.annotation.Evolving */ @Evolving class ResourceInformation( - private val _name: String, - private val _addresses: Array[String]) extends Serializable { + val name: String, + val addresses: Array[String]) extends Serializable { - def name: String = _name - def addresses: Array[String] = _addresses - - override def toString: String = s"[name: ${name}, addresses: ${_addresses.mkString(",")}]" + override def toString: String = s"[name: ${name}, addresses: ${addresses.mkString(",")}]" } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3e4b19f33a60..330ce41c03bf 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import com.fasterxml.jackson.databind.exc.MismatchedInputException import org.json4s.DefaultFormats -import org.json4s.JsonAST.{JArray, JValue} +import org.json4s.JsonAST.JArray import org.json4s.MappingException import org.json4s.jackson.JsonMethods._ @@ -68,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, parseOrFindResources(resourcesFile))) + extractAttributes, resources)) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -82,42 +83,38 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } - // Check that the executor resources at startup will satisfy the user specified task - // requirements (spark.task.resource.*) and that they match the executor configs - // specified by the user (spark.executor.resource.*) to catch mismatches between what - // the user requested and what resource manager gave or what the discovery script found. - private def checkExecResourcesMeetTaskRequirements( - taskResourceConfigs: Array[(String, String)], - actualExecResources: Map[String, ResourceInformation]): Unit = { - - // get just the map of resource name to count - val taskResourcesAndCounts = taskResourceConfigs. - withFilter { case (k, v) => k.endsWith(SPARK_RESOURCE_COUNT_POSTFIX)}. - map { case (k, v) => (k.dropRight(SPARK_RESOURCE_COUNT_POSTFIX.size), v)} - - taskResourcesAndCounts.foreach { case (rName, taskReqCount) => - if (actualExecResources.contains(rName)) { - val execResourceInfo = actualExecResources(rName) - - if (execResourceInfo.addresses.size < taskReqCount.toLong) { - throw new SparkException(s"Executor resource: $rName with addresses: " + - s"${execResourceInfo.addresses.mkString(",")} doesn't meet the task " + - s"requirements of needing $taskReqCount of them") + // 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( + 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 executor resource count on start matches the - // spark.executor.resource configs specified by user - val userExecCountConfigName = - SPARK_EXECUTOR_RESOURCE_PREFIX + rName + SPARK_RESOURCE_COUNT_POSTFIX - val userExecConfigCount = env.conf.getOption(userExecCountConfigName). - getOrElse(throw new SparkException(s"Executor resource: $rName not specified " + - s"via config: $userExecCountConfigName, but required " + - s"by the task, please fix your configuration")) - - if (userExecConfigCount.toLong != execResourceInfo.addresses.size) { - throw new SparkException(s"Executor resource: $rName, with addresses: " + - s"${execResourceInfo.addresses.mkString(",")} " + - s"doesn't match what the user requested for executor count: $userExecConfigCount, " + - s"via $userExecCountConfigName") + // 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, " + + s"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") @@ -136,7 +133,7 @@ private[spark] class CoarseGrainedExecutorBackend( val parsedJson = parse(source).asInstanceOf[JArray].arr parsedJson.map { json => val name = (json \ "name").extract[String] - val addresses = (json \ "addresses").extract[Array[JValue]].map(_.extract[String]) + val addresses = (json \ "addresses").extract[Array[String]] new ResourceInformation(name, addresses) }.map(x => (x.name -> x)).toMap } catch { @@ -147,23 +144,25 @@ private[spark] class CoarseGrainedExecutorBackend( source.close() } resourceMap - }}.getOrElse(ResourceDiscoverer.findResources(env.conf, false)) + }}.getOrElse(ResourceDiscoverer.findResources(env.conf, isDriver = false)) if (execResources.isEmpty) { throw new SparkException(s"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)} - // check that the executor has all the resources required by the application/task - checkExecResourcesMeetTaskRequirements(taskResourceConfigs, execResources) + checkResourcesMeetRequirements(SPARK_EXECUTOR_RESOURCE_PREFIX, resourcesAndCounts, + execResources) + + logInfo("===============================================================================") + logInfo("Executor ${executorId} Resources:") + execResources.foreach { case (k, v) => logInfo(s"$k -> $v") } + logInfo("===============================================================================") - logInfo(s"Executor ${executorId} using resources: ${execResources.keys}") - if (log.isDebugEnabled) { - logDebug("===============================================================================") - logDebug("Executor Resources:") - execResources.foreach { case (k, v) => logDebug(s"$k -> $v") } - logDebug("===============================================================================") - } execResources } else { if (resourcesFile.nonEmpty) { diff --git a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala index a30ba66dd042..77f8a0bae742 100644 --- a/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala +++ b/core/src/test/scala/org/apache/spark/ResourceDiscovererSuite.scala @@ -31,9 +31,16 @@ import org.apache.spark.util.Utils class ResourceDiscovererSuite extends SparkFunSuite with LocalSparkContext { + def mockDiscoveryScript(file: File, result: String): String = { + Files.write(s"echo $result", file, StandardCharsets.UTF_8) + JavaFiles.setPosixFilePermissions(file.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + file.getPath() + } + test("Resource discoverer no resources") { val sparkconf = new SparkConf - val resources = ResourceDiscoverer.findResources(sparkconf, false) + val resources = ResourceDiscoverer.findResources(sparkconf, isDriver = false) assert(resources.size === 0) assert(resources.get("gpu").isEmpty, "Should have a gpus entry that is empty") @@ -43,15 +50,12 @@ class ResourceDiscovererSuite extends SparkFunSuite val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => - val file1 = new File(dir, "resourceDiscoverScript1") - // this is a bit ugly but do it the hardway to test out some formatting - Files.write("echo {\\\"name\\\":\\\"gpu\\\"," + - " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(file1.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + val gpuFile = new File(dir, "gpuDiscoverScript") + val scriptPath = mockDiscoveryScript(gpuFile, + """'{"name": "gpu","addresses":["0", "1"]}'""") sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - val resources = ResourceDiscoverer.findResources(sparkconf, false) + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, scriptPath) + val resources = ResourceDiscoverer.findResources(sparkconf, isDriver = false) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") assert(gpuValue.get.name == "gpu", "name should be gpu") @@ -60,19 +64,16 @@ class ResourceDiscovererSuite extends SparkFunSuite } } - // TODO test("Resource discoverer no addresses errors") { val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => - val file1 = new File(dir, "resourceDiscoverScript1") - Files.write("echo {\\\"name\\\":\\\"gpu\\\"}", - file1, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(file1.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + val gpuFile = new File(dir, "gpuDiscoverScript") + val scriptPath = mockDiscoveryScript(gpuFile, + """'{"name": "gpu"}'""") sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - val resources = ResourceDiscoverer.findResources(sparkconf, false) + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, scriptPath) + val resources = ResourceDiscoverer.findResources(sparkconf, isDriver = false) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") assert(gpuValue.get.name == "gpu", "name should be gpu") @@ -84,24 +85,19 @@ class ResourceDiscovererSuite extends SparkFunSuite val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => - val gpuDiscovery = new File(dir, "resourceDiscoverScriptgpu") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", " + - " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", gpuDiscovery, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(gpuDiscovery.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) - - val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") - Files.write("echo {\\\"name\\\":\\\"fpga\\\"," + - " \\\"addresses\\\":[\\\"f1\\\",\\\"f2\\\",\\\"f3\\\"]}", - fpgaDiscovery, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) - + val gpuFile = new File(dir, "gpuDiscoverScript") + val gpuDiscovery = mockDiscoveryScript(gpuFile, + """'{"name": "gpu", "addresses": ["0", "1"]}'""") sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, gpuDiscovery.getPath()) + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, gpuDiscovery) + + val fpgaFile = new File(dir, "fpgaDiscoverScript") + val fpgaDiscovery = mockDiscoveryScript(fpgaFile, + """'{"name": "fpga", "addresses": ["f1", "f2", "f3"]}'""") sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "fpga" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, fpgaDiscovery.getPath()) - val resources = ResourceDiscoverer.findResources(sparkconf, false) + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, fpgaDiscovery) + + val resources = ResourceDiscoverer.findResources(sparkconf, isDriver = false) assert(resources.size === 2) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") @@ -122,17 +118,15 @@ class ResourceDiscovererSuite extends SparkFunSuite val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => - val file1 = new File(dir, "resourceDiscoverScript2") - Files.write("echo {\\\"name\\\":\\\"gpu\\\", " + - " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(file1.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + val gpuFile = new File(dir, "gpuDiscoverScript") + val gpuDiscovery = mockDiscoveryScript(gpuFile, + """'{"name": "gpu", "addresses": ["0", "1"]}'""") sparkconf.set(SPARK_DRIVER_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, gpuDiscovery) + sparkconf set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, "boguspath") // make sure it reads from correct config, here it should use driver - val resources = ResourceDiscoverer.findResources(sparkconf, true) + val resources = ResourceDiscoverer.findResources(sparkconf, isDriver = true) val gpuValue = resources.get("gpu") assert(gpuValue.nonEmpty, "Should have a gpu entry") assert(gpuValue.get.name == "gpu", "name should be gpu") @@ -145,16 +139,13 @@ class ResourceDiscovererSuite extends SparkFunSuite val sparkconf = new SparkConf assume(!(Utils.isWindows)) withTempDir { dir => - val file1 = new File(dir, "resourceDiscoverScript3") - Files.write("echo {\\\"units\\\":\\\"\\\"," + - " \\\"addresses\\\":[\\\"0\\\",\\\"1\\\"]}", file1, StandardCharsets.UTF_8) - JavaFiles.setPosixFilePermissions(file1.toPath(), - EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) + val gpuFile = new File(dir, "gpuDiscoverScript") + val gpuDiscovery = mockDiscoveryScript(gpuFile, + """'{"addresses": ["0", "1"]}'""") sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, gpuDiscovery) val error = intercept[SparkException] { - ResourceDiscoverer.findResources(sparkconf, false) + ResourceDiscoverer.findResources(sparkconf, isDriver = false) }.getMessage() assert(error.contains("Error running the resource discovery")) @@ -163,36 +154,33 @@ class ResourceDiscovererSuite extends SparkFunSuite test("Resource discoverer script doesn't exist") { val sparkconf = new SparkConf - val file1 = new File("/tmp/bogus") - try { - sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) - - val error = intercept[SparkException] { - ResourceDiscoverer.findResources(sparkconf, false) - }.getMessage() - - assert(error.contains("doesn't exist")) - } finally { - JavaFiles.deleteIfExists(file1.toPath()) + withTempDir { dir => + val file1 = new File(dir, "bogusfilepath") + try { + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_DISCOVERY_SCRIPT_POSTFIX, file1.getPath()) + + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, isDriver = false) + }.getMessage() + + assert(error.contains("doesn't exist")) + } finally { + JavaFiles.deleteIfExists(file1.toPath()) + } } } test("gpu's specified but not discovery script") { val sparkconf = new SparkConf - val file1 = new File("/tmp/bogus") - try { - sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + - SPARK_RESOURCE_COUNT_POSTFIX, "2") + sparkconf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + + SPARK_RESOURCE_COUNT_POSTFIX, "2") - val error = intercept[SparkException] { - ResourceDiscoverer.findResources(sparkconf, false) - }.getMessage() + val error = intercept[SparkException] { + ResourceDiscoverer.findResources(sparkconf, isDriver = false) + }.getMessage() - assert(error.contains("User is expecting to use")) - } finally { - JavaFiles.deleteIfExists(file1.toPath()) - } + assert(error.contains("User is expecting to use")) } } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 01e1000548c7..e59d1b0e047b 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -41,15 +41,11 @@ import org.apache.spark.util.Utils class CoarseGrainedExecutorBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { - // scalastyle:off println private def writeFileWithJson(dir: File, strToWrite: JArray): String = { val f1 = File.createTempFile("test-resource-parser1", "", dir) - val writer1 = new PrintWriter(f1) - writer1.println(compact(render(strToWrite))) - writer1.close() + JavaFiles.write(f1.toPath(), compact(render(strToWrite)).getBytes()) f1.getPath() } - // scalastyle:on println test("parsing no resources") { val conf = new SparkConf @@ -68,7 +64,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("Exception parsing the resources in")) + assert(error.contains("Exception parsing the resources in"), + s"Calling with no resources didn't error as expected, error: $error") } } @@ -84,7 +81,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val testResourceArgs = ("name" -> "gpu") ~ - ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) + ("addresses" -> Seq("0", "1")) val ja = JArray(List(testResourceArgs)) val f1 = writeFileWithJson(tmpDir, ja) val parsedResources = backend.parseOrFindResources(Some(f1)) @@ -111,10 +108,10 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite withTempDir { tmpDir => val gpuArgs = ("name" -> "gpu") ~ - ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) + ("addresses" -> Seq("0", "1")) val fpgaArgs = ("name" -> "fpga") ~ - ("addresses" -> JArray(Array("f1", "f2", "f3").map(JString(_)).toList)) + ("addresses" -> Seq("f1", "f2", "f3")) val ja = JArray(List(gpuArgs, fpgaArgs)) val f1 = writeFileWithJson(tmpDir, ja) val parsedResources = backend.parseOrFindResources(Some(f1)) @@ -136,14 +133,14 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) // not enough gpu's on the executor withTempDir { tmpDir => val gpuArgs = ("name" -> "gpu") ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + ("addresses" -> Seq("0")) val ja = JArray(List(gpuArgs)) val f1 = writeFileWithJson(tmpDir, ja) @@ -151,14 +148,14 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("doesn't meet the task requirements of needing")) + assert(error.contains("doesn't meet the requirements of needing")) } // missing resource on the executor withTempDir { tmpDir => val gpuArgs = ("name" -> "fpga") ~ - ("addresses" -> JArray(Array("0").map(JString(_)).toList)) + ("addresses" -> Seq("0")) val ja = JArray(List(gpuArgs)) val f1 = writeFileWithJson(tmpDir, ja) @@ -170,7 +167,33 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } - test("parsing resources task configs with missing executor count config") { + test("executor resource found less than required") { + val conf = new SparkConf + conf.set(SPARK_EXECUTOR_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "4") + conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "1") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + // executor resources < required + withTempDir { tmpDir => + val gpuArgs = + ("name" -> "gpu") ~ + ("addresses" -> Seq("0", "1")) + val ja = JArray(List(gpuArgs)) + val f1 = writeFileWithJson(tmpDir, ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseOrFindResources(Some(f1)) + }.getMessage() + + assert(error.contains("is less than what the user requested for count")) + } + } + + test("parsing resources task configs with missing executor config") { val conf = new SparkConf conf.set(SPARK_TASK_RESOURCE_PREFIX + "gpu" + SPARK_RESOURCE_COUNT_POSTFIX, "2") val serializer = new JavaSerializer(conf) @@ -179,11 +202,10 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", 4, Seq.empty[URL], env, None) - // executor config doesn't have units on gpu and task one does withTempDir { tmpDir => val gpuArgs = ("name" -> "gpu") ~ - ("addresses" -> JArray(Array("0", "1").map(JString(_)).toList)) + ("addresses" -> Seq("0", "1")) val ja = JArray(List(gpuArgs)) val f1 = writeFileWithJson(tmpDir, ja) @@ -191,8 +213,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("Executor resource: gpu not specified via config: " + - "spark.executor.resource.gpu.count, but required by the task, please " + + assert(error.contains("Resource: gpu not specified via config: " + + "spark.executor.resource.gpu.count, but required, please " + "fix your configuration")) } } @@ -204,8 +226,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite assume(!(Utils.isWindows)) withTempDir { dir => val fpgaDiscovery = new File(dir, "resourceDiscoverScriptfpga") - Files.write("echo {\\\"name\\\":\\\"fpga\\\", " + - " \\\"addresses\\\":[\\\"f1\\\",\\\"f2\\\",\\\"f3\\\"]}", + Files.write("""echo '{"name": "fpga","addresses":["f1", "f2", "f3"]}'""", fpgaDiscovery, StandardCharsets.UTF_8) JavaFiles.setPosixFilePermissions(fpgaDiscovery.toPath(), EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)) From dd60e42738022734458ffaecbdf4d9d6a326742a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 13 May 2019 14:13:34 -0500 Subject: [PATCH 21/22] nit rework --- .../spark/executor/CoarseGrainedExecutorBackend.scala | 8 ++++---- docs/configuration.md | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 73630205aeb9..15211b1f855f 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -104,11 +104,11 @@ private[spark] class CoarseGrainedExecutorBackend( // also make sure the resource count on start matches the // resource configs specified by user val userCountConfigName = - resourceConfigPrefix + rName + SPARK_RESOURCE_COUNT_POSTFIX + 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, " + - s"please fix your configuration")) + "please fix your configuration")) if (userConfigCount.toLong > resourceInfo.addresses.size) { throw new SparkException(s"Resource: $rName, with addresses: " + @@ -147,7 +147,7 @@ private[spark] class CoarseGrainedExecutorBackend( }}.getOrElse(ResourceDiscoverer.findResources(env.conf, isDriver = false)) if (execResources.isEmpty) { - throw new SparkException(s"User specified resources per task via: " + + 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 @@ -159,7 +159,7 @@ private[spark] class CoarseGrainedExecutorBackend( execResources) logInfo("===============================================================================") - logInfo("Executor ${executorId} Resources:") + logInfo("Executor $executorId Resources:") execResources.foreach { case (k, v) => logInfo(s"$k -> $v") } logInfo("===============================================================================") diff --git a/docs/configuration.md b/docs/configuration.md index 539c96a66e18..d0b2699a5dc7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -235,9 +235,9 @@ of the most common options to set are: spark.executor.resource.{resourceType}.discoveryScript None - The script the executor should run to discover a particular resource type. This should + A script for the executor to run to discover a particular resource type. This should write to STDOUT a JSON string in the format of the ResourceInformation class. This has a - name and array of addresses. + name and an array of addresses. From b9dacef1d7d47d19df300628d3841b3a13c03547 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 13 May 2019 21:26:41 -0500 Subject: [PATCH 22/22] fix missing string interpolation --- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 15211b1f855f..af01e0b23dad 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -159,7 +159,7 @@ private[spark] class CoarseGrainedExecutorBackend( execResources) logInfo("===============================================================================") - logInfo("Executor $executorId Resources:") + logInfo(s"Executor $executorId Resources:") execResources.foreach { case (k, v) => logInfo(s"$k -> $v") } logInfo("===============================================================================")