From acdf6e8ea6f1f68ad44157bcf5566cb2782cb069 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 27 Mar 2020 10:25:46 -0500 Subject: [PATCH 01/31] Stage level scheduling python api support --- .../org/apache/spark/BarrierTaskContext.scala | 2 + .../scala/org/apache/spark/TaskContext.scala | 6 ++ .../org/apache/spark/TaskContextImpl.scala | 4 +- .../org/apache/spark/api/java/JavaRDD.scala | 15 ++++ .../spark/api/python/PythonRunner.scala | 28 +++++-- .../CoarseGrainedExecutorBackend.scala | 6 +- .../spark/resource/ResourceProfile.scala | 43 +++++++++++ .../apache/spark/scheduler/DAGScheduler.scala | 5 +- .../apache/spark/scheduler/ResultTask.scala | 7 +- .../spark/scheduler/ShuffleMapTask.scala | 7 +- .../org/apache/spark/scheduler/Task.scala | 8 +- pom.xml | 2 +- python/pyspark/__init__.py | 8 ++ python/pyspark/executorresourcerequest.py | 73 ++++++++++++++++++ python/pyspark/executorresourcerequests.py | 65 ++++++++++++++++ python/pyspark/java_gateway.py | 1 + python/pyspark/rdd.py | 31 +++++++- python/pyspark/resourceprofile.py | 53 +++++++++++++ python/pyspark/resourceprofilebuilder.py | 76 +++++++++++++++++++ python/pyspark/taskresourcerequest.py | 45 +++++++++++ python/pyspark/taskresourcerequests.py | 52 +++++++++++++ python/pyspark/tests/test_rdd.py | 29 ++++++- 22 files changed, 546 insertions(+), 20 deletions(-) create mode 100644 python/pyspark/executorresourcerequest.py create mode 100644 python/pyspark/executorresourcerequests.py create mode 100644 python/pyspark/resourceprofile.py create mode 100644 python/pyspark/resourceprofilebuilder.py create mode 100644 python/pyspark/taskresourcerequest.py create mode 100644 python/pyspark/taskresourcerequests.py diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 0c2ceb1a02c7b..b266de4084111 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -275,6 +275,8 @@ class BarrierTaskContext private[spark] ( resources().asJava } + override def resourceProfileId(): Int = taskContext.resourceProfileId() + override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted() override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason() diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index fd41facf95c76..b116690c62d31 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -193,6 +193,12 @@ abstract class TaskContext extends Serializable { @Evolving def resourcesJMap(): java.util.Map[String, ResourceInformation] + /** + * ResourceProfile Id used by the task. + */ + @Evolving + def resourceProfileId(): Int + @DeveloperApi def taskMetrics(): TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 08a58a029528b..01ff58d89dbac 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -29,6 +29,7 @@ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util._ @@ -54,7 +55,8 @@ private[spark] class TaskContextImpl( @transient private val metricsSystem: MetricsSystem, // The default value is only used in tests. override val taskMetrics: TaskMetrics = TaskMetrics.empty, - override val resources: Map[String, ResourceInformation] = Map.empty) + override val resources: Map[String, ResourceInformation] = Map.empty, + override val resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) extends TaskContext with Logging { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 6f0182255e5fe..36307e9fc325d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -27,6 +27,7 @@ import org.apache.spark._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -49,6 +50,20 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) */ def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel)) + /** + * Specify a ResourceProfile to use when calculating this RDD. This is only supported on + * certain cluster managers and currently requires dynamic allocation to be enabled. + * It will result in new executors with the resources specified being acquired to + * calculate the RDD. + */ + def withResources(prof: ResourceProfile): JavaRDD[T] = wrapRDD(rdd.withResources(prof)) + + /** + * Get the ResourceProfile specified with this RDD or None if it wasn't specified. + * @return the user specified ResourceProfile or null if none was specified + */ + def getResourceProfile(): ResourceProfile = rdd.getResourceProfile() + /** * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. * This method blocks until all blocks are deleted. diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 06c9446c7534e..72d227064c523 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -35,6 +35,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} import org.apache.spark.internal.config.Python._ +import org.apache.spark.resource.ResourceProfile.{getPysparkMemoryFromInternalConfs, DEFAULT_RESOURCE_PROFILE_ID} import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -85,9 +86,6 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( private val conf = SparkEnv.get.conf protected val bufferSize: Int = conf.get(BUFFER_SIZE) private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) - // each python worker gets an equal part of the allocation. the worker pool will grow to the - // number of concurrent tasks, which is determined by the number of cores in this executor. - private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY).map(_ / conf.get(EXECUTOR_CORES)) // All the Python functions should have the same exec, version and envvars. protected val envVars: java.util.Map[String, String] = funcs.head.funcs.head.envVars @@ -106,6 +104,12 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( // Authentication helper used when serving method calls via socket from Python side. private lazy val authHelper = new SocketAuthHelper(conf) + // each python worker gets an equal part of the allocation. the worker pool will grow to the + // number of concurrent tasks, which is determined by the number of cores in this executor. + private def getWorkerMemoryMb(mem: Option[Long]): Option[Long] = { + mem.map(_ / conf.get(EXECUTOR_CORES)) + } + def compute( inputIterator: Iterator[IN], partitionIndex: Int, @@ -124,8 +128,22 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (reuseWorker) { envVars.put("SPARK_REUSE_WORKER", "1") } - if (memoryMb.isDefined) { - envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", memoryMb.get.toString) + // Check to see if the pyspark memory conf set for the resource profile id being used. + // Not all cluster managers are supported so fall back to the application level config + // when its the default profile id. + val rpId = context.resourceProfileId() + val memoryMb = if (rpId == DEFAULT_RESOURCE_PROFILE_ID) { + logInfo("using default profile so default executor memory") + conf.get(PYSPARK_EXECUTOR_MEMORY) + } else { + val mem = getPysparkMemoryFromInternalConfs(conf, rpId) + logInfo(s"using prorfile $rpId memory $mem") + mem + + } + val workerMemoryMb = getWorkerMemoryMb(memoryMb) + if (workerMemoryMb.isDefined) { + envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", workerMemoryMb.get.toString) } envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString) val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap) 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 6625457749f6a..572ab7bfd7160 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -347,7 +347,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig(arguments.resourceProfileId)) - val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", arguments.appId)) + // we have to add the pyspark memory conf into SparkConfs so the PythonRunner can + // pick it up properly + val pysparkMemoryConf = cfg.resourceProfile.getInternalPysparkMemoryConfs + val props = cfg.sparkProperties ++ + Seq[(String, String)](("spark.app.id", arguments.appId)) ++ pysparkMemoryConf fetcher.shutdown() // Create SparkEnv using properties we fetched from the driver. diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 96c456ed51746..d6ba11381c80b 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -51,6 +51,8 @@ class ResourceProfile( private var _limitingResource: Option[String] = None private var _maxTasksPerExecutor: Option[Int] = None private var _coresLimitKnown: Boolean = false + private var _internalPysparkMemoryConf: Seq[(String, String)] = + ResourceProfile.createPysparkMemoryInternalConfs(this) def id: Int = _id @@ -76,6 +78,10 @@ class ResourceProfile( taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt) } + private[spark] def getInternalPysparkMemoryConfs: Seq[(String, String)] = { + _internalPysparkMemoryConf + } + /* * This function takes into account fractional amounts for the task resource requirement. * Spark only supports fractional amounts < 1 to basically allow for multiple tasks @@ -325,4 +331,41 @@ object ResourceProfile extends Logging { private[spark] def getTaskCpusOrDefaultForProfile(rp: ResourceProfile, conf: SparkConf): Int = { rp.getTaskCpus.getOrElse(conf.get(CPUS_PER_TASK)) } + + private[spark] val SPARK_RP_EXEC_PREFIX = "spark.resourceProfile.executor" + + // Helper class for constructing the resource profile internal configs. The configs look like: + // spark.resourceProfile.executor.[rpId].[resourceName].amount + private[spark] case class ResourceProfileInternalConf(rpId: Int, resourceName: String) { + val prefix = s"$SPARK_RP_EXEC_PREFIX.$rpId." + def resourceNameAndAmount: String = s"$resourceName.${ResourceUtils.AMOUNT}" + def amountConf: String = s"$prefix$resourceNameAndAmount" + } + + /** + * Create the ResourceProfile internal pyspark memory conf that are used by the executors. + * It pulls any pyspark.memory config from the profile and returns a Seq of key and value + * where the keys get formatted as: + * + * spark.resourceProfile.executor.[rpId].[resourceName].[amount, vendor, discoveryScript] + */ + private[spark] def createPysparkMemoryInternalConfs( + rp: ResourceProfile + ): Seq[(String, String)] = { + rp.executorResources.get(ResourceProfile.PYSPARK_MEM).map { pysparkMem => + val pysparkMemIntConf = ResourceProfileInternalConf(rp.id, ResourceProfile.PYSPARK_MEM) + Seq((pysparkMemIntConf.amountConf, pysparkMem.amount.toString)) + }.getOrElse(Seq.empty) + } + + /** + * Get the pyspark memory setting from internal resource confs. + * The config looks like: spark.resourceProfile.executor.[rpId].pyspark.memory.amount + */ + private[spark] def getPysparkMemoryFromInternalConfs( + sparkConf: SparkConf, + rpId: Int): Option[Long] = { + val rName = ResourceProfile.PYSPARK_MEM + sparkConf.getOption(ResourceProfileInternalConf(rpId, rName).amountConf).map(_.toLong) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 079cf110a7ce9..08632715bcc95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1255,7 +1255,8 @@ private[spark] class DAGScheduler( stage.pendingPartitions += id new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), - Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier()) + Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(), + stage.resourceProfileId) } case stage: ResultStage => @@ -1266,7 +1267,7 @@ private[spark] class DAGScheduler( new ResultTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, id, properties, serializedTaskMetrics, Option(jobId), Option(sc.applicationId), sc.applicationAttemptId, - stage.rdd.isBarrier()) + stage.rdd.isBarrier(), stage.resourceProfileId) } } } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 857c89d7a98f5..174dff3241713 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -25,6 +25,7 @@ import java.util.Properties import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID /** * A task that sends back the output to the driver application. @@ -50,6 +51,7 @@ import org.apache.spark.rdd.RDD * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. + * @param resourceProfileId the id of the ResourceProfile used with this task */ private[spark] class ResultTask[T, U]( stageId: Int, @@ -63,9 +65,10 @@ private[spark] class ResultTask[T, U]( jobId: Option[Int] = None, appId: Option[String] = None, appAttemptId: Option[String] = None, - isBarrier: Boolean = false) + isBarrier: Boolean = false, + resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics, - jobId, appId, appAttemptId, isBarrier) + jobId, appId, appAttemptId, isBarrier, resourceProfileId) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 4c0c30a3caf67..008dfc950b749 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -25,6 +25,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.{config, Logging} import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID /** * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner @@ -48,6 +49,7 @@ import org.apache.spark.rdd.RDD * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. + * @param resourceProfileId the id of the ResourceProfile used with this task */ private[spark] class ShuffleMapTask( stageId: Int, @@ -60,9 +62,10 @@ private[spark] class ShuffleMapTask( jobId: Option[Int] = None, appId: Option[String] = None, appAttemptId: Option[String] = None, - isBarrier: Boolean = false) + isBarrier: Boolean = false, + resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties, - serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier) + serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, resourceProfileId) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index ebc1c05435fee..e785d4770ddac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -27,6 +27,7 @@ import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.util._ /** @@ -53,6 +54,7 @@ import org.apache.spark.util._ * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. + * @param resourceProfileId the id of the ResourceProfile used with this task */ private[spark] abstract class Task[T]( val stageId: Int, @@ -65,7 +67,8 @@ private[spark] abstract class Task[T]( val jobId: Option[Int] = None, val appId: Option[String] = None, val appAttemptId: Option[String] = None, - val isBarrier: Boolean = false) extends Serializable { + val isBarrier: Boolean = false, + val resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) extends Serializable { @transient lazy val metrics: TaskMetrics = SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) @@ -96,7 +99,8 @@ private[spark] abstract class Task[T]( localProperties, metricsSystem, metrics, - resources) + resources, + resourceProfileId) context = if (isBarrier) { new BarrierTaskContext(taskContext) diff --git a/pom.xml b/pom.xml index cc48ee794ea04..f3d547db28c4a 100644 --- a/pom.xml +++ b/pom.xml @@ -2341,7 +2341,7 @@ net.alchim31.maven scala-maven-plugin - 4.3.0 + 3.4.4 eclipse-add-source diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 76a5bd0d645ba..e3c3254bdf58b 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -55,6 +55,12 @@ from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast from pyspark.resourceinformation import ResourceInformation +from pyspark.taskresourcerequest import TaskResourceRequest +from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.taskresourcerequests import TaskResourceRequests +from pyspark.executorresourcerequests import ExecutorResourceRequests +from pyspark.resourceprofilebuilder import ResourceProfileBuilder +from pyspark.resourceprofile import ResourceProfile from pyspark.serializers import MarshalSerializer, PickleSerializer from pyspark.status import * from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo @@ -120,4 +126,6 @@ def wrapper(self, *args, **kwargs): "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation", + "TaskResourceRequest", "TaskResourceRequests", "ExecutorResourceRequest", + "ExecutorResourceRequests", "ResourceProfile" ] diff --git a/python/pyspark/executorresourcerequest.py b/python/pyspark/executorresourcerequest.py new file mode 100644 index 0000000000000..79c824f64cb5f --- /dev/null +++ b/python/pyspark/executorresourcerequest.py @@ -0,0 +1,73 @@ +# +# 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. +# + + +class ExecutorResourceRequest(object): + """ + .. note:: Evolving + + An Executor resource request. This is used in conjunction with the ResourceProfile to + programmatically specify the resources needed for an RDD that will be applied at the + stage level. + + This is used to specify what the resource requirements are for an Executor and how + Spark can find out specific details about those resources. Not all the parameters are + required for every resource type. Resources like GPUs are supported and have same limitations + as using the global spark configs spark.executor.resource.gpu.*. The amount, discoveryScript, + and vendor parameters for resources are all the same parameters a user would specify through the + configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. + + For instance, a user wants to allocate an Executor with GPU resources on YARN. The user has + to specify the resource name (gpu), the amount or number of GPUs per Executor, + the discovery script would be specified so that when the Executor starts up it can + discovery what GPU addresses are available for it to use because YARN doesn't tell + Spark that, then vendor would not be used because its specific for Kubernetes. + + See the configuration and cluster specific docs for more details. + + Use ExecutorResourceRequests class as a convenience API. + + :param resourceName: Name of the resource + :param amount: Amount requesting + :param discoveryScript: Optional script used to discover the resources. This is required on some + cluster managers that don't tell Spark the addresses of the resources + allocated. The script runs on Executors startup to discover the addresses + of the resources available. + :param vendor: Vendor, required for some cluster managers + """ + + def __init__(self, resourceName, amount, discoveryScript="", vendor=""): + """Create a new ExecutorResourceRequest that wraps the underlying JVM object.""" + from pyspark.context import SparkContext + self._jExecRequest = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequest( + resourceName, amount, discoveryScript, vendor) + + @property + def resourceName(self): + return self._jExecRequest.resourceName() + + @property + def amount(self): + return self._jExecRequest.amount() + + @property + def discoveryScript(self): + return self._jExecRequest.discoveryScript() + + @property + def vendor(self): + return self._jExecRequest.vendor() diff --git a/python/pyspark/executorresourcerequests.py b/python/pyspark/executorresourcerequests.py new file mode 100644 index 0000000000000..761282b38a795 --- /dev/null +++ b/python/pyspark/executorresourcerequests.py @@ -0,0 +1,65 @@ +# +# 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. +# + +from pyspark.executorresourcerequest import ExecutorResourceRequest + + +class ExecutorResourceRequests(object): + + """ + .. note:: Evolving + + A set of Executor resource requests. This is used in conjunction with the + ResourceProfileBuilder to programmatically specify the resources needed for an RDD + that will be applied at the stage level. + """ + + def __init__(self): + """Create a new ExecutorResourceRequests that wraps the underlying JVM object.""" + from pyspark import SparkContext + self._javaExecutorResourceRequests \ + = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequests() + + def memory(self, amount): + self._javaExecutorResourceRequests.memory(amount) + return self + + def memoryOverhead(self, amount): + self._javaExecutorResourceRequests.memoryOverhead(amount) + return self + + def pysparkMemory(self, amount): + self._javaExecutorResourceRequests.pysparkMemory(amount) + return self + + def cores(self, amount): + self._javaExecutorResourceRequests.cores(amount) + return self + + def resource(self, resourceName, amount, discoveryScript="", vendor=""): + self._javaExecutorResourceRequests.resource(resourceName, amount, discoveryScript, vendor) + return self + + @property + def requests(self): + execRes = self._javaExecutorResourceRequests.requestsJMap() + result = {} + # convert back to python ExecutorResourceRequest + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 316a5b4d01273..0daf09b17a82a 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -152,6 +152,7 @@ def killChild(): java_import(gateway.jvm, "org.apache.spark.api.python.*") java_import(gateway.jvm, "org.apache.spark.ml.python.*") java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*") + java_import(gateway.jvm, "org.apache.spark.resource.*") # TODO(davies): move into sql java_import(gateway.jvm, "org.apache.spark.sql.*") java_import(gateway.jvm, "org.apache.spark.sql.api.python.*") diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 52ab86c0d88ee..8aefb13d2c9a7 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -47,6 +47,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel +from pyspark.resourceprofile import ResourceProfile from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy @@ -256,6 +257,7 @@ def __init__(self, jrdd, ctx, jrdd_deserializer=AutoBatchedSerializer(PickleSeri self._jrdd = jrdd self.is_cached = False self.is_checkpointed = False + self.has_resourceProfile = False self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer self._id = jrdd.id() @@ -2455,7 +2457,7 @@ def toLocalIterator(self, prefetchPartitions=False): self._jrdd.rdd(), prefetchPartitions) return _local_iterator_from_socket(sock_info, self._jrdd_deserializer) - +jj def barrier(self): """ .. note:: Experimental @@ -2483,6 +2485,28 @@ def _is_barrier(self): """ return self._jrdd.rdd().isBarrier() + def withResources(self, profile): + """ + .. note:: Experimental + Specify a ResourceProfile to use when calculating this RDD. This is only supported on + certain cluster managers and currently requires dynamic allocation to be enabled. + It will result in new executors with the resources specified being acquired to + calculate the RDD. + .. versionadded:: 3.0.0 + """ + self.has_resourceProfile = True + self._jrdd.withResources(profile._jResourceProfile) + return self + + def getResourceProfile(self): + """ + .. note:: Experimental + Get the ResourceProfile specified with this RDD or None if it wasn't specified. + :return: the user specified ResourceProfile or null if none was specified + .. versionadded:: 3.0.0 + """ + return ResourceProfile(self._jrdd.getResourceProfile()) + def _prepare_for_python_RDD(sc, command): # the serialized command will be compressed by broadcast @@ -2587,6 +2611,7 @@ def pipeline_func(split, iterator): self._prev_jrdd = prev._prev_jrdd # maintain the pipeline self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer self.is_cached = False + self.has_resourceProfile = False self.is_checkpointed = False self.ctx = prev.ctx self.prev = prev @@ -2629,9 +2654,9 @@ def id(self): return self._id def _is_pipelinable(self): - return not (self.is_cached or self.is_checkpointed) + return not (self.is_cached or self.is_checkpointed or self.has_resourceProfile) - def _is_barrier(self): +def _is_barrier(self): return self.is_barrier diff --git a/python/pyspark/resourceprofile.py b/python/pyspark/resourceprofile.py new file mode 100644 index 0000000000000..62e8d70590f21 --- /dev/null +++ b/python/pyspark/resourceprofile.py @@ -0,0 +1,53 @@ + +# 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. +# + +from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.taskresourcerequest import TaskResourceRequest + + +class ResourceProfile(object): + + """ + .. note:: Evolving + + Resource profile to associate with an RDD. A ResourceProfile allows the user to + specify executor and task requirements for an RDD that will get applied during a + stage. This allows the user to change the resource requirements between stages. + This is meant to be immutable so user doesn't change it after building. + """ + + def __init__(self, _jResourceProfile): + self._jResourceProfile = _jResourceProfile + + @property + def taskResources(self): + taskRes = self._jResourceProfile.taskResourcesJMap() + result = {} + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result + + @property + def executorResources(self): + execRes = self._jResourceProfile.executorResourcesJMap() + result = {} + # convert back to python ExecutorResourceRequest + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result diff --git a/python/pyspark/resourceprofilebuilder.py b/python/pyspark/resourceprofilebuilder.py new file mode 100644 index 0000000000000..2fe3eca1a9b62 --- /dev/null +++ b/python/pyspark/resourceprofilebuilder.py @@ -0,0 +1,76 @@ +# +# 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. +# + +from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.resourceprofile import ResourceProfile +from pyspark.taskresourcerequest import TaskResourceRequest +from pyspark.taskresourcerequests import TaskResourceRequests + + +class ResourceProfileBuilder(object): + + """ + .. note:: Evolving + + Resource profile Builder to build a resource profile to associate with an RDD. + A ResourceProfile allows the user to specify executor and task requirements for + an RDD that will get applied during a stage. This allows the user to change the + resource requirements between stages. + """ + + def __init__(self, ): + """Create a new ResourceProfileBuilder that wraps the underlying JVM object.""" + from pyspark.context import SparkContext + self._jResourceProfileBuilder \ + = SparkContext._jvm.org.apache.spark.resource.ResourceProfileBuilder() + + def require(self, resourceRequest): + if isinstance(resourceRequest, TaskResourceRequests): + self._jResourceProfileBuilder.require(resourceRequest._javaTaskResourceRequests) + else: + self._jResourceProfileBuilder.require(resourceRequest._javaExecutorResourceRequests) + return self + + def clearExecutorResourceRequests(self): + self._jResourceProfileBuilder.clearExecutorResourceRequests() + + def clearTaskResourceRequests(self): + self._jResourceProfileBuilder.clearTaskResourceRequests() + + @property + def taskResources(self): + taskRes = self._jResourceProfileBuilder.taskResourcesJMap() + result = {} + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result + + @property + def executorResources(self): + execRes = self._jResourceProfileBuilder.executorResourcesJMap() + result = {} + # convert back to python ExecutorResourceRequest + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result + + @property + def build(self): + jresourceProfile = self._jResourceProfileBuilder.build() + return ResourceProfile(jresourceProfile) diff --git a/python/pyspark/taskresourcerequest.py b/python/pyspark/taskresourcerequest.py new file mode 100644 index 0000000000000..784416ea57056 --- /dev/null +++ b/python/pyspark/taskresourcerequest.py @@ -0,0 +1,45 @@ +# +# 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. +# + + +class TaskResourceRequest(object): + """ + .. note:: Evolving + + A task resource request. This is used in conjuntion with the ResourceProfile to + programmatically specify the resources needed for an RDD that will be applied at the + stage level. The amount is specified as a Double to allow for saying you want more then + 1 task per resource. Valid values are less than or equal to 0.5 or whole numbers. + Use TaskResourceRequests class as a convenience API. + + :param resourceName: Name of the resource + :param amount: Amount requesting as a Double to support fractional resource requests. + Valid values are less than or equal to 0.5 or whole numbers. + """ + def __init__(self, resourceName, amount): + """Create a new TaskResourceRequest that wraps the underlying JVM object.""" + from pyspark.context import SparkContext + self._jTaskRequest = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( + resourceName, amount) + + @property + def resourceName(self): + return self._jTaskRequest.resourceName() + + @property + def amount(self): + return self._jTaskRequest.amount() diff --git a/python/pyspark/taskresourcerequests.py b/python/pyspark/taskresourcerequests.py new file mode 100644 index 0000000000000..76cc177c1db90 --- /dev/null +++ b/python/pyspark/taskresourcerequests.py @@ -0,0 +1,52 @@ +# +# 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. +# + +from pyspark.taskresourcerequest import TaskResourceRequest + + +class TaskResourceRequests(object): + + """ + .. note:: Evolving + + A set of task resource requests. This is used in conjuntion with the + ResourceProfileBuilder to programmatically specify the resources needed for + an RDD that will be applied at the stage level. + """ + + def __init__(self): + """Create a new TaskResourceRequests that wraps the underlying JVM object.""" + from pyspark import SparkContext + self._javaTaskResourceRequests \ + = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() + + def cpus(self, amount): + self._javaTaskResourceRequests.cpus(amount) + return self + + def resource(self, resourceName, amount): + self._javaTaskResourceRequests.resource(resourceName, float(amount)) + return self + + @property + def requests(self): + taskRes = self._javaTaskResourceRequests.requestsJMap() + result = {} + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index 15cc48ae2153d..10f53841fb69e 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -25,7 +25,8 @@ from py4j.protocol import Py4JJavaError -from pyspark import shuffle, RDD +from pyspark import ExecutorResourceRequests, shuffle, RDD, ResourceProfile,\ + ResourceProfileBuilder, TaskResourceRequests from pyspark.serializers import CloudPickleSerializer, BatchedSerializer, PickleSerializer,\ MarshalSerializer, UTF8Deserializer, NoOpSerializer from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME, QuietTest @@ -783,6 +784,32 @@ def fail_last(x): for i in range(4): self.assertEqual(i, next(it)) + def test_resourceprofile(self): + rp_builder = ResourceProfileBuilder() + ereqs = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g") + ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com") + treqs = TaskResourceRequests().cpus(2).resource("gpu", 2) + + def assert_request_contents(exec_reqs, task_reqs): + self.assertEqual(len(exec_reqs), 5) + self.assertEqual(exec_reqs["cores"].amount, 2) + self.assertEqual(exec_reqs["memory"].amount, 6144) + self.assertEqual(exec_reqs["memoryOverhead"].amount, 1024) + self.assertEqual(exec_reqs["pyspark.memory"].amount, 2048) + self.assertEqual(exec_reqs["gpu"].amount, 2) + self.assertEqual(exec_reqs["gpu"].discoveryScript, "testGpus") + self.assertEqual(exec_reqs["gpu"].resourceName, "gpu") + self.assertEqual(exec_reqs["gpu"].vendor, "nvidia.com") + self.assertEqual(len(task_reqs), 2) + self.assertEqual(task_reqs["cpus"].amount, 2.0) + self.assertEqual(task_reqs["gpu"].amount, 2.0) + + assert_request_contents(ereqs.requests, treqs.requests) + rp = rp_builder.require(ereqs).require(treqs).build + assert_request_contents(rp.executorResources, rp.taskResources) + rdd = self.sc.parallelize(range(10)).withResources(rp) + return_rp = rdd.getResourceProfile() + assert_request_contents(return_rp.executorResources, return_rp.taskResources) if __name__ == "__main__": import unittest From 24c1a96c96d58c991cd5bf59d19ad1fb390f5efa Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 27 Mar 2020 10:35:32 -0500 Subject: [PATCH 02/31] revert pom changes --- pom.xml | 2 +- python/pyspark/rdd.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index f3d547db28c4a..cc48ee794ea04 100644 --- a/pom.xml +++ b/pom.xml @@ -2341,7 +2341,7 @@ net.alchim31.maven scala-maven-plugin - 3.4.4 + 4.3.0 eclipse-add-source diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8aefb13d2c9a7..5caae89a3db69 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2457,7 +2457,7 @@ def toLocalIterator(self, prefetchPartitions=False): self._jrdd.rdd(), prefetchPartitions) return _local_iterator_from_socket(sock_info, self._jrdd_deserializer) -jj + def barrier(self): """ .. note:: Experimental From 56475350a3145141dcbcf0be1ff2d2e399f2ed57 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 30 Mar 2020 14:30:58 -0500 Subject: [PATCH 03/31] Fix log messages --- .../org/apache/spark/api/python/PythonRunner.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 72d227064c523..4fc6ebdff6409 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -128,16 +128,16 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (reuseWorker) { envVars.put("SPARK_REUSE_WORKER", "1") } - // Check to see if the pyspark memory conf set for the resource profile id being used. - // Not all cluster managers are supported so fall back to the application level config - // when its the default profile id. + // Check to see if the pyspark memory conf is set for the resource profile id being used. + // Note we don't fallback to the cluster default if its not set in the profile because + // this is a totally optional config and the user may not want it set. val rpId = context.resourceProfileId() val memoryMb = if (rpId == DEFAULT_RESOURCE_PROFILE_ID) { - logInfo("using default profile so default executor memory") + logInfo("Using the default pyspark executor memory") conf.get(PYSPARK_EXECUTOR_MEMORY) } else { val mem = getPysparkMemoryFromInternalConfs(conf, rpId) - logInfo(s"using prorfile $rpId memory $mem") + logInfo(s"Using profile $rpId pyspark executor memory $mem") mem } From af69e4bf5c7a52c8375b3a936f508bbc1d2831bc Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 30 Mar 2020 15:42:50 -0500 Subject: [PATCH 04/31] Try changing way we pass pyspark memory --- .../org/apache/spark/api/python/PythonRunner.scala | 2 ++ .../org/apache/spark/resource/ResourceProfile.scala | 13 +++++++++---- .../org/apache/spark/scheduler/DAGScheduler.scala | 3 +++ .../org/apache/spark/scheduler/TaskSetManager.scala | 4 +++- 4 files changed, 17 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 4fc6ebdff6409..a8508c59602bd 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -128,6 +128,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (reuseWorker) { envVars.put("SPARK_REUSE_WORKER", "1") } + val pymem = Option(context.getLocalProperty("spark.resource.pyspark.memory")) + logInfo("task context pyspark memory is: $pymem") // Check to see if the pyspark memory conf is set for the resource profile id being used. // Note we don't fallback to the cluster default if its not set in the profile because // this is a totally optional config and the user may not want it set. diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index d6ba11381c80b..89e4d955d70e3 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -78,6 +78,10 @@ class ResourceProfile( taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt) } + private[spark] def getPysparkMemory: Option[Long] = { + taskResources.get(ResourceProfile.PYSPARK_MEM).map(_.amount.toLong) + } + private[spark] def getInternalPysparkMemoryConfs: Seq[(String, String)] = { _internalPysparkMemoryConf } @@ -332,6 +336,7 @@ object ResourceProfile extends Logging { rp.getTaskCpus.getOrElse(conf.get(CPUS_PER_TASK)) } + private[spark] val PYSPARK_MEMORY_PROPERTY = "resource.pyspark.memory" private[spark] val SPARK_RP_EXEC_PREFIX = "spark.resourceProfile.executor" // Helper class for constructing the resource profile internal configs. The configs look like: @@ -343,11 +348,11 @@ object ResourceProfile extends Logging { } /** - * Create the ResourceProfile internal pyspark memory conf that are used by the executors. - * It pulls any pyspark.memory config from the profile and returns a Seq of key and value - * where the keys get formatted as: + * Create the ResourceProfile internal pyspark memory conf that is used by the executors. + * It pulls any pyspark.memory config from the profile and returns Seq of key and value + * where the key is formatted as: * - * spark.resourceProfile.executor.[rpId].[resourceName].[amount, vendor, discoveryScript] + * spark.resourceProfile.executor.[rpId].pyspark.memory.amount */ private[spark] def createPysparkMemoryInternalConfs( rp: ResourceProfile diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 08632715bcc95..22dda935f0b60 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -39,6 +39,7 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.ResourceProfile.PYSPARK_MEMORY_PROPERTY import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1154,6 +1155,8 @@ private[spark] class DAGScheduler( // Use the scheduling pool, job group, description, etc. from an ActiveJob associated // with this Stage val properties = jobIdToActiveJob(jobId).properties + val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) + rp.getPysparkMemory.map(m => properties.setProperty(PYSPARK_MEMORY_PROPERTY, m.toString)) runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 2c792338b5295..0f62eeb804d41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -469,7 +469,7 @@ private[spark] class TaskSetManager( s"taskResourceAssignments ${taskResourceAssignments}") sched.dagScheduler.taskStarted(task, info) - new TaskDescription( + val desc = new TaskDescription( taskId, attemptNum, execId, @@ -481,6 +481,8 @@ private[spark] class TaskSetManager( task.localProperties, taskResourceAssignments, serializedTask) + desc.properties.setProperty("spark.resource.pyspark.memory", "2048") + desc } } else { None From 4a7f39af760b687b605c468592d1c9dca6f1d0e7 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 31 Mar 2020 09:28:28 -0500 Subject: [PATCH 05/31] Change to use local property to pass pyspark memory --- .../org/apache/spark/BarrierTaskContext.scala | 2 - .../scala/org/apache/spark/TaskContext.scala | 6 --- .../org/apache/spark/TaskContextImpl.scala | 3 +- .../spark/api/python/PythonRunner.scala | 19 ++------ .../CoarseGrainedExecutorBackend.scala | 6 +-- .../spark/resource/ResourceProfile.scala | 44 +------------------ .../apache/spark/scheduler/DAGScheduler.scala | 19 +++++--- .../apache/spark/scheduler/ResultTask.scala | 7 +-- .../spark/scheduler/ShuffleMapTask.scala | 7 +-- .../org/apache/spark/scheduler/Task.scala | 8 +--- .../spark/scheduler/TaskSetManager.scala | 4 +- 11 files changed, 26 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index b266de4084111..0c2ceb1a02c7b 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -275,8 +275,6 @@ class BarrierTaskContext private[spark] ( resources().asJava } - override def resourceProfileId(): Int = taskContext.resourceProfileId() - override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted() override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason() diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index b116690c62d31..fd41facf95c76 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -193,12 +193,6 @@ abstract class TaskContext extends Serializable { @Evolving def resourcesJMap(): java.util.Map[String, ResourceInformation] - /** - * ResourceProfile Id used by the task. - */ - @Evolving - def resourceProfileId(): Int - @DeveloperApi def taskMetrics(): TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 01ff58d89dbac..d9063a1790aab 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -55,8 +55,7 @@ private[spark] class TaskContextImpl( @transient private val metricsSystem: MetricsSystem, // The default value is only used in tests. override val taskMetrics: TaskMetrics = TaskMetrics.empty, - override val resources: Map[String, ResourceInformation] = Map.empty, - override val resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) + override val resources: Map[String, ResourceInformation] = Map.empty) extends TaskContext with Logging { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index a8508c59602bd..04591e87a72e4 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -35,7 +35,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} import org.apache.spark.internal.config.Python._ -import org.apache.spark.resource.ResourceProfile.{getPysparkMemoryFromInternalConfs, DEFAULT_RESOURCE_PROFILE_ID} +import org.apache.spark.resource.ResourceProfile.PYSPARK_MEMORY_PROPERTY import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -128,21 +128,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (reuseWorker) { envVars.put("SPARK_REUSE_WORKER", "1") } - val pymem = Option(context.getLocalProperty("spark.resource.pyspark.memory")) - logInfo("task context pyspark memory is: $pymem") - // Check to see if the pyspark memory conf is set for the resource profile id being used. - // Note we don't fallback to the cluster default if its not set in the profile because - // this is a totally optional config and the user may not want it set. - val rpId = context.resourceProfileId() - val memoryMb = if (rpId == DEFAULT_RESOURCE_PROFILE_ID) { - logInfo("Using the default pyspark executor memory") - conf.get(PYSPARK_EXECUTOR_MEMORY) - } else { - val mem = getPysparkMemoryFromInternalConfs(conf, rpId) - logInfo(s"Using profile $rpId pyspark executor memory $mem") - mem - - } + val memoryMb = Option(context.getLocalProperty(PYSPARK_MEMORY_PROPERTY)).map(_.toLong) + logInfo(s"task context pyspark memory is: $memoryMb") val workerMemoryMb = getWorkerMemoryMb(memoryMb) if (workerMemoryMb.isDefined) { envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", workerMemoryMb.get.toString) 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 572ab7bfd7160..6625457749f6a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -347,11 +347,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig(arguments.resourceProfileId)) - // we have to add the pyspark memory conf into SparkConfs so the PythonRunner can - // pick it up properly - val pysparkMemoryConf = cfg.resourceProfile.getInternalPysparkMemoryConfs - val props = cfg.sparkProperties ++ - Seq[(String, String)](("spark.app.id", arguments.appId)) ++ pysparkMemoryConf + val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", arguments.appId)) fetcher.shutdown() // Create SparkEnv using properties we fetched from the driver. diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 89e4d955d70e3..5e041293340da 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -51,8 +51,6 @@ class ResourceProfile( private var _limitingResource: Option[String] = None private var _maxTasksPerExecutor: Option[Int] = None private var _coresLimitKnown: Boolean = false - private var _internalPysparkMemoryConf: Seq[(String, String)] = - ResourceProfile.createPysparkMemoryInternalConfs(this) def id: Int = _id @@ -79,11 +77,7 @@ class ResourceProfile( } private[spark] def getPysparkMemory: Option[Long] = { - taskResources.get(ResourceProfile.PYSPARK_MEM).map(_.amount.toLong) - } - - private[spark] def getInternalPysparkMemoryConfs: Seq[(String, String)] = { - _internalPysparkMemoryConf + executorResources.get(ResourceProfile.PYSPARK_MEM).map(_.amount.toLong) } /* @@ -337,40 +331,4 @@ object ResourceProfile extends Logging { } private[spark] val PYSPARK_MEMORY_PROPERTY = "resource.pyspark.memory" - private[spark] val SPARK_RP_EXEC_PREFIX = "spark.resourceProfile.executor" - - // Helper class for constructing the resource profile internal configs. The configs look like: - // spark.resourceProfile.executor.[rpId].[resourceName].amount - private[spark] case class ResourceProfileInternalConf(rpId: Int, resourceName: String) { - val prefix = s"$SPARK_RP_EXEC_PREFIX.$rpId." - def resourceNameAndAmount: String = s"$resourceName.${ResourceUtils.AMOUNT}" - def amountConf: String = s"$prefix$resourceNameAndAmount" - } - - /** - * Create the ResourceProfile internal pyspark memory conf that is used by the executors. - * It pulls any pyspark.memory config from the profile and returns Seq of key and value - * where the key is formatted as: - * - * spark.resourceProfile.executor.[rpId].pyspark.memory.amount - */ - private[spark] def createPysparkMemoryInternalConfs( - rp: ResourceProfile - ): Seq[(String, String)] = { - rp.executorResources.get(ResourceProfile.PYSPARK_MEM).map { pysparkMem => - val pysparkMemIntConf = ResourceProfileInternalConf(rp.id, ResourceProfile.PYSPARK_MEM) - Seq((pysparkMemIntConf.amountConf, pysparkMem.amount.toString)) - }.getOrElse(Seq.empty) - } - - /** - * Get the pyspark memory setting from internal resource confs. - * The config looks like: spark.resourceProfile.executor.[rpId].pyspark.memory.amount - */ - private[spark] def getPysparkMemoryFromInternalConfs( - sparkConf: SparkConf, - rpId: Int): Option[Long] = { - val rName = ResourceProfile.PYSPARK_MEM - sparkConf.getOption(ResourceProfileInternalConf(rpId, rName).amountConf).map(_.toLong) - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 22dda935f0b60..0aefa82f78ba4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -34,12 +34,13 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.resource.ResourceProfile -import org.apache.spark.resource.ResourceProfile.PYSPARK_MEMORY_PROPERTY +import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, PYSPARK_MEMORY_PROPERTY} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1155,8 +1156,15 @@ private[spark] class DAGScheduler( // Use the scheduling pool, job group, description, etc. from an ActiveJob associated // with this Stage val properties = jobIdToActiveJob(jobId).properties - val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) - rp.getPysparkMemory.map(m => properties.setProperty(PYSPARK_MEMORY_PROPERTY, m.toString)) + val pysparkMem = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { + logDebug("Using the default pyspark executor memory") + sc.conf.get(PYSPARK_EXECUTOR_MEMORY) + } else { + val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) + logDebug(s"Using profile ${stage.resourceProfileId} pyspark executor memory") + rp.getPysparkMemory + } + pysparkMem.map(m => properties.setProperty(PYSPARK_MEMORY_PROPERTY, m.toString)) runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are @@ -1258,8 +1266,7 @@ private[spark] class DAGScheduler( stage.pendingPartitions += id new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), - Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(), - stage.resourceProfileId) + Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier()) } case stage: ResultStage => @@ -1270,7 +1277,7 @@ private[spark] class DAGScheduler( new ResultTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, id, properties, serializedTaskMetrics, Option(jobId), Option(sc.applicationId), sc.applicationAttemptId, - stage.rdd.isBarrier(), stage.resourceProfileId) + stage.rdd.isBarrier()) } } } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 174dff3241713..857c89d7a98f5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -25,7 +25,6 @@ import java.util.Properties import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID /** * A task that sends back the output to the driver application. @@ -51,7 +50,6 @@ import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. - * @param resourceProfileId the id of the ResourceProfile used with this task */ private[spark] class ResultTask[T, U]( stageId: Int, @@ -65,10 +63,9 @@ private[spark] class ResultTask[T, U]( jobId: Option[Int] = None, appId: Option[String] = None, appAttemptId: Option[String] = None, - isBarrier: Boolean = false, - resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) + isBarrier: Boolean = false) extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics, - jobId, appId, appAttemptId, isBarrier, resourceProfileId) + jobId, appId, appAttemptId, isBarrier) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 008dfc950b749..4c0c30a3caf67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -25,7 +25,6 @@ import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.{config, Logging} import org.apache.spark.rdd.RDD -import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID /** * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner @@ -49,7 +48,6 @@ import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. - * @param resourceProfileId the id of the ResourceProfile used with this task */ private[spark] class ShuffleMapTask( stageId: Int, @@ -62,10 +60,9 @@ private[spark] class ShuffleMapTask( jobId: Option[Int] = None, appId: Option[String] = None, appAttemptId: Option[String] = None, - isBarrier: Boolean = false, - resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) + isBarrier: Boolean = false) extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties, - serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, resourceProfileId) + serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index e785d4770ddac..ebc1c05435fee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -27,7 +27,6 @@ import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rdd.InputFileBlockHolder import org.apache.spark.resource.ResourceInformation -import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.util._ /** @@ -54,7 +53,6 @@ import org.apache.spark.util._ * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. - * @param resourceProfileId the id of the ResourceProfile used with this task */ private[spark] abstract class Task[T]( val stageId: Int, @@ -67,8 +65,7 @@ private[spark] abstract class Task[T]( val jobId: Option[Int] = None, val appId: Option[String] = None, val appAttemptId: Option[String] = None, - val isBarrier: Boolean = false, - val resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) extends Serializable { + val isBarrier: Boolean = false) extends Serializable { @transient lazy val metrics: TaskMetrics = SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) @@ -99,8 +96,7 @@ private[spark] abstract class Task[T]( localProperties, metricsSystem, metrics, - resources, - resourceProfileId) + resources) context = if (isBarrier) { new BarrierTaskContext(taskContext) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 0f62eeb804d41..2c792338b5295 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -469,7 +469,7 @@ private[spark] class TaskSetManager( s"taskResourceAssignments ${taskResourceAssignments}") sched.dagScheduler.taskStarted(task, info) - val desc = new TaskDescription( + new TaskDescription( taskId, attemptNum, execId, @@ -481,8 +481,6 @@ private[spark] class TaskSetManager( task.localProperties, taskResourceAssignments, serializedTask) - desc.properties.setProperty("spark.resource.pyspark.memory", "2048") - desc } } else { None From 27e1a1033d70b27a892ef68091766a5c658af3ee Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 31 Mar 2020 09:41:11 -0500 Subject: [PATCH 06/31] add missing api to get java map --- core/src/main/scala/org/apache/spark/TaskContextImpl.scala | 1 - .../org/apache/spark/resource/ExecutorResourceRequests.scala | 3 +++ .../scala/org/apache/spark/resource/TaskResourceRequests.scala | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index d9063a1790aab..08a58a029528b 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -29,7 +29,6 @@ import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source import org.apache.spark.resource.ResourceInformation -import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util._ diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index d4c29f9a70c44..b7a2e8dc61af2 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -17,6 +17,7 @@ package org.apache.spark.resource +import java.util.{Map => JMap} import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -38,6 +39,8 @@ private[spark] class ExecutorResourceRequests() extends Serializable { def requests: Map[String, ExecutorResourceRequest] = _executorResources.asScala.toMap + def requestsJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asScala.asJava + /** * Specify heap memory. The value specified will be converted to MiB. * diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala index 9a5114fc1d0a4..fed5d811a9d17 100644 --- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala @@ -17,6 +17,7 @@ package org.apache.spark.resource +import java.util.{Map => JMap} import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -37,6 +38,8 @@ private[spark] class TaskResourceRequests() extends Serializable { def requests: Map[String, TaskResourceRequest] = _taskResources.asScala.toMap + def requestsJMap: JMap[String, TaskResourceRequest] = _taskResources.asScala.asJava + /** * Specify number of cpus per Task. * From af602b6ee8a1e2dd2fc433a15ac940b8456acf9e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 31 Mar 2020 10:12:05 -0500 Subject: [PATCH 07/31] Add java api test --- .../java/test/org/apache/spark/JavaAPISuite.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index a8252e03b5c15..566d60ed406c4 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -73,6 +73,10 @@ import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; import org.apache.spark.rdd.RDD; +import org.apache.spark.resource.ExecutorResourceRequests; +import org.apache.spark.resource.ResourceProfile; +import org.apache.spark.resource.ResourceProfileBuilder; +import org.apache.spark.resource.TaskResourceRequests; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.LongAccumulator; @@ -897,6 +901,18 @@ public void persist() { assertEquals(1, rdd.first().intValue()); } + @Test + public void withResources() { + ExecutorResourceRequests ereqs = new ExecutorResourceRequests().cores(4); + TaskResourceRequests treqs = new TaskResourceRequests().cpus(1); + ResourceProfile rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build(); + List expected = Arrays.asList("1", "2", "3", "4"); + JavaRDD in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + in1.withResources(rp1); + assertEquals(in1.getResourceProfile(), rp1); + assertEquals(expected, in1.collect()); + } + @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); From 2b515c8945d1da64e33c64edc7f6d00433ca7b44 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 31 Mar 2020 14:55:39 -0500 Subject: [PATCH 08/31] cleanup --- .../spark/api/python/PythonRunner.scala | 1 - .../spark/resource/ResourceProfile.scala | 1 - .../apache/spark/scheduler/DAGScheduler.scala | 26 ++++++++++++------- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 04591e87a72e4..7cf214865280a 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -129,7 +129,6 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( envVars.put("SPARK_REUSE_WORKER", "1") } val memoryMb = Option(context.getLocalProperty(PYSPARK_MEMORY_PROPERTY)).map(_.toLong) - logInfo(s"task context pyspark memory is: $memoryMb") val workerMemoryMb = getWorkerMemoryMb(memoryMb) if (workerMemoryMb.isDefined) { envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", workerMemoryMb.get.toString) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 5e041293340da..1739f1c127603 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -29,7 +29,6 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY -import org.apache.spark.util.Utils /** * Resource profile to associate with an RDD. A ResourceProfile allows the user to diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0aefa82f78ba4..8e015f84f1120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1137,6 +1137,22 @@ private[spark] class DAGScheduler( } } + /** + * PythonRunner needs to know what the pyspark memory setting is for the profile being run. + * Pass it in the local properties of the task if it's set for the stage profile. + */ + private def addPysparkMemToProperties(stage: Stage, properties: Properties): Unit = { + val pysparkMem = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { + logDebug("Using the default pyspark executor memory") + sc.conf.get(PYSPARK_EXECUTOR_MEMORY) + } else { + val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) + logDebug(s"Using profile ${stage.resourceProfileId} pyspark executor memory") + rp.getPysparkMemory + } + pysparkMem.map(m => properties.setProperty(PYSPARK_MEMORY_PROPERTY, m.toString)) + } + /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int): Unit = { logDebug("submitMissingTasks(" + stage + ")") @@ -1156,15 +1172,7 @@ private[spark] class DAGScheduler( // Use the scheduling pool, job group, description, etc. from an ActiveJob associated // with this Stage val properties = jobIdToActiveJob(jobId).properties - val pysparkMem = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { - logDebug("Using the default pyspark executor memory") - sc.conf.get(PYSPARK_EXECUTOR_MEMORY) - } else { - val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) - logDebug(s"Using profile ${stage.resourceProfileId} pyspark executor memory") - rp.getPysparkMemory - } - pysparkMem.map(m => properties.setProperty(PYSPARK_MEMORY_PROPERTY, m.toString)) + addPysparkMemToProperties(stage, properties) runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are From a052427e9a5767fb837c779d0e8f109227f17831 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 31 Mar 2020 16:21:07 -0500 Subject: [PATCH 09/31] fix indentation --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 5caae89a3db69..1bcf0e5bb9144 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2656,7 +2656,7 @@ def id(self): def _is_pipelinable(self): return not (self.is_cached or self.is_checkpointed or self.has_resourceProfile) -def _is_barrier(self): + def _is_barrier(self): return self.is_barrier From 6a90fbed75bfdc67d45807295d6d093e6afaf778 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 31 Mar 2020 16:58:04 -0500 Subject: [PATCH 10/31] fix newline around markup in python --- python/pyspark/rdd.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1bcf0e5bb9144..b45743753b36e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2488,10 +2488,12 @@ def _is_barrier(self): def withResources(self, profile): """ .. note:: Experimental + Specify a ResourceProfile to use when calculating this RDD. This is only supported on certain cluster managers and currently requires dynamic allocation to be enabled. It will result in new executors with the resources specified being acquired to calculate the RDD. + .. versionadded:: 3.0.0 """ self.has_resourceProfile = True @@ -2501,8 +2503,10 @@ def withResources(self, profile): def getResourceProfile(self): """ .. note:: Experimental + Get the ResourceProfile specified with this RDD or None if it wasn't specified. :return: the user specified ResourceProfile or null if none was specified + .. versionadded:: 3.0.0 """ return ResourceProfile(self._jrdd.getResourceProfile()) From 2d754f7e630bab3b10dfebe142698067e667121e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 1 Apr 2020 15:59:15 -0500 Subject: [PATCH 11/31] Update the version added for rdd api's --- python/pyspark/rdd.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b45743753b36e..54895116fd600 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2494,7 +2494,7 @@ def withResources(self, profile): It will result in new executors with the resources specified being acquired to calculate the RDD. - .. versionadded:: 3.0.0 + .. versionadded:: 3.1.0 """ self.has_resourceProfile = True self._jrdd.withResources(profile._jResourceProfile) @@ -2507,7 +2507,7 @@ def getResourceProfile(self): Get the ResourceProfile specified with this RDD or None if it wasn't specified. :return: the user specified ResourceProfile or null if none was specified - .. versionadded:: 3.0.0 + .. versionadded:: 3.1.0 """ return ResourceProfile(self._jrdd.getResourceProfile()) From 1071f404baf2a32f270fcb0ac813d9fae763485d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 2 Apr 2020 08:28:23 -0500 Subject: [PATCH 12/31] make java return values immutable --- .../org/apache/spark/resource/ExecutorResourceRequests.scala | 2 +- .../scala/org/apache/spark/resource/TaskResourceRequests.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index b7a2e8dc61af2..7e5fe9a4295c0 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -39,7 +39,7 @@ private[spark] class ExecutorResourceRequests() extends Serializable { def requests: Map[String, ExecutorResourceRequest] = _executorResources.asScala.toMap - def requestsJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asScala.asJava + def requestsJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asScala.toMap.asJava /** * Specify heap memory. The value specified will be converted to MiB. diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala index fed5d811a9d17..c7bcd4d7e569d 100644 --- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala @@ -38,7 +38,7 @@ private[spark] class TaskResourceRequests() extends Serializable { def requests: Map[String, TaskResourceRequest] = _taskResources.asScala.toMap - def requestsJMap: JMap[String, TaskResourceRequest] = _taskResources.asScala.asJava + def requestsJMap: JMap[String, TaskResourceRequest] = _taskResources.asScala.toMap.asJava /** * Specify number of cpus per Task. From e81a480cae424c31ab551b25d24c086db4fae5ac Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Apr 2020 09:49:54 -0500 Subject: [PATCH 13/31] Try reverting java api suite --- .../java/test/org/apache/spark/JavaAPISuite.java | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index 566d60ed406c4..a8252e03b5c15 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -73,10 +73,6 @@ import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; import org.apache.spark.rdd.RDD; -import org.apache.spark.resource.ExecutorResourceRequests; -import org.apache.spark.resource.ResourceProfile; -import org.apache.spark.resource.ResourceProfileBuilder; -import org.apache.spark.resource.TaskResourceRequests; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.LongAccumulator; @@ -901,18 +897,6 @@ public void persist() { assertEquals(1, rdd.first().intValue()); } - @Test - public void withResources() { - ExecutorResourceRequests ereqs = new ExecutorResourceRequests().cores(4); - TaskResourceRequests treqs = new TaskResourceRequests().cpus(1); - ResourceProfile rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build(); - List expected = Arrays.asList("1", "2", "3", "4"); - JavaRDD in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - in1.withResources(rp1); - assertEquals(in1.getResourceProfile(), rp1); - assertEquals(expected, in1.collect()); - } - @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); From ae8e312383af1e26c530ac5b002efc7ce356441e Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Apr 2020 11:24:47 -0500 Subject: [PATCH 14/31] Fix minor review comments --- core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala | 2 +- .../org/apache/spark/resource/ExecutorResourceRequests.scala | 2 +- .../org/apache/spark/resource/TaskResourceRequests.scala | 2 +- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- python/pyspark/executorresourcerequest.py | 2 +- python/pyspark/taskresourcerequest.py | 2 +- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 36307e9fc325d..e4140f659d979 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -56,7 +56,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) * It will result in new executors with the resources specified being acquired to * calculate the RDD. */ - def withResources(prof: ResourceProfile): JavaRDD[T] = wrapRDD(rdd.withResources(prof)) + def withResources(rp: ResourceProfile): JavaRDD[T] = wrapRDD(rdd.withResources(rp)) /** * Get the ResourceProfile specified with this RDD or None if it wasn't specified. diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index 7e5fe9a4295c0..4ee1a07564042 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -39,7 +39,7 @@ private[spark] class ExecutorResourceRequests() extends Serializable { def requests: Map[String, ExecutorResourceRequest] = _executorResources.asScala.toMap - def requestsJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asScala.toMap.asJava + def requestsJMap: JMap[String, ExecutorResourceRequest] = requests.asJava /** * Specify heap memory. The value specified will be converted to MiB. diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala index c7bcd4d7e569d..09f4e02eee9e0 100644 --- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala @@ -38,7 +38,7 @@ private[spark] class TaskResourceRequests() extends Serializable { def requests: Map[String, TaskResourceRequest] = _taskResources.asScala.toMap - def requestsJMap: JMap[String, TaskResourceRequest] = _taskResources.asScala.toMap.asJava + def requestsJMap: JMap[String, TaskResourceRequest] = requests.asJava /** * Specify number of cpus per Task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8e015f84f1120..2e82ff626c816 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1138,8 +1138,8 @@ private[spark] class DAGScheduler( } /** - * PythonRunner needs to know what the pyspark memory setting is for the profile being run. - * Pass it in the local properties of the task if it's set for the stage profile. + * `PythonRunner` needs to know what the pyspark memory setting is for the profile being run. + * Pass it in the local properties of the task if it's set for the stage profile. */ private def addPysparkMemToProperties(stage: Stage, properties: Properties): Unit = { val pysparkMem = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { diff --git a/python/pyspark/executorresourcerequest.py b/python/pyspark/executorresourcerequest.py index 79c824f64cb5f..ce8ca3ccf171f 100644 --- a/python/pyspark/executorresourcerequest.py +++ b/python/pyspark/executorresourcerequest.py @@ -39,7 +39,7 @@ class ExecutorResourceRequest(object): See the configuration and cluster specific docs for more details. - Use ExecutorResourceRequests class as a convenience API. + Use `pyspark.ExecutorResourceRequests` class as a convenience API. :param resourceName: Name of the resource :param amount: Amount requesting diff --git a/python/pyspark/taskresourcerequest.py b/python/pyspark/taskresourcerequest.py index 784416ea57056..94a08b2911779 100644 --- a/python/pyspark/taskresourcerequest.py +++ b/python/pyspark/taskresourcerequest.py @@ -24,7 +24,7 @@ class TaskResourceRequest(object): programmatically specify the resources needed for an RDD that will be applied at the stage level. The amount is specified as a Double to allow for saying you want more then 1 task per resource. Valid values are less than or equal to 0.5 or whole numbers. - Use TaskResourceRequests class as a convenience API. + Use `pyspark.TaskResourceRequests` class as a convenience API. :param resourceName: Name of the resource :param amount: Amount requesting as a Double to support fractional resource requests. From 0d7c79fe5e76df4656e33ea9c8d975a0b8bac0dd Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 6 Apr 2020 15:03:55 -0500 Subject: [PATCH 15/31] Update to pass the executor cores into PythonRunner --- .../spark/api/python/PythonRunner.scala | 20 +++++++++++++------ .../spark/resource/ResourceProfile.scala | 3 ++- .../apache/spark/scheduler/DAGScheduler.scala | 13 +++++++----- 3 files changed, 24 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 7cf214865280a..a762ec82193ea 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -35,7 +35,7 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} import org.apache.spark.internal.config.Python._ -import org.apache.spark.resource.ResourceProfile.PYSPARK_MEMORY_PROPERTY +import org.apache.spark.resource.ResourceProfile.{EXECUTOR_CORES_LOCAL_PROPERTY, PYSPARK_MEMORY_LOCAL_PROPERTY} import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -106,8 +106,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( // each python worker gets an equal part of the allocation. the worker pool will grow to the // number of concurrent tasks, which is determined by the number of cores in this executor. - private def getWorkerMemoryMb(mem: Option[Long]): Option[Long] = { - mem.map(_ / conf.get(EXECUTOR_CORES)) + private def getWorkerMemoryMb(mem: Option[Long], cores: Int): Option[Long] = { + mem.map(_ / cores) } def compute( @@ -116,20 +116,28 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( context: TaskContext): Iterator[OUT] = { val startTime = System.currentTimeMillis val env = SparkEnv.get + + // Get the executor cores and pyspark memory, they are passed via the local properties when + // the user specified them in a ResourceProfile + val execCoresProp = Option(context.getLocalProperty(EXECUTOR_CORES_LOCAL_PROPERTY)) + val memoryMb = Option(context.getLocalProperty(PYSPARK_MEMORY_LOCAL_PROPERTY)).map(_.toLong) val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",") // if OMP_NUM_THREADS is not explicitly set, override it with the number of cores if (conf.getOption("spark.executorEnv.OMP_NUM_THREADS").isEmpty) { // SPARK-28843: limit the OpenMP thread pool to the number of cores assigned to this executor // this avoids high memory consumption with pandas/numpy because of a large OpenMP thread pool // see https://github.com/numpy/numpy/issues/10455 - conf.getOption("spark.executor.cores").foreach(envVars.put("OMP_NUM_THREADS", _)) + val coresOption = if (execCoresProp.isEmpty) conf.getOption("spark.executor.cores") else execCoresProp + coresOption.foreach(envVars.put("OMP_NUM_THREADS", _)) } envVars.put("SPARK_LOCAL_DIRS", localdir) // it's also used in monitor thread if (reuseWorker) { envVars.put("SPARK_REUSE_WORKER", "1") } - val memoryMb = Option(context.getLocalProperty(PYSPARK_MEMORY_PROPERTY)).map(_.toLong) - val workerMemoryMb = getWorkerMemoryMb(memoryMb) + // SPARK-30299 this could be wrong with standalone mode when executor + // cores might not be correct because it defaults to all cores on the box. + val execCores = execCoresProp.map(_.toInt).getOrElse(conf.get(EXECUTOR_CORES)) + val workerMemoryMb = getWorkerMemoryMb(memoryMb, execCores) if (workerMemoryMb.isDefined) { envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", workerMemoryMb.get.toString) } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 1739f1c127603..963864a5d1560 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -329,5 +329,6 @@ object ResourceProfile extends Logging { rp.getTaskCpus.getOrElse(conf.get(CPUS_PER_TASK)) } - private[spark] val PYSPARK_MEMORY_PROPERTY = "resource.pyspark.memory" + private[spark] val PYSPARK_MEMORY_LOCAL_PROPERTY = "resource.pyspark.memory" + private[spark] val EXECUTOR_CORES_LOCAL_PROPERTY = "resource.executor.cores" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2e82ff626c816..89e14b127576d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -40,7 +40,7 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.resource.ResourceProfile -import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, PYSPARK_MEMORY_PROPERTY} +import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, EXECUTOR_CORES_LOCAL_PROPERTY, PYSPARK_MEMORY_LOCAL_PROPERTY} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1142,15 +1142,18 @@ private[spark] class DAGScheduler( * Pass it in the local properties of the task if it's set for the stage profile. */ private def addPysparkMemToProperties(stage: Stage, properties: Properties): Unit = { - val pysparkMem = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { + val (pysparkMem, execCores) = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { logDebug("Using the default pyspark executor memory") - sc.conf.get(PYSPARK_EXECUTOR_MEMORY) + // use the getOption on spark.executor.cores instead of using the EXECUTOR_CORES config + // because the default for this config isn't correct for standalone mode + (sc.conf.get(PYSPARK_EXECUTOR_MEMORY), sc.conf.getOption("spark.executor.cores")) } else { val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) logDebug(s"Using profile ${stage.resourceProfileId} pyspark executor memory") - rp.getPysparkMemory + (rp.getPysparkMemory, rp.getExecutorCores.map(_.toString)) } - pysparkMem.map(m => properties.setProperty(PYSPARK_MEMORY_PROPERTY, m.toString)) + pysparkMem.map(mem => properties.setProperty(PYSPARK_MEMORY_LOCAL_PROPERTY, mem.toString)) + execCores.map(cores => properties.setProperty(EXECUTOR_CORES_LOCAL_PROPERTY, cores)) } /** Called when stage's parents are available and we can now do its task. */ From 956dc8441c8771fa6e1b7c61b62234a4d81fa2d6 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 7 Apr 2020 16:27:55 -0500 Subject: [PATCH 16/31] move python files to resources module and misc fixes --- .../spark/api/python/PythonRunner.scala | 6 +++- python/pyspark/__init__.py | 10 +----- python/pyspark/rdd.py | 2 +- python/pyspark/resource/__init__.py | 31 +++++++++++++++++++ .../{ => resource}/executorresourcerequest.py | 0 .../executorresourcerequests.py | 2 +- .../pyspark/{ => resource}/resourceprofile.py | 4 +-- .../{ => resource}/resourceprofilebuilder.py | 8 ++--- .../{ => resource}/taskresourcerequest.py | 0 .../{ => resource}/taskresourcerequests.py | 2 +- python/pyspark/worker.py | 16 ++++++---- 11 files changed, 56 insertions(+), 25 deletions(-) create mode 100644 python/pyspark/resource/__init__.py rename python/pyspark/{ => resource}/executorresourcerequest.py (100%) rename python/pyspark/{ => resource}/executorresourcerequests.py (96%) rename python/pyspark/{ => resource}/resourceprofile.py (93%) rename python/pyspark/{ => resource}/resourceprofilebuilder.py (91%) rename python/pyspark/{ => resource}/taskresourcerequest.py (100%) rename python/pyspark/{ => resource}/taskresourcerequests.py (96%) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index a762ec82193ea..ac82243d70b3f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -127,7 +127,11 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( // SPARK-28843: limit the OpenMP thread pool to the number of cores assigned to this executor // this avoids high memory consumption with pandas/numpy because of a large OpenMP thread pool // see https://github.com/numpy/numpy/issues/10455 - val coresOption = if (execCoresProp.isEmpty) conf.getOption("spark.executor.cores") else execCoresProp + val coresOption = if (execCoresProp.isEmpty) { + conf.getOption("spark.executor.cores") + } else { + execCoresProp + } coresOption.foreach(envVars.put("OMP_NUM_THREADS", _)) } envVars.put("SPARK_LOCAL_DIRS", localdir) // it's also used in monitor thread diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index e3c3254bdf58b..a73e32536d601 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -55,12 +55,6 @@ from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast from pyspark.resourceinformation import ResourceInformation -from pyspark.taskresourcerequest import TaskResourceRequest -from pyspark.executorresourcerequest import ExecutorResourceRequest -from pyspark.taskresourcerequests import TaskResourceRequests -from pyspark.executorresourcerequests import ExecutorResourceRequests -from pyspark.resourceprofilebuilder import ResourceProfileBuilder -from pyspark.resourceprofile import ResourceProfile from pyspark.serializers import MarshalSerializer, PickleSerializer from pyspark.status import * from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo @@ -125,7 +119,5 @@ def wrapper(self, *args, **kwargs): "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", - "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation", - "TaskResourceRequest", "TaskResourceRequests", "ExecutorResourceRequest", - "ExecutorResourceRequests", "ResourceProfile" + "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation" ] diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 54895116fd600..b5fa8fd0f73c4 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -47,7 +47,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel -from pyspark.resourceprofile import ResourceProfile +from pyspark.resource.resourceprofile import ResourceProfile from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy diff --git a/python/pyspark/resource/__init__.py b/python/pyspark/resource/__init__.py new file mode 100644 index 0000000000000..73a3e22a31b77 --- /dev/null +++ b/python/pyspark/resource/__init__.py @@ -0,0 +1,31 @@ +# +# 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. +# + +""" +APIs to let users manipulate resource requirements. +""" +from pyspark.resource.taskresourcerequest import TaskResourceRequest +from pyspark.resource.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.taskresourcerequests import TaskResourceRequests +from pyspark.resource.executorresourcerequests import ExecutorResourceRequests +from pyspark.resource.resourceprofilebuilder import ResourceProfileBuilder +from pyspark.resource.resourceprofile import ResourceProfile + +__all__ = [ + "TaskResourceRequest", "TaskResourceRequests", "ExecutorResourceRequest", + "ExecutorResourceRequests", "ResourceProfile", "ResourceProfileBuilder" +] diff --git a/python/pyspark/executorresourcerequest.py b/python/pyspark/resource/executorresourcerequest.py similarity index 100% rename from python/pyspark/executorresourcerequest.py rename to python/pyspark/resource/executorresourcerequest.py diff --git a/python/pyspark/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py similarity index 96% rename from python/pyspark/executorresourcerequests.py rename to python/pyspark/resource/executorresourcerequests.py index 761282b38a795..9080b38bac2e2 100644 --- a/python/pyspark/executorresourcerequests.py +++ b/python/pyspark/resource/executorresourcerequests.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.executorresourcerequest import ExecutorResourceRequest class ExecutorResourceRequests(object): diff --git a/python/pyspark/resourceprofile.py b/python/pyspark/resource/resourceprofile.py similarity index 93% rename from python/pyspark/resourceprofile.py rename to python/pyspark/resource/resourceprofile.py index 62e8d70590f21..f6c67585a89c4 100644 --- a/python/pyspark/resourceprofile.py +++ b/python/pyspark/resource/resourceprofile.py @@ -15,8 +15,8 @@ # limitations under the License. # -from pyspark.executorresourcerequest import ExecutorResourceRequest -from pyspark.taskresourcerequest import TaskResourceRequest +from pyspark.resource.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.taskresourcerequest import TaskResourceRequest class ResourceProfile(object): diff --git a/python/pyspark/resourceprofilebuilder.py b/python/pyspark/resource/resourceprofilebuilder.py similarity index 91% rename from python/pyspark/resourceprofilebuilder.py rename to python/pyspark/resource/resourceprofilebuilder.py index 2fe3eca1a9b62..ecf1399022aef 100644 --- a/python/pyspark/resourceprofilebuilder.py +++ b/python/pyspark/resource/resourceprofilebuilder.py @@ -15,10 +15,10 @@ # limitations under the License. # -from pyspark.executorresourcerequest import ExecutorResourceRequest -from pyspark.resourceprofile import ResourceProfile -from pyspark.taskresourcerequest import TaskResourceRequest -from pyspark.taskresourcerequests import TaskResourceRequests +from pyspark.resource.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.resourceprofile import ResourceProfile +from pyspark.resource.taskresourcerequest import TaskResourceRequest +from pyspark.resource.taskresourcerequests import TaskResourceRequests class ResourceProfileBuilder(object): diff --git a/python/pyspark/taskresourcerequest.py b/python/pyspark/resource/taskresourcerequest.py similarity index 100% rename from python/pyspark/taskresourcerequest.py rename to python/pyspark/resource/taskresourcerequest.py diff --git a/python/pyspark/taskresourcerequests.py b/python/pyspark/resource/taskresourcerequests.py similarity index 96% rename from python/pyspark/taskresourcerequests.py rename to python/pyspark/resource/taskresourcerequests.py index 76cc177c1db90..1f2afc73e5e95 100644 --- a/python/pyspark/taskresourcerequests.py +++ b/python/pyspark/resource/taskresourcerequests.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark.taskresourcerequest import TaskResourceRequest +from pyspark.resource.taskresourcerequest import TaskResourceRequest class TaskResourceRequests(object): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 7d62bf1783931..70222d258ee4f 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -25,7 +25,11 @@ # 'resource' is a Unix specific module. has_resource_module = True try: - import resource + from resource import RLIMIT_AS + from resource import RLIM_INFINITY + from resource import getrlimit + from resource import setrlimit + from resource import error except ImportError: has_resource_module = False import traceback @@ -478,21 +482,21 @@ def main(infile, outfile): # set up memory limits memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1")) if memory_limit_mb > 0 and has_resource_module: - total_memory = resource.RLIMIT_AS + total_memory = RLIMIT_AS try: - (soft_limit, hard_limit) = resource.getrlimit(total_memory) + (soft_limit, hard_limit) = getrlimit(total_memory) msg = "Current mem limits: {0} of max {1}\n".format(soft_limit, hard_limit) print(msg, file=sys.stderr) # convert to bytes new_limit = memory_limit_mb * 1024 * 1024 - if soft_limit == resource.RLIM_INFINITY or new_limit < soft_limit: + if soft_limit == RLIM_INFINITY or new_limit < soft_limit: msg = "Setting mem limits to {0} of max {1}\n".format(new_limit, new_limit) print(msg, file=sys.stderr) - resource.setrlimit(total_memory, (new_limit, new_limit)) + setrlimit(total_memory, (new_limit, new_limit)) - except (resource.error, OSError, ValueError) as e: + except (error, OSError, ValueError) as e: # not all systems support resource limits, so warn instead of failing print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr) From 32bca9518c59ea46a5b9296e9bcf3780ea2effb0 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 7 Apr 2020 17:47:04 -0500 Subject: [PATCH 17/31] Fix how we pass the pyspark memory and cores --- .../spark/api/python/PythonRunner.scala | 9 ++----- .../apache/spark/scheduler/DAGScheduler.scala | 27 ++++++++++--------- 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index ac82243d70b3f..b3d8780190fc2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -118,7 +118,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val env = SparkEnv.get // Get the executor cores and pyspark memory, they are passed via the local properties when - // the user specified them in a ResourceProfile + // the user specified them in a ResourceProfile. val execCoresProp = Option(context.getLocalProperty(EXECUTOR_CORES_LOCAL_PROPERTY)) val memoryMb = Option(context.getLocalProperty(PYSPARK_MEMORY_LOCAL_PROPERTY)).map(_.toLong) val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",") @@ -127,12 +127,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( // SPARK-28843: limit the OpenMP thread pool to the number of cores assigned to this executor // this avoids high memory consumption with pandas/numpy because of a large OpenMP thread pool // see https://github.com/numpy/numpy/issues/10455 - val coresOption = if (execCoresProp.isEmpty) { - conf.getOption("spark.executor.cores") - } else { - execCoresProp - } - coresOption.foreach(envVars.put("OMP_NUM_THREADS", _)) + execCoresProp.foreach(envVars.put("OMP_NUM_THREADS", _)) } envVars.put("SPARK_LOCAL_DIRS", localdir) // it's also used in monitor thread if (reuseWorker) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 89e14b127576d..d4e7973e110bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -34,7 +34,6 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config -import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} @@ -1138,19 +1137,21 @@ private[spark] class DAGScheduler( } /** - * `PythonRunner` needs to know what the pyspark memory setting is for the profile being run. - * Pass it in the local properties of the task if it's set for the stage profile. + * `PythonRunner` needs to know what the pyspark memory and cores settings are for the profile + * being run. Pass them in the local properties of the task if it's set for the stage profile. */ - private def addPysparkMemToProperties(stage: Stage, properties: Properties): Unit = { - val (pysparkMem, execCores) = if (stage.resourceProfileId == DEFAULT_RESOURCE_PROFILE_ID) { - logDebug("Using the default pyspark executor memory") - // use the getOption on spark.executor.cores instead of using the EXECUTOR_CORES config - // because the default for this config isn't correct for standalone mode - (sc.conf.get(PYSPARK_EXECUTOR_MEMORY), sc.conf.getOption("spark.executor.cores")) + private def addPysparkConfigsToProperties(stage: Stage, properties: Properties): Unit = { + val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) + val pysparkMem = rp.getPysparkMemory + // use the getOption on EXECUTOR_CORES.key instead of using the EXECUTOR_CORES config reader + // because the default for this config isn't correct for standalone mode. Here we want + // to know if it was explicitly set or not. The default profile always has it set to either + // what user specified or default so special case it here. + val execCores = if (rp.id == DEFAULT_RESOURCE_PROFILE_ID) { + sc.conf.getOption(config.EXECUTOR_CORES.key) } else { - val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) - logDebug(s"Using profile ${stage.resourceProfileId} pyspark executor memory") - (rp.getPysparkMemory, rp.getExecutorCores.map(_.toString)) + val profCores = rp.getExecutorCores.map(_.toString) + if (profCores.isEmpty) sc.conf.getOption(config.EXECUTOR_CORES.key) else profCores } pysparkMem.map(mem => properties.setProperty(PYSPARK_MEMORY_LOCAL_PROPERTY, mem.toString)) execCores.map(cores => properties.setProperty(EXECUTOR_CORES_LOCAL_PROPERTY, cores)) @@ -1175,7 +1176,7 @@ private[spark] class DAGScheduler( // Use the scheduling pool, job group, description, etc. from an ActiveJob associated // with this Stage val properties = jobIdToActiveJob(jobId).properties - addPysparkMemToProperties(stage, properties) + addPysparkConfigsToProperties(stage, properties) runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are From e494c054ae190ea63009adcf8e3dfc5f11aee257 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 8 Apr 2020 08:48:15 -0500 Subject: [PATCH 18/31] Fix python imports --- python/pyspark/tests/test_rdd.py | 5 +++-- python/pyspark/worker.py | 17 +++++++---------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index 10f53841fb69e..3c12f8ff4c8a6 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -25,8 +25,9 @@ from py4j.protocol import Py4JJavaError -from pyspark import ExecutorResourceRequests, shuffle, RDD, ResourceProfile,\ - ResourceProfileBuilder, TaskResourceRequests +from pyspark import shuffle, RDD +from pyspark.resource import ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder,\ + TaskResourceRequests from pyspark.serializers import CloudPickleSerializer, BatchedSerializer, PickleSerializer,\ MarshalSerializer, UTF8Deserializer, NoOpSerializer from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME, QuietTest diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 70222d258ee4f..32867f914bbd5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -19,17 +19,14 @@ Worker that receives input from Piped RDD. """ from __future__ import print_function +from __future__ import absolute_import import os import sys import time # 'resource' is a Unix specific module. has_resource_module = True try: - from resource import RLIMIT_AS - from resource import RLIM_INFINITY - from resource import getrlimit - from resource import setrlimit - from resource import error + import resource except ImportError: has_resource_module = False import traceback @@ -482,21 +479,21 @@ def main(infile, outfile): # set up memory limits memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1")) if memory_limit_mb > 0 and has_resource_module: - total_memory = RLIMIT_AS + total_memory = resource.RLIMIT_AS try: - (soft_limit, hard_limit) = getrlimit(total_memory) + (soft_limit, hard_limit) = resource.getrlimit(total_memory) msg = "Current mem limits: {0} of max {1}\n".format(soft_limit, hard_limit) print(msg, file=sys.stderr) # convert to bytes new_limit = memory_limit_mb * 1024 * 1024 - if soft_limit == RLIM_INFINITY or new_limit < soft_limit: + if soft_limit == resource.RLIM_INFINITY or new_limit < soft_limit: msg = "Setting mem limits to {0} of max {1}\n".format(new_limit, new_limit) print(msg, file=sys.stderr) - setrlimit(total_memory, (new_limit, new_limit)) + resource.setrlimit(total_memory, (new_limit, new_limit)) - except (error, OSError, ValueError) as e: + except (resource.error, OSError, ValueError) as e: # not all systems support resource limits, so warn instead of failing print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr) From 3e15ed9a4a88b76bc26626a4c9a711149a86760f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 8 Apr 2020 09:15:26 -0500 Subject: [PATCH 19/31] Fix java api suite test from hanging --- .../java/test/org/apache/spark/JavaAPISuite.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index a8252e03b5c15..82b11de29d350 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -73,6 +73,10 @@ import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; import org.apache.spark.rdd.RDD; +import org.apache.spark.resource.ExecutorResourceRequests; +import org.apache.spark.resource.ResourceProfile; +import org.apache.spark.resource.ResourceProfileBuilder; +import org.apache.spark.resource.TaskResourceRequests; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.LongAccumulator; @@ -897,6 +901,16 @@ public void persist() { assertEquals(1, rdd.first().intValue()); } + @Test + public void withResources() { + ExecutorResourceRequests ereqs = new ExecutorResourceRequests().cores(4); + TaskResourceRequests treqs = new TaskResourceRequests().cpus(1); + ResourceProfile rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build(); + JavaRDD in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + in1.withResources(rp1); + assertEquals(rp1, in1.getResourceProfile()); + } + @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); From c3c885aec0bc4070154425b6bce5fec4f4928bae Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 8 Apr 2020 12:38:42 -0500 Subject: [PATCH 20/31] change test to not rely on being able to set pyspark memory after startup --- python/pyspark/tests/test_worker.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/tests/test_worker.py b/python/pyspark/tests/test_worker.py index 9d7deb23da604..dba9298ee161a 100644 --- a/python/pyspark/tests/test_worker.py +++ b/python/pyspark/tests/test_worker.py @@ -29,6 +29,7 @@ from py4j.protocol import Py4JJavaError +from pyspark import SparkConf, SparkContext from pyspark.testing.utils import ReusedPySparkTestCase, PySparkTestCase, QuietTest if sys.version_info[0] >= 3: @@ -180,10 +181,14 @@ def test_reuse_worker_of_parallelize_xrange(self): not has_resource_module, "Memory limit feature in Python worker is dependent on " "Python's 'resource' module; however, not found.") -class WorkerMemoryTest(PySparkTestCase): +class WorkerMemoryTest(unittest.TestCase): + + def setUp(self): + class_name = self.__class__.__name__ + conf = SparkConf().set("spark.executor.pyspark.memory", "2g") + self.sc = SparkContext('local[4]', class_name, conf=conf) def test_memory_limit(self): - self.sc._conf.set("spark.executor.pyspark.memory", "2g") rdd = self.sc.parallelize(xrange(1), 1) def getrlimit(): @@ -197,6 +202,8 @@ def getrlimit(): self.assertEqual(soft_limit, 2 * 1024 * 1024 * 1024) self.assertEqual(hard_limit, 2 * 1024 * 1024 * 1024) + def tearDown(self): + self.sc.stop() if __name__ == "__main__": import unittest From 3562539f6c8c668eae230971ca268672bb4787bb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 8 Apr 2020 15:50:40 -0500 Subject: [PATCH 21/31] Add in pyspark.resource module --- python/setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/setup.py b/python/setup.py index 622e6077dc36e..e823eb9a80fc7 100755 --- a/python/setup.py +++ b/python/setup.py @@ -183,6 +183,7 @@ def _supports_symlinks(): 'pyspark.python.lib', 'pyspark.data', 'pyspark.licenses', + 'pyspark.resource', 'pyspark.examples.src.main.python'], include_package_data=True, package_dir={ From 544119e99ea0c00fa3b3c09498161b0eb5cba282 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 13 Apr 2020 13:13:42 -0500 Subject: [PATCH 22/31] review comments --- .../spark/resource/ResourceProfile.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- python/pyspark/__init__.py | 2 +- python/pyspark/rdd.py | 11 +++++---- python/pyspark/resource/__init__.py | 2 +- .../resource/executorresourcerequest.py | 17 ++++++++----- .../resource/executorresourcerequests.py | 23 +++++++++++------- python/pyspark/resource/resourceprofile.py | 10 ++++---- .../resource/resourceprofilebuilder.py | 20 +++++++++------- .../pyspark/resource/taskresourcerequest.py | 24 ++++++++++++------- .../pyspark/resource/taskresourcerequests.py | 19 +++++++++------ 11 files changed, 79 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 963864a5d1560..e072f2d27f60d 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -75,7 +75,7 @@ class ResourceProfile( taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt) } - private[spark] def getPysparkMemory: Option[Long] = { + private[spark] def getPySparkMemory: Option[Long] = { executorResources.get(ResourceProfile.PYSPARK_MEM).map(_.amount.toLong) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d4e7973e110bd..e7f84770cf56a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1142,7 +1142,7 @@ private[spark] class DAGScheduler( */ private def addPysparkConfigsToProperties(stage: Stage, properties: Properties): Unit = { val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) - val pysparkMem = rp.getPysparkMemory + val pysparkMem = rp.getPySparkMemory // use the getOption on EXECUTOR_CORES.key instead of using the EXECUTOR_CORES config reader // because the default for this config isn't correct for standalone mode. Here we want // to know if it was explicitly set or not. The default profile always has it set to either diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index a73e32536d601..76a5bd0d645ba 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -119,5 +119,5 @@ def wrapper(self, *args, **kwargs): "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", - "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation" + "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation", ] diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b5fa8fd0f73c4..756dd33d5008d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2489,10 +2489,10 @@ def withResources(self, profile): """ .. note:: Experimental - Specify a ResourceProfile to use when calculating this RDD. This is only supported on - certain cluster managers and currently requires dynamic allocation to be enabled. - It will result in new executors with the resources specified being acquired to - calculate the RDD. + Specify a :class:`pyspark.resource.ResourceProfile` to use when calculating this RDD. + This is only supported on certain cluster managers and currently requires dynamic + allocation to be enabled. It will result in new executors with the resources specified + being acquired to calculate the RDD. .. versionadded:: 3.1.0 """ @@ -2504,7 +2504,8 @@ def getResourceProfile(self): """ .. note:: Experimental - Get the ResourceProfile specified with this RDD or None if it wasn't specified. + Get the :class:`pyspark.resource.ResourceProfile` specified with this RDD or None + if it wasn't specified. :return: the user specified ResourceProfile or null if none was specified .. versionadded:: 3.1.0 diff --git a/python/pyspark/resource/__init__.py b/python/pyspark/resource/__init__.py index 73a3e22a31b77..cddc10874cd1a 100644 --- a/python/pyspark/resource/__init__.py +++ b/python/pyspark/resource/__init__.py @@ -27,5 +27,5 @@ __all__ = [ "TaskResourceRequest", "TaskResourceRequests", "ExecutorResourceRequest", - "ExecutorResourceRequests", "ResourceProfile", "ResourceProfileBuilder" + "ExecutorResourceRequests", "ResourceProfile", "ResourceProfileBuilder", ] diff --git a/python/pyspark/resource/executorresourcerequest.py b/python/pyspark/resource/executorresourcerequest.py index ce8ca3ccf171f..2de8dee5e12d2 100644 --- a/python/pyspark/resource/executorresourcerequest.py +++ b/python/pyspark/resource/executorresourcerequest.py @@ -48,26 +48,31 @@ class ExecutorResourceRequest(object): allocated. The script runs on Executors startup to discover the addresses of the resources available. :param vendor: Vendor, required for some cluster managers + + .. versionadded:: 3.1.0 """ def __init__(self, resourceName, amount, discoveryScript="", vendor=""): - """Create a new ExecutorResourceRequest that wraps the underlying JVM object.""" + """ + Create a new :class:`pyspark.resource.ExecutorResourceRequest` that wraps the + underlying JVM object. + """ from pyspark.context import SparkContext - self._jExecRequest = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequest( + self._java_exec_request = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequest( resourceName, amount, discoveryScript, vendor) @property def resourceName(self): - return self._jExecRequest.resourceName() + return self._java_exec_request.resourceName() @property def amount(self): - return self._jExecRequest.amount() + return self._java_exec_request.amount() @property def discoveryScript(self): - return self._jExecRequest.discoveryScript() + return self._java_exec_request.discoveryScript() @property def vendor(self): - return self._jExecRequest.vendor() + return self._java_exec_request.vendor() diff --git a/python/pyspark/resource/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py index 9080b38bac2e2..0cc1776e50581 100644 --- a/python/pyspark/resource/executorresourcerequests.py +++ b/python/pyspark/resource/executorresourcerequests.py @@ -26,37 +26,42 @@ class ExecutorResourceRequests(object): A set of Executor resource requests. This is used in conjunction with the ResourceProfileBuilder to programmatically specify the resources needed for an RDD that will be applied at the stage level. + + .. versionadded:: 3.1.0 """ def __init__(self): - """Create a new ExecutorResourceRequests that wraps the underlying JVM object.""" + """ + Create a new :class:`pyspark.resource.ExecutorResourceRequests` that wraps the + underlying JVM object. + """ from pyspark import SparkContext - self._javaExecutorResourceRequests \ - = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequests() + self._java_executor_resource_requests = \ + SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequests() def memory(self, amount): - self._javaExecutorResourceRequests.memory(amount) + self._java_executor_resource_requests.memory(amount) return self def memoryOverhead(self, amount): - self._javaExecutorResourceRequests.memoryOverhead(amount) + self._java_executor_resource_requests.memoryOverhead(amount) return self def pysparkMemory(self, amount): - self._javaExecutorResourceRequests.pysparkMemory(amount) + self._java_executor_resource_requests.pysparkMemory(amount) return self def cores(self, amount): - self._javaExecutorResourceRequests.cores(amount) + self._java_executor_resource_requests.cores(amount) return self def resource(self, resourceName, amount, discoveryScript="", vendor=""): - self._javaExecutorResourceRequests.resource(resourceName, amount, discoveryScript, vendor) + self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, vendor) return self @property def requests(self): - execRes = self._javaExecutorResourceRequests.requestsJMap() + execRes = self._java_executor_resource_requests.requestsJMap() result = {} # convert back to python ExecutorResourceRequest for k, v in execRes.items(): diff --git a/python/pyspark/resource/resourceprofile.py b/python/pyspark/resource/resourceprofile.py index f6c67585a89c4..91cd79aa72302 100644 --- a/python/pyspark/resource/resourceprofile.py +++ b/python/pyspark/resource/resourceprofile.py @@ -28,14 +28,16 @@ class ResourceProfile(object): specify executor and task requirements for an RDD that will get applied during a stage. This allows the user to change the resource requirements between stages. This is meant to be immutable so user doesn't change it after building. + + .. versionadded:: 3.1.0 """ - def __init__(self, _jResourceProfile): - self._jResourceProfile = _jResourceProfile + def __init__(self, _java_resource_profile): + self._java_resource_profile = _java_resource_profile @property def taskResources(self): - taskRes = self._jResourceProfile.taskResourcesJMap() + taskRes = self._java_resource_profile.taskResourcesJMap() result = {} # convert back to python TaskResourceRequest for k, v in taskRes.items(): @@ -44,7 +46,7 @@ def taskResources(self): @property def executorResources(self): - execRes = self._jResourceProfile.executorResourcesJMap() + execRes = self._java_resource_profile.executorResourcesJMap() result = {} # convert back to python ExecutorResourceRequest for k, v in execRes.items(): diff --git a/python/pyspark/resource/resourceprofilebuilder.py b/python/pyspark/resource/resourceprofilebuilder.py index ecf1399022aef..5e861282490d9 100644 --- a/python/pyspark/resource/resourceprofilebuilder.py +++ b/python/pyspark/resource/resourceprofilebuilder.py @@ -30,30 +30,32 @@ class ResourceProfileBuilder(object): A ResourceProfile allows the user to specify executor and task requirements for an RDD that will get applied during a stage. This allows the user to change the resource requirements between stages. + + .. versionadded:: 3.1.0 """ def __init__(self, ): """Create a new ResourceProfileBuilder that wraps the underlying JVM object.""" from pyspark.context import SparkContext - self._jResourceProfileBuilder \ - = SparkContext._jvm.org.apache.spark.resource.ResourceProfileBuilder() + self._java_resource_profile_builder = \ + SparkContext._jvm.org.apache.spark.resource.ResourceProfileBuilder() def require(self, resourceRequest): if isinstance(resourceRequest, TaskResourceRequests): - self._jResourceProfileBuilder.require(resourceRequest._javaTaskResourceRequests) + self._java_resource_profile_builder.require(resourceRequest._java_task_resource_requests) else: - self._jResourceProfileBuilder.require(resourceRequest._javaExecutorResourceRequests) + self._java_resource_profile_builder.require(resourceRequest._java_executor_resource_requests) return self def clearExecutorResourceRequests(self): - self._jResourceProfileBuilder.clearExecutorResourceRequests() + self._java_resource_profile_builder.clearExecutorResourceRequests() def clearTaskResourceRequests(self): - self._jResourceProfileBuilder.clearTaskResourceRequests() + self._java_resource_profile_builder.clearTaskResourceRequests() @property def taskResources(self): - taskRes = self._jResourceProfileBuilder.taskResourcesJMap() + taskRes = self._java_resource_profile_builder.taskResourcesJMap() result = {} # convert back to python TaskResourceRequest for k, v in taskRes.items(): @@ -62,7 +64,7 @@ def taskResources(self): @property def executorResources(self): - execRes = self._jResourceProfileBuilder.executorResourcesJMap() + execRes = self._java_resource_profile_builder.executorResourcesJMap() result = {} # convert back to python ExecutorResourceRequest for k, v in execRes.items(): @@ -72,5 +74,5 @@ def executorResources(self): @property def build(self): - jresourceProfile = self._jResourceProfileBuilder.build() + jresourceProfile = self._java_resource_profile_builder.build() return ResourceProfile(jresourceProfile) diff --git a/python/pyspark/resource/taskresourcerequest.py b/python/pyspark/resource/taskresourcerequest.py index 94a08b2911779..5d79ea35284b1 100644 --- a/python/pyspark/resource/taskresourcerequest.py +++ b/python/pyspark/resource/taskresourcerequest.py @@ -20,26 +20,32 @@ class TaskResourceRequest(object): """ .. note:: Evolving - A task resource request. This is used in conjuntion with the ResourceProfile to - programmatically specify the resources needed for an RDD that will be applied at the - stage level. The amount is specified as a Double to allow for saying you want more then - 1 task per resource. Valid values are less than or equal to 0.5 or whole numbers. - Use `pyspark.TaskResourceRequests` class as a convenience API. + A task resource request. This is used in conjuntion with the + :class:`pyspark.resource.ResourceProfile` to programmatically specify the resources + needed for an RDD that will be applied at the stage level. The amount is specified + as a Double to allow for saying you want more then 1 task per resource. Valid values + are less than or equal to 0.5 or whole numbers. + Use :class:`pyspark.resource.TaskResourceRequests` class as a convenience API. :param resourceName: Name of the resource :param amount: Amount requesting as a Double to support fractional resource requests. Valid values are less than or equal to 0.5 or whole numbers. + + .. versionadded:: 3.1.0 """ def __init__(self, resourceName, amount): - """Create a new TaskResourceRequest that wraps the underlying JVM object.""" + """ + Create a new :class:`pyspark.resource.TaskResourceRequest` that wraps the underlying + JVM object. + """ from pyspark.context import SparkContext - self._jTaskRequest = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( + self._java_task_request = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( resourceName, amount) @property def resourceName(self): - return self._jTaskRequest.resourceName() + return self._java_task_request.resourceName() @property def amount(self): - return self._jTaskRequest.amount() + return self._java_task_request.amount() diff --git a/python/pyspark/resource/taskresourcerequests.py b/python/pyspark/resource/taskresourcerequests.py index 1f2afc73e5e95..79c0c0490234f 100644 --- a/python/pyspark/resource/taskresourcerequests.py +++ b/python/pyspark/resource/taskresourcerequests.py @@ -24,27 +24,32 @@ class TaskResourceRequests(object): .. note:: Evolving A set of task resource requests. This is used in conjuntion with the - ResourceProfileBuilder to programmatically specify the resources needed for + :class:`pyspark.resource.ResourceProfileBuilder` to programmatically specify the resources needed for an RDD that will be applied at the stage level. + + .. versionadded:: 3.1.0 """ def __init__(self): - """Create a new TaskResourceRequests that wraps the underlying JVM object.""" + """ + Create a new :class:`pyspark.resource.TaskResourceRequests` that wraps the underlying + JVM object. + """ from pyspark import SparkContext - self._javaTaskResourceRequests \ - = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() + self._java_task_resource_requests = \ + SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() def cpus(self, amount): - self._javaTaskResourceRequests.cpus(amount) + self._java_task_resource_requests.cpus(amount) return self def resource(self, resourceName, amount): - self._javaTaskResourceRequests.resource(resourceName, float(amount)) + self._java_task_resource_requests.resource(resourceName, float(amount)) return self @property def requests(self): - taskRes = self._javaTaskResourceRequests.requestsJMap() + taskRes = self._java_task_resource_requests.requestsJMap() result = {} # convert back to python TaskResourceRequest for k, v in taskRes.items(): From 84690384a92d4811981f4c9837c4c52ec9cff51d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 15 Apr 2020 15:30:27 -0500 Subject: [PATCH 23/31] Changes to allow using resource apis without SparkContext --- python/pyspark/rdd.py | 16 +++++- .../resource/executorresourcerequest.py | 32 +++++++++-- .../resource/executorresourcerequests.py | 46 ++++++++++++--- python/pyspark/resource/resourceprofile.py | 28 ++++++--- .../resource/resourceprofilebuilder.py | 57 +++++++++++++++---- .../pyspark/resource/taskresourcerequest.py | 20 +++++-- .../pyspark/resource/taskresourcerequests.py | 39 ++++++++++--- 7 files changed, 188 insertions(+), 50 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 756dd33d5008d..d33fa3d31fbfa 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -47,7 +47,9 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel +from pyspark.resource.executorresourcerequests import ExecutorResourceRequests from pyspark.resource.resourceprofile import ResourceProfile +from pyspark.resource.taskresourcerequests import TaskResourceRequests from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy @@ -2497,7 +2499,17 @@ def withResources(self, profile): .. versionadded:: 3.1.0 """ self.has_resourceProfile = True - self._jrdd.withResources(profile._jResourceProfile) + if profile._java_resource_profile is not None: + jrp = profile._java_resource_profile + else: + builder = self.ctx._jvm.org.apache.spark.resource.ResourceProfileBuilder() + ereqs = ExecutorResourceRequests(self.ctx._jvm, profile._executor_resource_request) + treqs = TaskResourceRequests(self.ctx._jvm, profile._task_resource_request) + builder.require(ereqs._java_executor_resource_requests) + builder.require(treqs._java_task_resource_requests) + jrp = builder.build() + + self._jrdd.withResources(jrp) return self def getResourceProfile(self): @@ -2510,7 +2522,7 @@ def getResourceProfile(self): .. versionadded:: 3.1.0 """ - return ResourceProfile(self._jrdd.getResourceProfile()) + return ResourceProfile(_java_resource_profile = self._jrdd.getResourceProfile()) def _prepare_for_python_RDD(sc, command): diff --git a/python/pyspark/resource/executorresourcerequest.py b/python/pyspark/resource/executorresourcerequest.py index 2de8dee5e12d2..19fb210a39962 100644 --- a/python/pyspark/resource/executorresourcerequest.py +++ b/python/pyspark/resource/executorresourcerequest.py @@ -58,21 +58,41 @@ def __init__(self, resourceName, amount, discoveryScript="", vendor=""): underlying JVM object. """ from pyspark.context import SparkContext - self._java_exec_request = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequest( - resourceName, amount, discoveryScript, vendor) + _jvm = SparkContext._jvm + if _jvm is not None: + self._java_exec_request = _jvm.org.apache.spark.resource.ExecutorResourceRequest( + resourceName, amount, discoveryScript, vendor) + else: + self._java_exec_request = None + self._name = resourceName + self._amount = amount + self._discoveryScript = discoveryScript + self._vendor = vendor @property def resourceName(self): - return self._java_exec_request.resourceName() + if self._java_exec_request is not None: + return self._java_exec_request.resourceName() + else: + return self._name @property def amount(self): - return self._java_exec_request.amount() + if self._java_exec_request is not None: + return self._java_exec_request.amount() + else: + return self._amount @property def discoveryScript(self): - return self._java_exec_request.discoveryScript() + if self._java_exec_request is not None: + return self._java_exec_request.discoveryScript() + else: + return self._discoveryScript @property def vendor(self): - return self._java_exec_request.vendor() + if self._java_exec_request is not None: + return self._java_exec_request.vendor() + else: + return self._vendor diff --git a/python/pyspark/resource/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py index 0cc1776e50581..e4dc6df47910e 100644 --- a/python/pyspark/resource/executorresourcerequests.py +++ b/python/pyspark/resource/executorresourcerequests.py @@ -30,38 +30,66 @@ class ExecutorResourceRequests(object): .. versionadded:: 3.1.0 """ - def __init__(self): + def __init__(self, _jvm = None, _requests = None): """ Create a new :class:`pyspark.resource.ExecutorResourceRequests` that wraps the underlying JVM object. """ from pyspark import SparkContext - self._java_executor_resource_requests = \ - SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequests() + _jvm = _jvm or SparkContext._jvm + if _jvm is not None: + self._java_executor_resource_requests = \ + _jvm.org.apache.spark.resource.ExecutorResourceRequests() + if _requests is not None: + self._java_executor_resource_requests.memory(_requests._memory) + self._java_executor_resource_requests.cores(_requests._cores) + else: + self._java_executor_resource_requests = None + self._custom_resources = [] def memory(self, amount): - self._java_executor_resource_requests.memory(amount) + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.memory(amount) + else: + self._memory = amount return self + def memoryOverhead(self, amount): - self._java_executor_resource_requests.memoryOverhead(amount) + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.memoryOverhead(amount) + else: + self._overhead_memory = amount return self def pysparkMemory(self, amount): - self._java_executor_resource_requests.pysparkMemory(amount) + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.pysparkMemory(amount) + else: + self._pyspark_memory = amount return self def cores(self, amount): - self._java_executor_resource_requests.cores(amount) + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.cores(amount) + else: + self._cores = amount return self def resource(self, resourceName, amount, discoveryScript="", vendor=""): - self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, vendor) + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, vendor) + else: + self._custom_resources.append(ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor)) return self @property def requests(self): - execRes = self._java_executor_resource_requests.requestsJMap() + if self._java_executor_resource_requests is not None: + execRes = self._java_executor_resource_requests.requestsJMap() + else: + # TODO + execRes = self._custom_resources result = {} # convert back to python ExecutorResourceRequest for k, v in execRes.items(): diff --git a/python/pyspark/resource/resourceprofile.py b/python/pyspark/resource/resourceprofile.py index 91cd79aa72302..e5fcae2a6c237 100644 --- a/python/pyspark/resource/resourceprofile.py +++ b/python/pyspark/resource/resourceprofile.py @@ -15,8 +15,8 @@ # limitations under the License. # -from pyspark.resource.executorresourcerequest import ExecutorResourceRequest from pyspark.resource.taskresourcerequest import TaskResourceRequest +from pyspark.resource.taskresourcerequests import TaskResourceRequests class ResourceProfile(object): @@ -24,20 +24,32 @@ class ResourceProfile(object): """ .. note:: Evolving - Resource profile to associate with an RDD. A ResourceProfile allows the user to - specify executor and task requirements for an RDD that will get applied during a - stage. This allows the user to change the resource requirements between stages. - This is meant to be immutable so user doesn't change it after building. + Resource profile to associate with an RDD. A :class:`pyspark.resource.ResourceProfile` + allows the user to specify executor and task requirements for an RDD that will get + applied during a stage. This allows the user to change the resource requirements between + stages. This is meant to be immutable so user doesn't change it after building. .. versionadded:: 3.1.0 """ - def __init__(self, _java_resource_profile): - self._java_resource_profile = _java_resource_profile + def __init__(self, _java_resource_profile = None, _exec_req = None, _task_req = None): + if _java_resource_profile is not None: + self._java_resource_profile = _java_resource_profile + else: + self._java_resource_profile = None + self._executor_resource_requests = _exec_req + self._task_resource_requests = _task_req + + @property + def id(self): + return self._java_resource_profile.id() @property def taskResources(self): - taskRes = self._java_resource_profile.taskResourcesJMap() + if _java_resource_profile is not None: + taskRes = self._java_resource_profile.taskResourcesJMap() + else: + taskRes = self._task_resource_requests result = {} # convert back to python TaskResourceRequest for k, v in taskRes.items(): diff --git a/python/pyspark/resource/resourceprofilebuilder.py b/python/pyspark/resource/resourceprofilebuilder.py index 5e861282490d9..ecef59fbc6dc4 100644 --- a/python/pyspark/resource/resourceprofilebuilder.py +++ b/python/pyspark/resource/resourceprofilebuilder.py @@ -34,28 +34,53 @@ class ResourceProfileBuilder(object): .. versionadded:: 3.1.0 """ - def __init__(self, ): - """Create a new ResourceProfileBuilder that wraps the underlying JVM object.""" + def __init__(self): + """ + Create a new :class:`pyspark.resource.ResourceProfileBuilder` that wraps the + underlying JVM object. + """ from pyspark.context import SparkContext - self._java_resource_profile_builder = \ - SparkContext._jvm.org.apache.spark.resource.ResourceProfileBuilder() + _jvm = SparkContext._jvm + if _jvm is not None: + self._java_resource_profile_builder = \ + _jvm.org.apache.spark.resource.ResourceProfileBuilder() + else: + self._java_resource_profile_builder = None + self._executor_resource_requests = None + self._task_resource_requests = None def require(self, resourceRequest): if isinstance(resourceRequest, TaskResourceRequests): - self._java_resource_profile_builder.require(resourceRequest._java_task_resource_requests) + if self._java_resource_profile_builder is not None: + self._java_resource_profile_builder.require(resourceRequest._java_task_resource_requests) + else: + self._task_resource_requests = resourceRequest else: - self._java_resource_profile_builder.require(resourceRequest._java_executor_resource_requests) + if self._java_resource_profile_builder is not None: + self._java_resource_profile_builder.require(resourceRequest._java_executor_resource_requests) + else: + self._executor_resource_requests = resourceRequest return self def clearExecutorResourceRequests(self): - self._java_resource_profile_builder.clearExecutorResourceRequests() + if self._java_resource_profile_builder is not None: + self._java_resource_profile_builder.clearExecutorResourceRequests() + else: + self._executor_resource_requests = None def clearTaskResourceRequests(self): - self._java_resource_profile_builder.clearTaskResourceRequests() + if self._java_resource_profile_builder is not None: + self._java_resource_profile_builder.clearTaskResourceRequests() + else: + self._task_resource_requests = None @property def taskResources(self): - taskRes = self._java_resource_profile_builder.taskResourcesJMap() + if self._java_resource_profile_builder is not None: + taskRes = self._java_resource_profile_builder.taskResourcesJMap() + else: + taskRes = self._task_resource_requests + result = {} # convert back to python TaskResourceRequest for k, v in taskRes.items(): @@ -64,7 +89,10 @@ def taskResources(self): @property def executorResources(self): - execRes = self._java_resource_profile_builder.executorResourcesJMap() + if self._java_resource_profile_builder is not None: + execRes = self._java_resource_profile_builder.executorResourcesJMap() + else: + execRes = self._executor_resource_requests result = {} # convert back to python ExecutorResourceRequest for k, v in execRes.items(): @@ -74,5 +102,10 @@ def executorResources(self): @property def build(self): - jresourceProfile = self._java_resource_profile_builder.build() - return ResourceProfile(jresourceProfile) + if self._java_resource_profile_builder is not None: + jresourceProfile = self._java_resource_profile_builder.build() + return ResourceProfile(_java_resource_profile = jresourceProfile) + else: + return ResourceProfile(_exec_req = self._executor_resource_requests, + _task_req = self._task_resource_requests) + diff --git a/python/pyspark/resource/taskresourcerequest.py b/python/pyspark/resource/taskresourcerequest.py index 5d79ea35284b1..92a1d426fa0d9 100644 --- a/python/pyspark/resource/taskresourcerequest.py +++ b/python/pyspark/resource/taskresourcerequest.py @@ -39,13 +39,25 @@ def __init__(self, resourceName, amount): JVM object. """ from pyspark.context import SparkContext - self._java_task_request = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( - resourceName, amount) + _jvm = SparkContext._jvm + if _jvm is not None: + self._java_task_request = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( + resourceName, float(amount)) + else: + self._java_task_request = None + self._name = resourceName + self._amount = amount @property def resourceName(self): - return self._java_task_request.resourceName() + if self._java_task_request is not None: + return self._java_task_request.resourceName() + else: + return self._name @property def amount(self): - return self._java_task_request.amount() + if self._java_task_request is not None: + return self._java_task_request.amount() + else: + return self._amount diff --git a/python/pyspark/resource/taskresourcerequests.py b/python/pyspark/resource/taskresourcerequests.py index 79c0c0490234f..333f45b890f07 100644 --- a/python/pyspark/resource/taskresourcerequests.py +++ b/python/pyspark/resource/taskresourcerequests.py @@ -30,28 +30,49 @@ class TaskResourceRequests(object): .. versionadded:: 3.1.0 """ - def __init__(self): + def __init__(self, _jvm = None, _requests = None): """ Create a new :class:`pyspark.resource.TaskResourceRequests` that wraps the underlying JVM object. """ from pyspark import SparkContext - self._java_task_resource_requests = \ - SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() + _jvm = _jvm or SparkContext._jvm + if _jvm is not None: + self._java_task_resource_requests = \ + SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() + if _requests is not None: + self._java_task_resource_requests.cpus(_requests._cpus) + else: + self._java_task_resource_requests = None + self._custom_resources = [] + self._cpus = None + def cpus(self, amount): - self._java_task_resource_requests.cpus(amount) + if self._java_task_resource_requests is not None: + self._java_task_resource_requests.cpus(amount) + else: + self._cpus=amount return self def resource(self, resourceName, amount): - self._java_task_resource_requests.resource(resourceName, float(amount)) + if self._java_task_resource_requests is not None: + self._java_task_resource_requests.resource(resourceName, float(amount)) + else: + self._custom_resources.append(TaskResourceRequest(resourceName, amount)) return self @property def requests(self): - taskRes = self._java_task_resource_requests.requestsJMap() result = {} - # convert back to python TaskResourceRequest - for k, v in taskRes.items(): - result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + if self._java_task_resource_requests is not None: + taskRes = self._java_task_resource_requests.requestsJMap() + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + else: + if self._cpus is not None: + result["cpus"] = TaskResourceRequest("cpus", self._cpus) + for t in self._custom_resources: + result[t.resourceName] = TaskResourceRequest(t.resourceName, t.amount) return result From a0b913764f071e62a9e145e10045e90d31fd9dc6 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 15 Apr 2020 16:50:56 -0500 Subject: [PATCH 24/31] More changes to call without SparkContext --- python/pyspark/rdd.py | 4 +- .../resource/executorresourcerequest.py | 39 ++++------------ .../resource/executorresourcerequests.py | 46 +++++++++++++------ python/pyspark/resource/resourceprofile.py | 37 ++++++++------- .../resource/resourceprofilebuilder.py | 29 +++++------- .../pyspark/resource/taskresourcerequest.py | 25 ++-------- .../pyspark/resource/taskresourcerequests.py | 10 ++-- 7 files changed, 84 insertions(+), 106 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d33fa3d31fbfa..c20153bf44ba9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2503,8 +2503,8 @@ def withResources(self, profile): jrp = profile._java_resource_profile else: builder = self.ctx._jvm.org.apache.spark.resource.ResourceProfileBuilder() - ereqs = ExecutorResourceRequests(self.ctx._jvm, profile._executor_resource_request) - treqs = TaskResourceRequests(self.ctx._jvm, profile._task_resource_request) + ereqs = ExecutorResourceRequests(self.ctx._jvm, profile._executor_resource_requests) + treqs = TaskResourceRequests(self.ctx._jvm, profile._task_resource_requests) builder.require(ereqs._java_executor_resource_requests) builder.require(treqs._java_task_resource_requests) jrp = builder.build() diff --git a/python/pyspark/resource/executorresourcerequest.py b/python/pyspark/resource/executorresourcerequest.py index 19fb210a39962..895edf0d08c59 100644 --- a/python/pyspark/resource/executorresourcerequest.py +++ b/python/pyspark/resource/executorresourcerequest.py @@ -53,46 +53,23 @@ class ExecutorResourceRequest(object): """ def __init__(self, resourceName, amount, discoveryScript="", vendor=""): - """ - Create a new :class:`pyspark.resource.ExecutorResourceRequest` that wraps the - underlying JVM object. - """ - from pyspark.context import SparkContext - _jvm = SparkContext._jvm - if _jvm is not None: - self._java_exec_request = _jvm.org.apache.spark.resource.ExecutorResourceRequest( - resourceName, amount, discoveryScript, vendor) - else: - self._java_exec_request = None - self._name = resourceName - self._amount = amount - self._discoveryScript = discoveryScript - self._vendor = vendor + self._name = resourceName + self._amount = amount + self._discoveryScript = discoveryScript + self._vendor = vendor @property def resourceName(self): - if self._java_exec_request is not None: - return self._java_exec_request.resourceName() - else: - return self._name + return self._name @property def amount(self): - if self._java_exec_request is not None: - return self._java_exec_request.amount() - else: - return self._amount + return self._amount @property def discoveryScript(self): - if self._java_exec_request is not None: - return self._java_exec_request.discoveryScript() - else: - return self._discoveryScript + return self._discoveryScript @property def vendor(self): - if self._java_exec_request is not None: - return self._java_exec_request.vendor() - else: - return self._vendor + return self._vendor diff --git a/python/pyspark/resource/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py index e4dc6df47910e..c50cfa9e3d4cd 100644 --- a/python/pyspark/resource/executorresourcerequests.py +++ b/python/pyspark/resource/executorresourcerequests.py @@ -16,6 +16,7 @@ # from pyspark.resource.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.taskresourcerequest import TaskResourceRequest class ExecutorResourceRequests(object): @@ -31,21 +32,30 @@ class ExecutorResourceRequests(object): """ def __init__(self, _jvm = None, _requests = None): - """ - Create a new :class:`pyspark.resource.ExecutorResourceRequests` that wraps the - underlying JVM object. - """ from pyspark import SparkContext _jvm = _jvm or SparkContext._jvm if _jvm is not None: self._java_executor_resource_requests = \ _jvm.org.apache.spark.resource.ExecutorResourceRequests() if _requests is not None: - self._java_executor_resource_requests.memory(_requests._memory) - self._java_executor_resource_requests.cores(_requests._cores) + if _requests._memory is not None: + self._java_executor_resource_requests.memory(_requests._memory) + if _requests._overhead_memory is not None: + self._java_executor_resource_requests.memoryOverhead(_requests._overhead_memory) + if _requests._pyspark_memory is not None: + self._java_executor_resource_requests.pysparkMemory(_requests._pyspark_memory) + if _requests._cores is not None: + self._java_executor_resource_requests.cores(_requests._cores) + for r in _requests._custom_resources: + self._java_executor_resource_requests.resource(r.resourceName, r.amount, + r.discoveryScript, r.vendor) else: self._java_executor_resource_requests = None self._custom_resources = [] + self._memory = None + self._overhead_memory = None + self._pyspark_memory = None + self._cores = None def memory(self, amount): if self._java_executor_resource_requests is not None: @@ -85,14 +95,24 @@ def resource(self, resourceName, amount, discoveryScript="", vendor=""): @property def requests(self): + result = {} if self._java_executor_resource_requests is not None: execRes = self._java_executor_resource_requests.requestsJMap() + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) else: - # TODO - execRes = self._custom_resources - result = {} - # convert back to python ExecutorResourceRequest - for k, v in execRes.items(): - result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) + if self._cores is not None: + result["cores"] = ExecutorResourceRequest("cores", self._cores) + if self._memory is not None: + result["memory"] = ExecutorResourceRequest("memory", self._memory) + if self._overhead_memory is not None: + result["memoryOverhead"] = ExecutorResourceRequest("memoryOverhead", + self._overhead_memory) + if self._pyspark_memory is not None: + result["pyspark.memory"] = ExecutorResourceRequest("pyspark.memory", + self._pyspark_memory) + for r in self._custom_resources: + result[r.resourceName] = ExecutorResourceRequest(r.resourceName, r.amount, + r.discoveryScript, r.vendor) return result diff --git a/python/pyspark/resource/resourceprofile.py b/python/pyspark/resource/resourceprofile.py index e5fcae2a6c237..a55859bb62ace 100644 --- a/python/pyspark/resource/resourceprofile.py +++ b/python/pyspark/resource/resourceprofile.py @@ -16,7 +16,7 @@ # from pyspark.resource.taskresourcerequest import TaskResourceRequest -from pyspark.resource.taskresourcerequests import TaskResourceRequests +from pyspark.resource.executorresourcerequest import ExecutorResourceRequest class ResourceProfile(object): @@ -42,26 +42,31 @@ def __init__(self, _java_resource_profile = None, _exec_req = None, _task_req = @property def id(self): - return self._java_resource_profile.id() + if self._java_resource_profile is not None: + return self._java_resource_profile.id() + else: + raise RuntimeError("SparkContext must be created to get the id, get the id " + "after adding the ResourceProfile to an RDD") @property def taskResources(self): - if _java_resource_profile is not None: + if self._java_resource_profile is not None: taskRes = self._java_resource_profile.taskResourcesJMap() + result = {} + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result else: - taskRes = self._task_resource_requests - result = {} - # convert back to python TaskResourceRequest - for k, v in taskRes.items(): - result[k] = TaskResourceRequest(v.resourceName(), v.amount()) - return result + return self._task_resource_requests @property def executorResources(self): - execRes = self._java_resource_profile.executorResourcesJMap() - result = {} - # convert back to python ExecutorResourceRequest - for k, v in execRes.items(): - result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) - return result + if self._java_resource_profile is not None: + execRes = self._java_resource_profile.executorResourcesJMap() + result = {} + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result + else: + return self._executor_resource_requests diff --git a/python/pyspark/resource/resourceprofilebuilder.py b/python/pyspark/resource/resourceprofilebuilder.py index ecef59fbc6dc4..a4d586e216c61 100644 --- a/python/pyspark/resource/resourceprofilebuilder.py +++ b/python/pyspark/resource/resourceprofilebuilder.py @@ -35,10 +35,6 @@ class ResourceProfileBuilder(object): """ def __init__(self): - """ - Create a new :class:`pyspark.resource.ResourceProfileBuilder` that wraps the - underlying JVM object. - """ from pyspark.context import SparkContext _jvm = SparkContext._jvm if _jvm is not None: @@ -78,27 +74,24 @@ def clearTaskResourceRequests(self): def taskResources(self): if self._java_resource_profile_builder is not None: taskRes = self._java_resource_profile_builder.taskResourcesJMap() + result = {} + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result else: - taskRes = self._task_resource_requests - - result = {} - # convert back to python TaskResourceRequest - for k, v in taskRes.items(): - result[k] = TaskResourceRequest(v.resourceName(), v.amount()) - return result + return self._task_resource_requests @property def executorResources(self): if self._java_resource_profile_builder is not None: + result = {} execRes = self._java_resource_profile_builder.executorResourcesJMap() + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result else: - execRes = self._executor_resource_requests - result = {} - # convert back to python ExecutorResourceRequest - for k, v in execRes.items(): - result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) - return result + return self._executor_resource_requests @property def build(self): diff --git a/python/pyspark/resource/taskresourcerequest.py b/python/pyspark/resource/taskresourcerequest.py index 92a1d426fa0d9..b67d08c035fde 100644 --- a/python/pyspark/resource/taskresourcerequest.py +++ b/python/pyspark/resource/taskresourcerequest.py @@ -34,30 +34,13 @@ class TaskResourceRequest(object): .. versionadded:: 3.1.0 """ def __init__(self, resourceName, amount): - """ - Create a new :class:`pyspark.resource.TaskResourceRequest` that wraps the underlying - JVM object. - """ - from pyspark.context import SparkContext - _jvm = SparkContext._jvm - if _jvm is not None: - self._java_task_request = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( - resourceName, float(amount)) - else: - self._java_task_request = None - self._name = resourceName - self._amount = amount + self._name = resourceName + self._amount = float(amount) @property def resourceName(self): - if self._java_task_request is not None: - return self._java_task_request.resourceName() - else: - return self._name + return self._name @property def amount(self): - if self._java_task_request is not None: - return self._java_task_request.amount() - else: - return self._amount + return self._amount diff --git a/python/pyspark/resource/taskresourcerequests.py b/python/pyspark/resource/taskresourcerequests.py index 333f45b890f07..03fd88c5ce122 100644 --- a/python/pyspark/resource/taskresourcerequests.py +++ b/python/pyspark/resource/taskresourcerequests.py @@ -31,17 +31,17 @@ class TaskResourceRequests(object): """ def __init__(self, _jvm = None, _requests = None): - """ - Create a new :class:`pyspark.resource.TaskResourceRequests` that wraps the underlying - JVM object. - """ from pyspark import SparkContext _jvm = _jvm or SparkContext._jvm if _jvm is not None: self._java_task_resource_requests = \ SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() if _requests is not None: - self._java_task_resource_requests.cpus(_requests._cpus) + if _requests._cpus is not None: + self._java_task_resource_requests.cpus(_requests._cpus) + for r in _requests._custom_resources: + self._java_executor_resource_requests.resource(r.resourceName, + r.amount, r.discoveryScript, r.vendor) else: self._java_task_resource_requests = None self._custom_resources = [] From 22356545715807d10d09aeb4c62d0daf600d21fe Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 15 Apr 2020 17:22:49 -0500 Subject: [PATCH 25/31] cleanup --- python/pyspark/resource/executorresourcerequests.py | 3 +-- python/pyspark/resource/taskresourcerequests.py | 4 +--- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/python/pyspark/resource/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py index c50cfa9e3d4cd..93fb28ff70f11 100644 --- a/python/pyspark/resource/executorresourcerequests.py +++ b/python/pyspark/resource/executorresourcerequests.py @@ -48,7 +48,7 @@ def __init__(self, _jvm = None, _requests = None): self._java_executor_resource_requests.cores(_requests._cores) for r in _requests._custom_resources: self._java_executor_resource_requests.resource(r.resourceName, r.amount, - r.discoveryScript, r.vendor) + r.discoveryScript, r.vendor) else: self._java_executor_resource_requests = None self._custom_resources = [] @@ -64,7 +64,6 @@ def memory(self, amount): self._memory = amount return self - def memoryOverhead(self, amount): if self._java_executor_resource_requests is not None: self._java_executor_resource_requests.memoryOverhead(amount) diff --git a/python/pyspark/resource/taskresourcerequests.py b/python/pyspark/resource/taskresourcerequests.py index 03fd88c5ce122..cac908b4f2fc5 100644 --- a/python/pyspark/resource/taskresourcerequests.py +++ b/python/pyspark/resource/taskresourcerequests.py @@ -40,8 +40,7 @@ def __init__(self, _jvm = None, _requests = None): if _requests._cpus is not None: self._java_task_resource_requests.cpus(_requests._cpus) for r in _requests._custom_resources: - self._java_executor_resource_requests.resource(r.resourceName, - r.amount, r.discoveryScript, r.vendor) + self._java_task_resource_requests.resource(r.resourceName, r.amount) else: self._java_task_resource_requests = None self._custom_resources = [] @@ -67,7 +66,6 @@ def requests(self): result = {} if self._java_task_resource_requests is not None: taskRes = self._java_task_resource_requests.requestsJMap() - # convert back to python TaskResourceRequest for k, v in taskRes.items(): result[k] = TaskResourceRequest(v.resourceName(), v.amount()) else: From bf1a215ed99018f856965077d4d9983d61b78524 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 15 Apr 2020 17:54:03 -0500 Subject: [PATCH 26/31] fix style issues --- python/pyspark/rdd.py | 2 +- .../resource/executorresourcerequests.py | 20 ++++++++++--------- python/pyspark/resource/resourceprofile.py | 4 ++-- .../resource/resourceprofilebuilder.py | 15 +++++++------- .../pyspark/resource/taskresourcerequests.py | 9 ++++----- 5 files changed, 26 insertions(+), 24 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c20153bf44ba9..30c9adbe21205 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2522,7 +2522,7 @@ def getResourceProfile(self): .. versionadded:: 3.1.0 """ - return ResourceProfile(_java_resource_profile = self._jrdd.getResourceProfile()) + return ResourceProfile(_java_resource_profile=self._jrdd.getResourceProfile()) def _prepare_for_python_RDD(sc, command): diff --git a/python/pyspark/resource/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py index 93fb28ff70f11..3ae8b16422ae1 100644 --- a/python/pyspark/resource/executorresourcerequests.py +++ b/python/pyspark/resource/executorresourcerequests.py @@ -31,7 +31,7 @@ class ExecutorResourceRequests(object): .. versionadded:: 3.1.0 """ - def __init__(self, _jvm = None, _requests = None): + def __init__(self, _jvm=None, _requests=None): from pyspark import SparkContext _jvm = _jvm or SparkContext._jvm if _jvm is not None: @@ -48,7 +48,7 @@ def __init__(self, _jvm = None, _requests = None): self._java_executor_resource_requests.cores(_requests._cores) for r in _requests._custom_resources: self._java_executor_resource_requests.resource(r.resourceName, r.amount, - r.discoveryScript, r.vendor) + r.discoveryScript, r.vendor) else: self._java_executor_resource_requests = None self._custom_resources = [] @@ -63,7 +63,7 @@ def memory(self, amount): else: self._memory = amount return self - + def memoryOverhead(self, amount): if self._java_executor_resource_requests is not None: self._java_executor_resource_requests.memoryOverhead(amount) @@ -87,9 +87,11 @@ def cores(self, amount): def resource(self, resourceName, amount, discoveryScript="", vendor=""): if self._java_executor_resource_requests is not None: - self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, vendor) + self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, + vendor) else: - self._custom_resources.append(ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor)) + self._custom_resources.append(ExecutorResourceRequest(resourceName, amount, + discoveryScript, vendor)) return self @property @@ -99,7 +101,7 @@ def requests(self): execRes = self._java_executor_resource_requests.requestsJMap() for k, v in execRes.items(): result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) + v.discoveryScript(), v.vendor()) else: if self._cores is not None: result["cores"] = ExecutorResourceRequest("cores", self._cores) @@ -107,11 +109,11 @@ def requests(self): result["memory"] = ExecutorResourceRequest("memory", self._memory) if self._overhead_memory is not None: result["memoryOverhead"] = ExecutorResourceRequest("memoryOverhead", - self._overhead_memory) + self._overhead_memory) if self._pyspark_memory is not None: result["pyspark.memory"] = ExecutorResourceRequest("pyspark.memory", - self._pyspark_memory) + self._pyspark_memory) for r in self._custom_resources: result[r.resourceName] = ExecutorResourceRequest(r.resourceName, r.amount, - r.discoveryScript, r.vendor) + r.discoveryScript, r.vendor) return result diff --git a/python/pyspark/resource/resourceprofile.py b/python/pyspark/resource/resourceprofile.py index a55859bb62ace..bd34554e45056 100644 --- a/python/pyspark/resource/resourceprofile.py +++ b/python/pyspark/resource/resourceprofile.py @@ -32,7 +32,7 @@ class ResourceProfile(object): .. versionadded:: 3.1.0 """ - def __init__(self, _java_resource_profile = None, _exec_req = None, _task_req = None): + def __init__(self, _java_resource_profile=None, _exec_req=None, _task_req=None): if _java_resource_profile is not None: self._java_resource_profile = _java_resource_profile else: @@ -66,7 +66,7 @@ def executorResources(self): result = {} for k, v in execRes.items(): result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) + v.discoveryScript(), v.vendor()) return result else: return self._executor_resource_requests diff --git a/python/pyspark/resource/resourceprofilebuilder.py b/python/pyspark/resource/resourceprofilebuilder.py index a4d586e216c61..460e0d4e16efa 100644 --- a/python/pyspark/resource/resourceprofilebuilder.py +++ b/python/pyspark/resource/resourceprofilebuilder.py @@ -48,12 +48,14 @@ def __init__(self): def require(self, resourceRequest): if isinstance(resourceRequest, TaskResourceRequests): if self._java_resource_profile_builder is not None: - self._java_resource_profile_builder.require(resourceRequest._java_task_resource_requests) + self._java_resource_profile_builder.require( + resourceRequest._java_task_resource_requests) else: self._task_resource_requests = resourceRequest else: if self._java_resource_profile_builder is not None: - self._java_resource_profile_builder.require(resourceRequest._java_executor_resource_requests) + self._java_resource_profile_builder.require( + resourceRequest._java_executor_resource_requests) else: self._executor_resource_requests = resourceRequest return self @@ -88,7 +90,7 @@ def executorResources(self): execRes = self._java_resource_profile_builder.executorResourcesJMap() for k, v in execRes.items(): result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) + v.discoveryScript(), v.vendor()) return result else: return self._executor_resource_requests @@ -97,8 +99,7 @@ def executorResources(self): def build(self): if self._java_resource_profile_builder is not None: jresourceProfile = self._java_resource_profile_builder.build() - return ResourceProfile(_java_resource_profile = jresourceProfile) + return ResourceProfile(_java_resource_profile=jresourceProfile) else: - return ResourceProfile(_exec_req = self._executor_resource_requests, - _task_req = self._task_resource_requests) - + return ResourceProfile(_exec_req=self._executor_resource_requests, + _task_req=self._task_resource_requests) diff --git a/python/pyspark/resource/taskresourcerequests.py b/python/pyspark/resource/taskresourcerequests.py index cac908b4f2fc5..2e25e169dc224 100644 --- a/python/pyspark/resource/taskresourcerequests.py +++ b/python/pyspark/resource/taskresourcerequests.py @@ -24,13 +24,13 @@ class TaskResourceRequests(object): .. note:: Evolving A set of task resource requests. This is used in conjuntion with the - :class:`pyspark.resource.ResourceProfileBuilder` to programmatically specify the resources needed for - an RDD that will be applied at the stage level. + :class:`pyspark.resource.ResourceProfileBuilder` to programmatically specify the resources + needed for an RDD that will be applied at the stage level. .. versionadded:: 3.1.0 """ - def __init__(self, _jvm = None, _requests = None): + def __init__(self, _jvm=None, _requests=None): from pyspark import SparkContext _jvm = _jvm or SparkContext._jvm if _jvm is not None: @@ -46,12 +46,11 @@ def __init__(self, _jvm = None, _requests = None): self._custom_resources = [] self._cpus = None - def cpus(self, amount): if self._java_task_resource_requests is not None: self._java_task_resource_requests.cpus(amount) else: - self._cpus=amount + self._cpus = amount return self def resource(self, resourceName, amount): From 62cb02c795dd9b73618d6dff57f50fb75c66838b Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 15 Apr 2020 18:25:47 -0500 Subject: [PATCH 27/31] Change getResourceProfile to return None to match scala side --- python/pyspark/rdd.py | 8 ++++++-- python/pyspark/tests/test_rdd.py | 2 ++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 30c9adbe21205..67e0b20bba35e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2518,11 +2518,15 @@ def getResourceProfile(self): Get the :class:`pyspark.resource.ResourceProfile` specified with this RDD or None if it wasn't specified. - :return: the user specified ResourceProfile or null if none was specified + :return: the user specified ResourceProfile or None if none were specified .. versionadded:: 3.1.0 """ - return ResourceProfile(_java_resource_profile=self._jrdd.getResourceProfile()) + rp = self._jrdd.getResourceProfile() + if rp is not None: + return ResourceProfile(_java_resource_profile=rp) + else: + return None def _prepare_for_python_RDD(sc, command): diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index 3c12f8ff4c8a6..31c5a7510a165 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -811,6 +811,8 @@ def assert_request_contents(exec_reqs, task_reqs): rdd = self.sc.parallelize(range(10)).withResources(rp) return_rp = rdd.getResourceProfile() assert_request_contents(return_rp.executorResources, return_rp.taskResources) + rddWithoutRp = self.sc.parallelize(range(10)) + self.assertEqual(rddWithoutRp.getResourceProfile(), None) if __name__ == "__main__": import unittest From a6e9ac23a9f75661225fa1ca47414a416c157153 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 16 Apr 2020 13:55:31 -0500 Subject: [PATCH 28/31] Change to make python versions do same thing as the scala versions as hit some issues with switching between objects created before SparkContext and ones after. Easier to understand this way. Add tests --- python/pyspark/rdd.py | 22 +-- python/pyspark/resource/__init__.py | 6 +- python/pyspark/resource/executorrequests.py | 169 ++++++++++++++++++ .../resource/executorresourcerequest.py | 75 -------- .../resource/executorresourcerequests.py | 119 ------------ python/pyspark/resource/resourceprofile.py | 6 +- .../resource/resourceprofilebuilder.py | 38 ++-- ...askresourcerequests.py => taskrequests.py} | 57 ++++-- .../pyspark/resource/taskresourcerequest.py | 46 ----- python/pyspark/resource/tests/__init__.py | 16 ++ .../pyspark/resource/tests/test_resources.py | 82 +++++++++ python/pyspark/util.py | 15 ++ 12 files changed, 357 insertions(+), 294 deletions(-) create mode 100644 python/pyspark/resource/executorrequests.py delete mode 100644 python/pyspark/resource/executorresourcerequest.py delete mode 100644 python/pyspark/resource/executorresourcerequests.py rename python/pyspark/resource/{taskresourcerequests.py => taskrequests.py} (57%) delete mode 100644 python/pyspark/resource/taskresourcerequest.py create mode 100644 python/pyspark/resource/tests/__init__.py create mode 100644 python/pyspark/resource/tests/test_resources.py diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 67e0b20bba35e..2a03f7e08a600 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -47,14 +47,14 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel -from pyspark.resource.executorresourcerequests import ExecutorResourceRequests +from pyspark.resource.executorrequests import ExecutorResourceRequests from pyspark.resource.resourceprofile import ResourceProfile -from pyspark.resource.taskresourcerequests import TaskResourceRequests +from pyspark.resource.taskrequests import TaskResourceRequests from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy from pyspark.traceback_utils import SCCallSiteSync -from pyspark.util import fail_on_stopiteration +from pyspark.util import fail_on_stopiteration, _parse_memory __all__ = ["RDD"] @@ -128,22 +128,6 @@ def __new__(cls, mean, confidence, low, high): return obj -def _parse_memory(s): - """ - Parse a memory string in the format supported by Java (e.g. 1g, 200m) and - return the value in MiB - - >>> _parse_memory("256m") - 256 - >>> _parse_memory("2g") - 2048 - """ - units = {'g': 1024, 'm': 1, 't': 1 << 20, 'k': 1.0 / 1024} - if s[-1].lower() not in units: - raise ValueError("invalid format: " + s) - return int(float(s[:-1]) * units[s[-1].lower()]) - - def _create_local_socket(sock_info): """ Create a local socket that can be used to load deserialized data from the JVM diff --git a/python/pyspark/resource/__init__.py b/python/pyspark/resource/__init__.py index cddc10874cd1a..89070ec4adc7e 100644 --- a/python/pyspark/resource/__init__.py +++ b/python/pyspark/resource/__init__.py @@ -18,10 +18,8 @@ """ APIs to let users manipulate resource requirements. """ -from pyspark.resource.taskresourcerequest import TaskResourceRequest -from pyspark.resource.executorresourcerequest import ExecutorResourceRequest -from pyspark.resource.taskresourcerequests import TaskResourceRequests -from pyspark.resource.executorresourcerequests import ExecutorResourceRequests +from pyspark.resource.executorrequests import ExecutorResourceRequest, ExecutorResourceRequests +from pyspark.resource.taskrequests import TaskResourceRequest, TaskResourceRequests from pyspark.resource.resourceprofilebuilder import ResourceProfileBuilder from pyspark.resource.resourceprofile import ResourceProfile diff --git a/python/pyspark/resource/executorrequests.py b/python/pyspark/resource/executorrequests.py new file mode 100644 index 0000000000000..36e747cf63b14 --- /dev/null +++ b/python/pyspark/resource/executorrequests.py @@ -0,0 +1,169 @@ +# +# 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. +# + +from pyspark.resource.taskrequests import TaskResourceRequest +from pyspark.util import _parse_memory + + +class ExecutorResourceRequest(object): + """ + .. note:: Evolving + + An Executor resource request. This is used in conjunction with the ResourceProfile to + programmatically specify the resources needed for an RDD that will be applied at the + stage level. + + This is used to specify what the resource requirements are for an Executor and how + Spark can find out specific details about those resources. Not all the parameters are + required for every resource type. Resources like GPUs are supported and have same limitations + as using the global spark configs spark.executor.resource.gpu.*. The amount, discoveryScript, + and vendor parameters for resources are all the same parameters a user would specify through the + configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. + + For instance, a user wants to allocate an Executor with GPU resources on YARN. The user has + to specify the resource name (gpu), the amount or number of GPUs per Executor, + the discovery script would be specified so that when the Executor starts up it can + discovery what GPU addresses are available for it to use because YARN doesn't tell + Spark that, then vendor would not be used because its specific for Kubernetes. + + See the configuration and cluster specific docs for more details. + + Use `pyspark.ExecutorResourceRequests` class as a convenience API. + + :param resourceName: Name of the resource + :param amount: Amount requesting + :param discoveryScript: Optional script used to discover the resources. This is required on some + cluster managers that don't tell Spark the addresses of the resources + allocated. The script runs on Executors startup to discover the addresses + of the resources available. + :param vendor: Vendor, required for some cluster managers + + .. versionadded:: 3.1.0 + """ + def __init__(self, resourceName, amount, discoveryScript="", vendor=""): + self._name = resourceName + self._amount = amount + self._discoveryScript = discoveryScript + self._vendor = vendor + + @property + def resourceName(self): + return self._name + + @property + def amount(self): + return self._amount + + @property + def discoveryScript(self): + return self._discoveryScript + + @property + def vendor(self): + return self._vendor + + +class ExecutorResourceRequests(object): + + """ + .. note:: Evolving + + A set of Executor resource requests. This is used in conjunction with the + ResourceProfileBuilder to programmatically specify the resources needed for an RDD + that will be applied at the stage level. + + .. versionadded:: 3.1.0 + """ + _CORES = "cores" + _MEMORY = "memory" + _OVERHEAD_MEM = "memoryOverhead" + _PYSPARK_MEM = "pyspark.memory" + + def __init__(self, _jvm=None, _requests=None): + from pyspark import SparkContext + _jvm = _jvm or SparkContext._jvm + if _jvm is not None: + self._java_executor_resource_requests = \ + _jvm.org.apache.spark.resource.ExecutorResourceRequests() + if _requests is not None: + for k, v in _requests.items(): + if k == self._MEMORY: + self._java_executor_resource_requests.memory(str(v.amount)) + elif k == self._OVERHEAD_MEM: + self._java_executor_resource_requests.memoryOverhead(str(v.amount)) + elif k == self._PYSPARK_MEM: + self._java_executor_resource_requests.pysparkMemory(str(v.amount)) + elif k == self._CORES: + self._java_executor_resource_requests.cores(v.amount) + else: + self._java_executor_resource_requests.resource(v.resourceName, v.amount, + v.discoveryScript, v.vendor) + else: + self._java_executor_resource_requests = None + self._executor_resources = {} + + def memory(self, amount): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.memory(amount) + else: + self._executor_resources[self._MEMORY] = ExecutorResourceRequest(self._MEMORY, + _parse_memory(amount)) + return self + + def memoryOverhead(self, amount): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.memoryOverhead(amount) + else: + self._executor_resources[self._OVERHEAD_MEM] = \ + ExecutorResourceRequest(self._OVERHEAD_MEM, _parse_memory(amount)) + return self + + def pysparkMemory(self, amount): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.pysparkMemory(amount) + else: + self._executor_resources[self._PYSPARK_MEM] = \ + ExecutorResourceRequest(self._PYSPARK_MEM, _parse_memory(amount)) + return self + + def cores(self, amount): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.cores(amount) + else: + self._executor_resources[self._CORES] = ExecutorResourceRequest(self._CORES, amount) + return self + + def resource(self, resourceName, amount, discoveryScript="", vendor=""): + if self._java_executor_resource_requests is not None: + self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, + vendor) + else: + self._executor_resources[resourceName] = \ + ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor) + return self + + @property + def requests(self): + if self._java_executor_resource_requests is not None: + result = {} + execRes = self._java_executor_resource_requests.requestsJMap() + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result + else: + return self._executor_resources diff --git a/python/pyspark/resource/executorresourcerequest.py b/python/pyspark/resource/executorresourcerequest.py deleted file mode 100644 index 895edf0d08c59..0000000000000 --- a/python/pyspark/resource/executorresourcerequest.py +++ /dev/null @@ -1,75 +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. -# - - -class ExecutorResourceRequest(object): - """ - .. note:: Evolving - - An Executor resource request. This is used in conjunction with the ResourceProfile to - programmatically specify the resources needed for an RDD that will be applied at the - stage level. - - This is used to specify what the resource requirements are for an Executor and how - Spark can find out specific details about those resources. Not all the parameters are - required for every resource type. Resources like GPUs are supported and have same limitations - as using the global spark configs spark.executor.resource.gpu.*. The amount, discoveryScript, - and vendor parameters for resources are all the same parameters a user would specify through the - configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. - - For instance, a user wants to allocate an Executor with GPU resources on YARN. The user has - to specify the resource name (gpu), the amount or number of GPUs per Executor, - the discovery script would be specified so that when the Executor starts up it can - discovery what GPU addresses are available for it to use because YARN doesn't tell - Spark that, then vendor would not be used because its specific for Kubernetes. - - See the configuration and cluster specific docs for more details. - - Use `pyspark.ExecutorResourceRequests` class as a convenience API. - - :param resourceName: Name of the resource - :param amount: Amount requesting - :param discoveryScript: Optional script used to discover the resources. This is required on some - cluster managers that don't tell Spark the addresses of the resources - allocated. The script runs on Executors startup to discover the addresses - of the resources available. - :param vendor: Vendor, required for some cluster managers - - .. versionadded:: 3.1.0 - """ - - def __init__(self, resourceName, amount, discoveryScript="", vendor=""): - self._name = resourceName - self._amount = amount - self._discoveryScript = discoveryScript - self._vendor = vendor - - @property - def resourceName(self): - return self._name - - @property - def amount(self): - return self._amount - - @property - def discoveryScript(self): - return self._discoveryScript - - @property - def vendor(self): - return self._vendor diff --git a/python/pyspark/resource/executorresourcerequests.py b/python/pyspark/resource/executorresourcerequests.py deleted file mode 100644 index 3ae8b16422ae1..0000000000000 --- a/python/pyspark/resource/executorresourcerequests.py +++ /dev/null @@ -1,119 +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. -# - -from pyspark.resource.executorresourcerequest import ExecutorResourceRequest -from pyspark.resource.taskresourcerequest import TaskResourceRequest - - -class ExecutorResourceRequests(object): - - """ - .. note:: Evolving - - A set of Executor resource requests. This is used in conjunction with the - ResourceProfileBuilder to programmatically specify the resources needed for an RDD - that will be applied at the stage level. - - .. versionadded:: 3.1.0 - """ - - def __init__(self, _jvm=None, _requests=None): - from pyspark import SparkContext - _jvm = _jvm or SparkContext._jvm - if _jvm is not None: - self._java_executor_resource_requests = \ - _jvm.org.apache.spark.resource.ExecutorResourceRequests() - if _requests is not None: - if _requests._memory is not None: - self._java_executor_resource_requests.memory(_requests._memory) - if _requests._overhead_memory is not None: - self._java_executor_resource_requests.memoryOverhead(_requests._overhead_memory) - if _requests._pyspark_memory is not None: - self._java_executor_resource_requests.pysparkMemory(_requests._pyspark_memory) - if _requests._cores is not None: - self._java_executor_resource_requests.cores(_requests._cores) - for r in _requests._custom_resources: - self._java_executor_resource_requests.resource(r.resourceName, r.amount, - r.discoveryScript, r.vendor) - else: - self._java_executor_resource_requests = None - self._custom_resources = [] - self._memory = None - self._overhead_memory = None - self._pyspark_memory = None - self._cores = None - - def memory(self, amount): - if self._java_executor_resource_requests is not None: - self._java_executor_resource_requests.memory(amount) - else: - self._memory = amount - return self - - def memoryOverhead(self, amount): - if self._java_executor_resource_requests is not None: - self._java_executor_resource_requests.memoryOverhead(amount) - else: - self._overhead_memory = amount - return self - - def pysparkMemory(self, amount): - if self._java_executor_resource_requests is not None: - self._java_executor_resource_requests.pysparkMemory(amount) - else: - self._pyspark_memory = amount - return self - - def cores(self, amount): - if self._java_executor_resource_requests is not None: - self._java_executor_resource_requests.cores(amount) - else: - self._cores = amount - return self - - def resource(self, resourceName, amount, discoveryScript="", vendor=""): - if self._java_executor_resource_requests is not None: - self._java_executor_resource_requests.resource(resourceName, amount, discoveryScript, - vendor) - else: - self._custom_resources.append(ExecutorResourceRequest(resourceName, amount, - discoveryScript, vendor)) - return self - - @property - def requests(self): - result = {} - if self._java_executor_resource_requests is not None: - execRes = self._java_executor_resource_requests.requestsJMap() - for k, v in execRes.items(): - result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), - v.discoveryScript(), v.vendor()) - else: - if self._cores is not None: - result["cores"] = ExecutorResourceRequest("cores", self._cores) - if self._memory is not None: - result["memory"] = ExecutorResourceRequest("memory", self._memory) - if self._overhead_memory is not None: - result["memoryOverhead"] = ExecutorResourceRequest("memoryOverhead", - self._overhead_memory) - if self._pyspark_memory is not None: - result["pyspark.memory"] = ExecutorResourceRequest("pyspark.memory", - self._pyspark_memory) - for r in self._custom_resources: - result[r.resourceName] = ExecutorResourceRequest(r.resourceName, r.amount, - r.discoveryScript, r.vendor) - return result diff --git a/python/pyspark/resource/resourceprofile.py b/python/pyspark/resource/resourceprofile.py index bd34554e45056..59e9ccb4b6ea0 100644 --- a/python/pyspark/resource/resourceprofile.py +++ b/python/pyspark/resource/resourceprofile.py @@ -15,8 +15,8 @@ # limitations under the License. # -from pyspark.resource.taskresourcerequest import TaskResourceRequest -from pyspark.resource.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.taskrequests import TaskResourceRequest +from pyspark.resource.executorrequests import ExecutorResourceRequest class ResourceProfile(object): @@ -32,7 +32,7 @@ class ResourceProfile(object): .. versionadded:: 3.1.0 """ - def __init__(self, _java_resource_profile=None, _exec_req=None, _task_req=None): + def __init__(self, _java_resource_profile=None, _exec_req={}, _task_req={}): if _java_resource_profile is not None: self._java_resource_profile = _java_resource_profile else: diff --git a/python/pyspark/resource/resourceprofilebuilder.py b/python/pyspark/resource/resourceprofilebuilder.py index 460e0d4e16efa..67654289d500f 100644 --- a/python/pyspark/resource/resourceprofilebuilder.py +++ b/python/pyspark/resource/resourceprofilebuilder.py @@ -15,10 +15,10 @@ # limitations under the License. # -from pyspark.resource.executorresourcerequest import ExecutorResourceRequest +from pyspark.resource.executorrequests import ExecutorResourceRequest,\ + ExecutorResourceRequests from pyspark.resource.resourceprofile import ResourceProfile -from pyspark.resource.taskresourcerequest import TaskResourceRequest -from pyspark.resource.taskresourcerequests import TaskResourceRequests +from pyspark.resource.taskrequests import TaskResourceRequest, TaskResourceRequests class ResourceProfileBuilder(object): @@ -38,39 +38,51 @@ def __init__(self): from pyspark.context import SparkContext _jvm = SparkContext._jvm if _jvm is not None: + self._jvm = _jvm self._java_resource_profile_builder = \ _jvm.org.apache.spark.resource.ResourceProfileBuilder() else: + self._jvm = None self._java_resource_profile_builder = None - self._executor_resource_requests = None - self._task_resource_requests = None + self._executor_resource_requests = {} + self._task_resource_requests = {} def require(self, resourceRequest): if isinstance(resourceRequest, TaskResourceRequests): if self._java_resource_profile_builder is not None: - self._java_resource_profile_builder.require( - resourceRequest._java_task_resource_requests) + if resourceRequest._java_task_resource_requests is not None: + self._java_resource_profile_builder.require( + resourceRequest._java_task_resource_requests) + else: + taskReqs = TaskResourceRequests(self._jvm, resourceRequest.requests) + self._java_resource_profile_builder.require( + taskReqs._java_task_resource_requests) else: - self._task_resource_requests = resourceRequest + self._task_resource_requests.update(resourceRequest.requests) else: if self._java_resource_profile_builder is not None: - self._java_resource_profile_builder.require( - resourceRequest._java_executor_resource_requests) + if resourceRequest._java_executor_resource_requests is not None: + self._java_resource_profile_builder.require( + resourceRequest._java_executor_resource_requests) + else: + execReqs = ExecutorResourceRequests(self._jvm, resourceRequest.requests) + self._java_resource_profile_builder.require( + execReqs._java_executor_resource_requests) else: - self._executor_resource_requests = resourceRequest + self._executor_resource_requests.update(resourceRequest.requests) return self def clearExecutorResourceRequests(self): if self._java_resource_profile_builder is not None: self._java_resource_profile_builder.clearExecutorResourceRequests() else: - self._executor_resource_requests = None + self._executor_resource_requests = {} def clearTaskResourceRequests(self): if self._java_resource_profile_builder is not None: self._java_resource_profile_builder.clearTaskResourceRequests() else: - self._task_resource_requests = None + self._task_resource_requests = {} @property def taskResources(self): diff --git a/python/pyspark/resource/taskresourcerequests.py b/python/pyspark/resource/taskrequests.py similarity index 57% rename from python/pyspark/resource/taskresourcerequests.py rename to python/pyspark/resource/taskrequests.py index 2e25e169dc224..e8dca98d14b61 100644 --- a/python/pyspark/resource/taskresourcerequests.py +++ b/python/pyspark/resource/taskrequests.py @@ -15,7 +15,35 @@ # limitations under the License. # -from pyspark.resource.taskresourcerequest import TaskResourceRequest + +class TaskResourceRequest(object): + """ + .. note:: Evolving + + A task resource request. This is used in conjuntion with the + :class:`pyspark.resource.ResourceProfile` to programmatically specify the resources + needed for an RDD that will be applied at the stage level. The amount is specified + as a Double to allow for saying you want more then 1 task per resource. Valid values + are less than or equal to 0.5 or whole numbers. + Use :class:`pyspark.resource.TaskResourceRequests` class as a convenience API. + + :param resourceName: Name of the resource + :param amount: Amount requesting as a Double to support fractional resource requests. + Valid values are less than or equal to 0.5 or whole numbers. + + .. versionadded:: 3.1.0 + """ + def __init__(self, resourceName, amount): + self._name = resourceName + self._amount = float(amount) + + @property + def resourceName(self): + return self._name + + @property + def amount(self): + return self._amount class TaskResourceRequests(object): @@ -30,6 +58,8 @@ class TaskResourceRequests(object): .. versionadded:: 3.1.0 """ + _CPUS = "cpus" + def __init__(self, _jvm=None, _requests=None): from pyspark import SparkContext _jvm = _jvm or SparkContext._jvm @@ -37,39 +67,36 @@ def __init__(self, _jvm=None, _requests=None): self._java_task_resource_requests = \ SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() if _requests is not None: - if _requests._cpus is not None: - self._java_task_resource_requests.cpus(_requests._cpus) - for r in _requests._custom_resources: - self._java_task_resource_requests.resource(r.resourceName, r.amount) + for k, v in _requests.items(): + if k == self._CPUS: + self._java_task_resource_requests.cpus(int(v.amount)) + else: + self._java_task_resource_requests.resource(v.resourceName, v.amount) else: self._java_task_resource_requests = None - self._custom_resources = [] - self._cpus = None + self._task_resources = {} def cpus(self, amount): if self._java_task_resource_requests is not None: self._java_task_resource_requests.cpus(amount) else: - self._cpus = amount + self._task_resources[self._CPUS] = TaskResourceRequest(self._CPUS, amount) return self def resource(self, resourceName, amount): if self._java_task_resource_requests is not None: self._java_task_resource_requests.resource(resourceName, float(amount)) else: - self._custom_resources.append(TaskResourceRequest(resourceName, amount)) + self._task_resources[resourceName] = TaskResourceRequest(resourceName, amount) return self @property def requests(self): - result = {} if self._java_task_resource_requests is not None: + result = {} taskRes = self._java_task_resource_requests.requestsJMap() for k, v in taskRes.items(): result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result else: - if self._cpus is not None: - result["cpus"] = TaskResourceRequest("cpus", self._cpus) - for t in self._custom_resources: - result[t.resourceName] = TaskResourceRequest(t.resourceName, t.amount) - return result + return self._task_resources diff --git a/python/pyspark/resource/taskresourcerequest.py b/python/pyspark/resource/taskresourcerequest.py deleted file mode 100644 index b67d08c035fde..0000000000000 --- a/python/pyspark/resource/taskresourcerequest.py +++ /dev/null @@ -1,46 +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. -# - - -class TaskResourceRequest(object): - """ - .. note:: Evolving - - A task resource request. This is used in conjuntion with the - :class:`pyspark.resource.ResourceProfile` to programmatically specify the resources - needed for an RDD that will be applied at the stage level. The amount is specified - as a Double to allow for saying you want more then 1 task per resource. Valid values - are less than or equal to 0.5 or whole numbers. - Use :class:`pyspark.resource.TaskResourceRequests` class as a convenience API. - - :param resourceName: Name of the resource - :param amount: Amount requesting as a Double to support fractional resource requests. - Valid values are less than or equal to 0.5 or whole numbers. - - .. versionadded:: 3.1.0 - """ - def __init__(self, resourceName, amount): - self._name = resourceName - self._amount = float(amount) - - @property - def resourceName(self): - return self._name - - @property - def amount(self): - return self._amount diff --git a/python/pyspark/resource/tests/__init__.py b/python/pyspark/resource/tests/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/resource/tests/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/python/pyspark/resource/tests/test_resources.py b/python/pyspark/resource/tests/test_resources.py new file mode 100644 index 0000000000000..9eb5a35e58df5 --- /dev/null +++ b/python/pyspark/resource/tests/test_resources.py @@ -0,0 +1,82 @@ +# +# 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. +# +import random +import unittest + +from pyspark.resource import ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder,\ + TaskResourceRequests + + +class ResourceProfileTests(unittest.TestCase): + def test_profile_before_sc(self): + rpb = ResourceProfileBuilder() + ereqs = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g") + ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com") + treqs = TaskResourceRequests().cpus(2).resource("gpu", 2) + + def assert_request_contents(exec_reqs, task_reqs): + self.assertEqual(len(exec_reqs), 5) + self.assertEqual(exec_reqs["cores"].amount, 2) + self.assertEqual(exec_reqs["memory"].amount, 6144) + self.assertEqual(exec_reqs["memoryOverhead"].amount, 1024) + self.assertEqual(exec_reqs["pyspark.memory"].amount, 2048) + self.assertEqual(exec_reqs["gpu"].amount, 2) + self.assertEqual(exec_reqs["gpu"].discoveryScript, "testGpus") + self.assertEqual(exec_reqs["gpu"].resourceName, "gpu") + self.assertEqual(exec_reqs["gpu"].vendor, "nvidia.com") + self.assertEqual(len(task_reqs), 2) + self.assertEqual(task_reqs["cpus"].amount, 2.0) + self.assertEqual(task_reqs["gpu"].amount, 2.0) + + assert_request_contents(ereqs.requests, treqs.requests) + rp = rpb.require(ereqs).require(treqs).build + assert_request_contents(rp.executorResources, rp.taskResources) + from pyspark import SparkContext, SparkConf + sc = SparkContext(conf=SparkConf()) + rdd = sc.parallelize(range(10)).withResources(rp) + return_rp = rdd.getResourceProfile() + assert_request_contents(return_rp.executorResources, return_rp.taskResources) + # intermix objects created before SparkContext init and after + rpb2 = ResourceProfileBuilder() + # use reqs created before SparkContext with Builder after + rpb2.require(ereqs) + rpb2.require(treqs) + rp2 = rpb2.build + self.assertTrue(rp2.id > 0) + rdd2 = sc.parallelize(range(10)).withResources(rp2) + return_rp2 = rdd2.getResourceProfile() + assert_request_contents(return_rp2.executorResources, return_rp2.taskResources) + ereqs2 = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g") + ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com") + treqs2 = TaskResourceRequests().cpus(2).resource("gpu", 2) + # use reqs created after SparkContext with Builder before + rpb.require(ereqs2) + rpb.require(treqs2) + rp3 = rpb.build + assert_request_contents(rp3.executorResources, rp3.taskResources) + sc.stop() + + +if __name__ == "__main__": + from pyspark.resource.tests.test_resources import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 93137560de25e..cc614a2c0eaed 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -168,6 +168,21 @@ def _print_missing_jar(lib_name, pkg_name, jar_name, spark_version): }) +def _parse_memory(s): + """ + Parse a memory string in the format supported by Java (e.g. 1g, 200m) and + return the value in MiB + + >>> _parse_memory("256m") + 256 + >>> _parse_memory("2g") + 2048 + """ + units = {'g': 1024, 'm': 1, 't': 1 << 20, 'k': 1.0 / 1024} + if s[-1].lower() not in units: + raise ValueError("invalid format: " + s) + return int(float(s[:-1]) * units[s[-1].lower()]) + if __name__ == "__main__": import doctest (failure_count, test_count) = doctest.testmod() From 528094cdf0266a63080a92c42354ee2bc7037bf7 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 20 Apr 2020 10:32:31 -0500 Subject: [PATCH 29/31] add pyspark resource module to testing module --- dev/sparktestsupport/modules.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 391e4bbe1b1f0..85e6a1e9fadac 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -398,6 +398,21 @@ def __hash__(self): ) +pyspark_resource = Module( + name="pyspark-resource", + dependencies=[ + pyspark_core + ], + source_file_regexes=[ + "python/pyspark/resource" + ], + python_test_goals=[ + # unittests + "pyspark.resource.tests.test_resources", + ] +) + + pyspark_streaming = Module( name="pyspark-streaming", dependencies=[ From 89be02ea5be1e53e4a3921aa8df5c90f8b2173ce Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Apr 2020 08:47:49 -0500 Subject: [PATCH 30/31] Update names of function/variable --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- python/pyspark/resource/executorrequests.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e7f84770cf56a..bede0c561993e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1140,7 +1140,7 @@ private[spark] class DAGScheduler( * `PythonRunner` needs to know what the pyspark memory and cores settings are for the profile * being run. Pass them in the local properties of the task if it's set for the stage profile. */ - private def addPysparkConfigsToProperties(stage: Stage, properties: Properties): Unit = { + private def addPySparkConfigsToProperties(stage: Stage, properties: Properties): Unit = { val rp = sc.resourceProfileManager.resourceProfileFromId(stage.resourceProfileId) val pysparkMem = rp.getPySparkMemory // use the getOption on EXECUTOR_CORES.key instead of using the EXECUTOR_CORES config reader @@ -1176,7 +1176,7 @@ private[spark] class DAGScheduler( // Use the scheduling pool, job group, description, etc. from an ActiveJob associated // with this Stage val properties = jobIdToActiveJob(jobId).properties - addPysparkConfigsToProperties(stage, properties) + addPySparkConfigsToProperties(stage, properties) runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are diff --git a/python/pyspark/resource/executorrequests.py b/python/pyspark/resource/executorrequests.py index 36e747cf63b14..b34822944028e 100644 --- a/python/pyspark/resource/executorrequests.py +++ b/python/pyspark/resource/executorrequests.py @@ -57,7 +57,7 @@ class ExecutorResourceRequest(object): def __init__(self, resourceName, amount, discoveryScript="", vendor=""): self._name = resourceName self._amount = amount - self._discoveryScript = discoveryScript + self._discovery_script = discoveryScript self._vendor = vendor @property @@ -70,7 +70,7 @@ def amount(self): @property def discoveryScript(self): - return self._discoveryScript + return self._discovery_script @property def vendor(self): From 354fb0c09ff9ed6be985f0ca7a8b6fae835303a2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 22 Apr 2020 09:03:24 -0500 Subject: [PATCH 31/31] Other variable name changes --- python/pyspark/rdd.py | 8 ++++---- python/pyspark/resource/executorrequests.py | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 2a03f7e08a600..fbf645d10ee86 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -243,7 +243,7 @@ def __init__(self, jrdd, ctx, jrdd_deserializer=AutoBatchedSerializer(PickleSeri self._jrdd = jrdd self.is_cached = False self.is_checkpointed = False - self.has_resourceProfile = False + self.has_resource_profile = False self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer self._id = jrdd.id() @@ -2482,7 +2482,7 @@ def withResources(self, profile): .. versionadded:: 3.1.0 """ - self.has_resourceProfile = True + self.has_resource_profile = True if profile._java_resource_profile is not None: jrp = profile._java_resource_profile else: @@ -2616,7 +2616,7 @@ def pipeline_func(split, iterator): self._prev_jrdd = prev._prev_jrdd # maintain the pipeline self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer self.is_cached = False - self.has_resourceProfile = False + self.has_resource_profile = False self.is_checkpointed = False self.ctx = prev.ctx self.prev = prev @@ -2659,7 +2659,7 @@ def id(self): return self._id def _is_pipelinable(self): - return not (self.is_cached or self.is_checkpointed or self.has_resourceProfile) + return not (self.is_cached or self.is_checkpointed or self.has_resource_profile) def _is_barrier(self): return self.is_barrier diff --git a/python/pyspark/resource/executorrequests.py b/python/pyspark/resource/executorrequests.py index b34822944028e..91a195c94b6e5 100644 --- a/python/pyspark/resource/executorrequests.py +++ b/python/pyspark/resource/executorrequests.py @@ -83,8 +83,8 @@ class ExecutorResourceRequests(object): .. note:: Evolving A set of Executor resource requests. This is used in conjunction with the - ResourceProfileBuilder to programmatically specify the resources needed for an RDD - that will be applied at the stage level. + :class:`pyspark.resource.ResourceProfileBuilder` to programmatically specify the + resources needed for an RDD that will be applied at the stage level. .. versionadded:: 3.1.0 """