Skip to content

Commit 9df4c68

Browse files
committed
SPARK-929: Fully deprecate usage of SPARK_MEM
This patch cements our deprecation of the SPARK_MEM environment variable by replacing it with case-specific variables: SPARK_DAEMON_MEMORY, SPARK_EXECUTOR_MEMORY, and SPARK_DRIVER_MEMORY The creation of the latter two variables means that we can safely set driver/job memory without accidentally setting the executor memory. Neither is public. SPARK_EXECUTOR_MEMORY is only used by the Mesos scheduler (and set within SparkContext). The proper way of configuring executor memory is through the "spark.executor.memory" property. SPARK_DRIVER_MEMORY is the new way of specifying the amount of memory run by jobs launched by spark-class, without possibly affecting executor memory. Other memory considerations: - The repl's memory can be set through the "--drivermem" command-line option, which really just sets SPARK_DRIVER_MEMORY. - run-example doesn't use spark-class, so the only way to modify examples' memory is actually an unusual use of SPARK_JAVA_OPTS (which is normally overriden in all cases by spark-class). This patch also fixes a lurking bug where spark-shell misused spark-class (the first argument is supposed to be the main class name, not java options).
1 parent 33baf14 commit 9df4c68

File tree

7 files changed

+90
-59
lines changed

7 files changed

+90
-59
lines changed

bin/spark-class

Lines changed: 28 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -40,34 +40,46 @@ if [ -z "$1" ]; then
4040
exit 1
4141
fi
4242

43-
# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
44-
# values for that; it doesn't need a lot
45-
if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
46-
SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
47-
SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
48-
# Do not overwrite SPARK_JAVA_OPTS environment variable in this script
49-
OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS" # Empty by default
50-
else
51-
OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
43+
if [ -n "$SPARK_MEM" ]; then
44+
echo "Warning: SPARK_MEM is deprecated, please use a more specific config option"
45+
echo "(e.g., spark.executor.memory or SPARK_DRIVER_MEMORY)."
5246
fi
5347

48+
# Use SPARK_MEM or 512m as the default memory, to be overridden by specific options
49+
DEFAULT_MEM=${SPARK_MEM:-512m}
50+
51+
SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
5452

55-
# Add java opts for master, worker, executor. The opts maybe null
53+
# Add java opts and memory settings for master, worker, executors, and repl.
5654
case "$1" in
55+
# Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY.
5756
'org.apache.spark.deploy.master.Master')
58-
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
57+
OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS"
58+
OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM}
5959
;;
6060
'org.apache.spark.deploy.worker.Worker')
61-
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
61+
OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS"
62+
OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM}
6263
;;
64+
65+
# Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY.
6366
'org.apache.spark.executor.CoarseGrainedExecutorBackend')
64-
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
67+
OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
68+
OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM}
6569
;;
6670
'org.apache.spark.executor.MesosExecutorBackend')
67-
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
71+
OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
72+
OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM}
6873
;;
74+
75+
# All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS.
6976
'org.apache.spark.repl.Main')
70-
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
77+
OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS"
78+
OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM}
79+
;;
80+
*)
81+
OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
82+
OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM}
7183
;;
7284
esac
7385

@@ -83,14 +95,10 @@ else
8395
fi
8496
fi
8597

86-
# Set SPARK_MEM if it isn't already set since we also use it for this process
87-
SPARK_MEM=${SPARK_MEM:-512m}
88-
export SPARK_MEM
89-
9098
# Set JAVA_OPTS to be able to load native libraries and to set heap size
9199
JAVA_OPTS="$OUR_JAVA_OPTS"
92100
JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
93-
JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
101+
JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM"
94102
# Load extra JAVA_OPTS from conf/java-opts, if it exists
95103
if [ -e "$FWDIR/conf/java-opts" ] ; then
96104
JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"

bin/spark-class2.cmd

Lines changed: 35 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -34,22 +34,45 @@ if not "x%1"=="x" goto arg_given
3434
goto exit
3535
:arg_given
3636

