-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25004][CORE] Add spark.executor.pyspark.memory limit. #21977
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
a5004ba
306538b
9535a6b
5288f5b
fbac4a5
f11b3bb
ac7de4a
a38eac3
fcee94c
bb8fecb
0b275cf
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -114,6 +114,10 @@ package object config { | |
| .checkValue(_ >= 0, "The off-heap memory size must not be negative") | ||
| .createWithDefault(0) | ||
|
|
||
| private[spark] val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Argh, should have noticed this before. Should this be added to
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it should. I'll fix it. |
||
| .bytesConf(ByteUnit.MiB) | ||
| .createOptional | ||
|
||
|
|
||
| private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() | ||
| .booleanConf.createWithDefault(false) | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |
| import os | ||
| import sys | ||
| import time | ||
| import resource | ||
| import socket | ||
| import traceback | ||
|
|
||
|
|
@@ -263,6 +264,27 @@ def main(infile, outfile): | |
| isBarrier = read_bool(infile) | ||
| boundPort = read_int(infile) | ||
| secret = UTF8Deserializer().loads(infile) | ||
|
|
||
| # set up memory limits | ||
| memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1")) | ||
| total_memory = resource.RLIMIT_AS | ||
| try: | ||
| (total_memory_limit, max_total_memory) = resource.getrlimit(total_memory) | ||
| msg = "Current mem: {0} of max {1}\n".format(total_memory_limit, max_total_memory) | ||
| sys.stderr.write() | ||
|
||
|
|
||
| if memory_limit_mb > 0 and total_memory_limit < 0: | ||
|
||
| # convert to bytes | ||
| total_memory_limit = memory_limit_mb * 1024 * 1024 | ||
|
|
||
| msg = "Setting mem to {0} of max {1}\n".format(total_memory_limit, max_total_memory) | ||
| sys.stderr.write(msg) | ||
| resource.setrlimit(total_memory, (total_memory_limit, total_memory_limit)) | ||
|
|
||
| except (resource.error, OSError) as e: | ||
| # not all systems support resource limits, so warn instead of failing | ||
| sys.stderr.write("WARN: Failed to set memory limit: {0}\n".format(e)) | ||
|
||
|
|
||
| # initialize global state | ||
| taskContext = None | ||
| if isBarrier: | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -91,6 +91,13 @@ private[spark] class Client( | |
| private val executorMemoryOverhead = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse( | ||
| math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toLong, MEMORY_OVERHEAD_MIN)).toInt | ||
|
|
||
| private val isPython = sparkConf.get(IS_PYTHON_APP) | ||
|
||
| private val pysparkWorkerMemory: Int = if (isPython) { | ||
| sparkConf.get(PYSPARK_EXECUTOR_MEMORY).map(_.toInt).getOrElse(0) | ||
| } else { | ||
| 0 | ||
| } | ||
|
|
||
| private val distCacheMgr = new ClientDistributedCacheManager() | ||
|
|
||
| private val principal = sparkConf.get(PRINCIPAL).orNull | ||
|
|
@@ -333,7 +340,7 @@ private[spark] class Client( | |
| val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() | ||
| logInfo("Verifying our application has not requested more than the maximum " + | ||
| s"memory capability of the cluster ($maxMem MB per container)") | ||
| val executorMem = executorMemory + executorMemoryOverhead | ||
| val executorMem = executorMemory + executorMemoryOverhead + pysparkWorkerMemory | ||
| if (executorMem > maxMem) { | ||
| throw new IllegalArgumentException(s"Required executor memory ($executorMemory" + | ||
| s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " + | ||
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -133,10 +133,17 @@ private[yarn] class YarnAllocator( | |
| // Additional memory overhead. | ||
| protected val memoryOverhead: Int = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse( | ||
| math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)).toInt | ||
| protected val pysparkWorkerMemory: Int = if (sparkConf.get(IS_PYTHON_APP)) { | ||
| sparkConf.get(PYSPARK_EXECUTOR_MEMORY).map(_.toInt).getOrElse(0) | ||
|
||
| } else { | ||
| 0 | ||
| } | ||
| // Number of cores per executor. | ||
| protected val executorCores = sparkConf.get(EXECUTOR_CORES) | ||
| // Resource capability requested for each executors | ||
| private[yarn] val resource = Resource.newInstance(executorMemory + memoryOverhead, executorCores) | ||
| private[yarn] val resource = Resource.newInstance( | ||
| executorMemory + memoryOverhead + pysparkWorkerMemory, | ||
| executorCores) | ||
|
|
||
| private val launcherPool = ThreadUtils.newDaemonCachedThreadPool( | ||
| "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer | |
|
|
||
| import org.apache.spark.{SparkEnv, TaskContext} | ||
| import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} | ||
| import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
|
|
@@ -81,6 +82,17 @@ case class AggregateInPandasExec( | |
|
|
||
| val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536) | ||
| val reuseWorker = inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true) | ||
| val memoryMb = { | ||
|
||
| val allocation = inputRDD.conf.get(PYSPARK_EXECUTOR_MEMORY) | ||
| if (reuseWorker) { | ||
| // the shared python worker gets the entire allocation | ||
| allocation | ||
| } else { | ||
| // each python worker gets an equal part of the allocation | ||
| allocation.map(_ / inputRDD.conf.getInt("spark.executor.cores", 1)) | ||
| } | ||
| } | ||
|
|
||
| val sessionLocalTimeZone = conf.sessionLocalTimeZone | ||
| val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) | ||
|
|
||
|
|
@@ -139,6 +151,7 @@ case class AggregateInPandasExec( | |
| pyFuncs, | ||
| bufferSize, | ||
| reuseWorker, | ||
| memoryMb, | ||
| PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, | ||
| argOffsets, | ||
| aggInputSchema, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer | |
|
|
||
| import org.apache.spark.{SparkEnv, TaskContext} | ||
| import org.apache.spark.api.python.ChainedPythonFunctions | ||
| import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
|
|
@@ -80,6 +81,7 @@ abstract class EvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chil | |
| funcs: Seq[ChainedPythonFunctions], | ||
| bufferSize: Int, | ||
| reuseWorker: Boolean, | ||
| pyMemoryMb: Option[Long], | ||
| argOffsets: Array[Array[Int]], | ||
| iter: Iterator[InternalRow], | ||
| schema: StructType, | ||
|
|
@@ -89,6 +91,16 @@ abstract class EvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chil | |
| val inputRDD = child.execute().map(_.copy()) | ||
| val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536) | ||
| val reuseWorker = inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true) | ||
| val memoryMb = { | ||
|
||
| val allocation = inputRDD.conf.get(PYSPARK_EXECUTOR_MEMORY) | ||
| if (reuseWorker) { | ||
| // the shared python worker gets the entire allocation | ||
| allocation | ||
| } else { | ||
| // each python worker gets an equal part of the allocation | ||
| allocation.map(_ / inputRDD.conf.getInt("spark.executor.cores", 1)) | ||
| } | ||
| } | ||
|
|
||
| inputRDD.mapPartitions { iter => | ||
| val context = TaskContext.get() | ||
|
|
@@ -129,7 +141,7 @@ abstract class EvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chil | |
| } | ||
|
|
||
| val outputRowIterator = evaluate( | ||
| pyFuncs, bufferSize, reuseWorker, argOffsets, projectedRowIter, schema, context) | ||
| pyFuncs, bufferSize, reuseWorker, memoryMb, argOffsets, projectedRowIter, schema, context) | ||
|
|
||
| val joined = new JoinedRow | ||
| val resultProj = UnsafeProjection.create(output, output) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's been awhile since I spent a lot of time thinking about how we launch our python worker processes. Maybe it would make sense to add a comment here explaining the logic a bit more? Based on the documentation in
PythonWorkerFactoryit appears we do the fork/not-fork decision not based on if reuseworker is set but instead on if we're in Windows or not. Is that the logic that this block was attempting to handle?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I thought the comments below were clear: if a single worker is reused, it gets the entire allocation. If each core starts its own worker, each one gets an equal share.
If
reuseWorkeris actually ignored, then this needs to be updated.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think there might be a misunderstanding on what
reuseWorkermeans perhaps. The workers will be reused but the decision on if we fork in Python or not is based on if we are in Windows or not. How about we both go and read the code path there and see if we reach the same understanding? I could be off too.