37-
set RUNNING_DAEMON=0
38-
if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1
39-
if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1
40-
if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m
37+
if not "x%SPARK_MEM%"=="x" (
38+
echo Warning: SPARK_MEM is deprecated, please use a more specific config option
39+
echo e.g., spark.executor.memory or SPARK_DRIVER_MEMORY.
40+
)
41+
42+
rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options
43+
set OUR_JAVA_MEM=%SPARK_MEM%
44+
if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m
45+
4146
set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true
42-
if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY%
43-
rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script
44-
if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
45-
if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS%
4647

47-
rem Figure out how much memory to use per executor and set it as an environment
48-
rem variable so that our process sees it and can report it to Mesos
49-
if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
48+
rem Add java opts and memory settings for master, worker, executors, and repl.
49+
rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY.
50+
if "%1"=="org.apache.spark.deploy.master.Master" (
51+
set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS%
52+
if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY%
53+
) else if "%1"=="org.apache.spark.deploy.worker.Worker" (
54+
set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS%
55+
if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY%
56+
57+
rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY.
58+
) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" (
59+
set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS%
60+
if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY%
61+
) else if "%1"=="org.apache.spark.executor.MesosExecutorBackend" (
62+
set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS%
63+
if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY%
64+
65+
rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS.
66+
) else if "%1"=="org.apache.spark.repl.Main" (
67+
set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_REPL_OPTS%
68+
if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY%
69+
) else (
70+
set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS%
71+
if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY%
72+
)
5073

5174
rem Set JAVA_OPTS to be able to load native libraries and to set heap size
52-
set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
75+
set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM%
5376
rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
5477

5578
rem Test whether the user has built Spark

bin/spark-shell

Lines changed: 14 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -45,13 +45,11 @@ if [ "$1" = "--help" ] || [ "$1" = "-h" ]; then
4545
exit
4646
fi
4747

48-
SPARK_SHELL_OPTS=""
49-
5048
for o in "$@"; do
5149
if [ "$1" = "-c" -o "$1" = "--cores" ]; then
5250
shift
5351
if [[ "$1" =~ $CORE_PATTERN ]]; then
54-
SPARK_SHELL_OPTS="$SPARK_SHELL_OPTS -Dspark.cores.max=$1"
52+
SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1"
5553
shift
5654
else
5755
echo "ERROR: wrong format for -c/--cores"
@@ -61,7 +59,7 @@ for o in "$@"; do
6159
if [ "$1" = "-em" -o "$1" = "--execmem" ]; then
6260
shift
6361
if [[ $1 =~ $MEM_PATTERN ]]; then
64-
SPARK_SHELL_OPTS="$SPARK_SHELL_OPTS -Dspark.executor.memory=$1"
62+
SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1"
6563
shift
6664
else
6765
echo "ERROR: wrong format for --execmem/-em"
@@ -71,7 +69,7 @@ for o in "$@"; do
7169
if [ "$1" = "-dm" -o "$1" = "--drivermem" ]; then
7270
shift
7371
if [[ $1 =~ $MEM_PATTERN ]]; then
74-
export SPARK_MEM=$1
72+
export SPARK_DRIVER_MEMORY=$1
7573
shift
7674
else
7775
echo "ERROR: wrong format for --drivermem/-dm"
@@ -125,16 +123,18 @@ if [[ ! $? ]]; then
125123
fi
126124

127125
if $cygwin; then
128-
# Workaround for issue involving JLine and Cygwin
129-
# (see http://sourceforge.net/p/jline/bugs/40/).
130-
# If you're using the Mintty terminal emulator in Cygwin, may need to set the
131-
# "Backspace sends ^H" setting in "Keys" section of the Mintty options
132-
# (see https://github.com/sbt/sbt/issues/562).
133-
stty -icanon min 1 -echo > /dev/null 2>&1
134-
$FWDIR/bin/spark-class -Djline.terminal=unix $SPARK_SHELL_OPTS org.apache.spark.repl.Main "$@"
135-
stty icanon echo > /dev/null 2>&1
126+
# Workaround for issue involving JLine and Cygwin
127+
# (see http://sourceforge.net/p/jline/bugs/40/).
128+
# If you're using the Mintty terminal emulator in Cygwin, may need to set the
129+
# "Backspace sends ^H" setting in "Keys" section of the Mintty options
130+
# (see https://github.com/sbt/sbt/issues/562).
131+
stty -icanon min 1 -echo > /dev/null 2>&1
132+
export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix"
133+
$FWDIR/bin/spark-class org.apache.spark.repl.Main "$@"
134+
stty icanon echo > /dev/null 2>&1
136135
else
137-
$FWDIR/bin/spark-class $SPARK_SHELL_OPTS org.apache.spark.repl.Main "$@"
136+
export SPARK_REPL_OPTS
137+
$FWDIR/bin/spark-class org.apache.spark.repl.Main "$@"
138138
fi
139139

140140
# record the exit status lest it be overwritten:

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

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -162,19 +162,20 @@ class SparkContext(
162162
jars.foreach(addJar)
163163
}
164164

165+
def warnSparkMem(value: String): String = {
166+
logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " +
167+
"deprecated, please use spark.executor.memory instead.")
168+
value
169+
}
170+
165171
private[spark] val executorMemory = conf.getOption("spark.executor.memory")
166-
.orElse(Option(System.getenv("SPARK_MEM")))
172+
.orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY")))
173+
.orElse(Option(System.getenv("SPARK_MEM")).map(warnSparkMem))
167174
.map(Utils.memoryStringToMb)
168175
.getOrElse(512)
169176

170-
if (!conf.contains("spark.executor.memory") && sys.env.contains("SPARK_MEM")) {
171-
logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " +
172-
"deprecated, instead use spark.executor.memory")
173-
}
174-
175177
// Environment variables to pass to our executors
176178
private[spark] val executorEnvs = HashMap[String, String]()
177-
// Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
178179
for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS");
179180
value <- Option(System.getenv(key))) {
180181
executorEnvs(key) = value
@@ -185,8 +186,9 @@ class SparkContext(
185186
value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} {
186187
executorEnvs(envKey) = value
187188
}
188-
// Since memory can be set with a system property too, use that
189-
executorEnvs("SPARK_MEM") = executorMemory + "m"
189+
// The Mesos scheduler backend relies on this environment variable to set executor memory.
190+
// TODO: Set this only in the Mesos scheduler.
191+
executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m"
190192
executorEnvs ++= conf.getExecutorEnv
191193

192194
// Set SPARK_USER for user who is running SparkContext.

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -532,8 +532,6 @@ private[spark] object Utils extends Logging {
532532

533533
/**
534534
* Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes.
535-
* This is used to figure out how much memory to claim from Mesos based on the SPARK_MEM
536-
* environment variable.
537535
*/
538536
def memoryStringToMb(str: String): Int = {
539537
val lower = str.toLowerCase

docs/tuning.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,7 @@ their work directories), *not* on your driver program.
163163
**Cache Size Tuning**
164164

165165
One important configuration parameter for GC is the amount of memory that should be used for caching RDDs.
166-
By default, Spark uses 60% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to
166+
By default, Spark uses 60% of the configured executor memory (`spark.executor.memory`) to
167167
cache RDDs. This means that 40% of memory is available for any objects created during task execution.
168168

169169
In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of

python/pyspark/java_gateway.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@
2929

3030
def launch_gateway():
3131
# Launch the Py4j gateway using Spark's run command so that we pick up the
32-
# proper classpath and SPARK_MEM settings from spark-env.sh
32+
# proper classpath and settings from spark-env.sh
3333
on_windows = platform.system() == "Windows"
3434
script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
3535
command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",

0 commit comments

Comments
 (0)