Skip to content

Commit 318739a

Browse files
andrewor14pwendell
authored andcommitted
[SPARK-1808] Route bin/pyspark through Spark submit
**Problem.** For `bin/pyspark`, there is currently no other way to specify Spark configuration properties other than through `SPARK_JAVA_OPTS` in `conf/spark-env.sh`. However, this mechanism is supposedly deprecated. Instead, it needs to pick up configurations explicitly specified in `conf/spark-defaults.conf`. **Solution.** Have `bin/pyspark` invoke `bin/spark-submit`, like all of its counterparts in Scala land (i.e. `bin/spark-shell`, `bin/run-example`). This has the additional benefit of making the invocation of all the user facing Spark scripts consistent. **Details.** `bin/pyspark` inherently handles two cases: (1) running python applications and (2) running the python shell. For (1), Spark submit already handles running python applications. For cases in which `bin/pyspark` is given a python file, we can simply call pass the file directly to Spark submit and let it handle the rest. For case (2), `bin/pyspark` starts a python process as before, which launches the JVM as a sub-process. The existing code already provides a code path to do this. All we needed to change is to use `bin/spark-submit` instead of `spark-class` to launch the JVM. This requires modifications to Spark submit to handle the pyspark shell as a special case. This has been tested locally (OSX and Windows 7), on a standalone cluster, and on a YARN cluster. Running IPython also works as before, except now it takes in Spark submit arguments too. Author: Andrew Or <[email protected]> Closes #799 from andrewor14/pyspark-submit and squashes the following commits: bf37e36 [Andrew Or] Minor changes 01066fa [Andrew Or] bin/pyspark for Windows c8cb3bf [Andrew Or] Handle perverse app names (with escaped quotes) 1866f85 [Andrew Or] Windows is not cooperating 456d844 [Andrew Or] Guard against shlex hanging if PYSPARK_SUBMIT_ARGS is not set 7eebda8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit b7ba0d8 [Andrew Or] Address a few comments (minor) 06eb138 [Andrew Or] Use shlex instead of writing our own parser 05879fa [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit a823661 [Andrew Or] Fix --die-on-broken-pipe not propagated properly 6fba412 [Andrew Or] Deal with quotes + address various comments fe4c8a7 [Andrew Or] Update --help for bin/pyspark afe47bf [Andrew Or] Fix spark shell f04aaa4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pyspark-submit a371d26 [Andrew Or] Route bin/pyspark through Spark submit (cherry picked from commit 4b8ec6f) Signed-off-by: Patrick Wendell <[email protected]>
1 parent 9cd12f3 commit 318739a

File tree

10 files changed

+107
-34
lines changed

10 files changed

+107
-34
lines changed

bin/pyspark

Lines changed: 31 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,12 @@ export SPARK_HOME="$FWDIR"
2525

2626
SCALA_VERSION=2.10
2727

28+
if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
29+
echo "Usage: ./bin/pyspark [options]"
30+
./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2
31+
exit 0
32+
fi
33+
2834
# Exit if the user hasn't compiled Spark
2935
if [ ! -f "$FWDIR/RELEASE" ]; then
3036
# Exit if the user hasn't compiled Spark
@@ -52,13 +58,34 @@ export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH
5258
export OLD_PYTHONSTARTUP=$PYTHONSTARTUP
5359
export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py
5460

61+
# If IPython options are specified, assume user wants to run IPython
5562
if [ -n "$IPYTHON_OPTS" ]; then
5663
IPYTHON=1
5764
fi
5865

59-
# Only use ipython if no command line arguments were provided [SPARK-1134]
60-
if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then
61-
exec ipython $IPYTHON_OPTS
66+
# Build up arguments list manually to preserve quotes and backslashes.
67+
# We export Spark submit arguments as an environment variable because shell.py must run as a
68+
# PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks.
69+
70+
PYSPARK_SUBMIT_ARGS=""
71+
whitespace="[[:space:]]"
72+
for i in "$@"; do
73+
if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi
74+
if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi
75+
PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i"
76+
done
77+
export PYSPARK_SUBMIT_ARGS
78+
79+
# If a python file is provided, directly run spark-submit.
80+
if [[ "$1" =~ \.py$ ]]; then
81+
echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2
82+
echo -e "Use ./bin/spark-submit <python file>\n" 1>&2
83+
exec $FWDIR/bin/spark-submit "$@"
6284
else
63-
exec "$PYSPARK_PYTHON" "$@"
85+
# Only use ipython if no command line arguments were provided [SPARK-1134]
86+
if [[ "$IPYTHON" = "1" ]]; then
87+
exec ipython $IPYTHON_OPTS
88+
else
89+
exec "$PYSPARK_PYTHON"
90+
fi
6491
fi

bin/pyspark2.cmd

Lines changed: 18 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ set FOUND_JAR=0
3131
for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do (
3232
set FOUND_JAR=1
3333
)
34-
if "%FOUND_JAR%"=="0" (
34+
if [%FOUND_JAR%] == [0] (
3535
echo Failed to find Spark assembly JAR.
3636
echo You need to build Spark with sbt\sbt assembly before running this program.
3737
goto exit
@@ -42,15 +42,30 @@ rem Load environment variables from conf\spark-env.cmd, if it exists
4242
if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
4343

4444
rem Figure out which Python to use.
45-
if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python
45+
if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python
4646

4747
set PYTHONPATH=%FWDIR%python;%PYTHONPATH%
4848
set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH%
4949

5050
set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
5151
set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py
52+
set PYSPARK_SUBMIT_ARGS=%*
5253

5354
echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH%
5455

55-
"%PYSPARK_PYTHON%" %*
56+
rem Check whether the argument is a file
57+
for /f %%i in ('echo %1^| findstr /R "\.py"') do (
58+
set PYTHON_FILE=%%i
59+
)
60+
61+
if [%PYTHON_FILE%] == [] (
62+
%PYSPARK_PYTHON%
63+
) else (
64+
echo.
65+
echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0.
66+
echo Use ./bin/spark-submit ^<python file^>
67+
echo.
68+
"%FWDIR%\bin\spark-submit.cmd" %PYSPARK_SUBMIT_ARGS%
69+
)
70+
5671
:exit

bin/spark-shell

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ esac
2828
# Enter posix mode for bash
2929
set -o posix
3030

31-
if [[ "$@" == *--help* ]]; then
31+
if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
3232
echo "Usage: ./bin/spark-shell [options]"
3333
./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2
3434
exit 0
@@ -46,11 +46,11 @@ function main(){
4646
# (see https://github.com/sbt/sbt/issues/562).
4747
stty -icanon min 1 -echo > /dev/null 2>&1
4848
export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix"
49-
$FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main
49+
$FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main
5050
stty icanon echo > /dev/null 2>&1
5151
else
5252
export SPARK_SUBMIT_OPTS
53-
$FWDIR/bin/spark-submit spark-internal "$@" --class org.apache.spark.repl.Main
53+
$FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main
5454
fi
5555
}
5656

bin/spark-shell.cmd

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,4 +19,4 @@ rem
1919

2020
set SPARK_HOME=%~dp0..
2121

22-
cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main
22+
cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main

core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ object PythonRunner {
4242
// Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the
4343
// python directories in SPARK_HOME (if set), and any files in the pyFiles argument
4444
val pathElements = new ArrayBuffer[String]
45-
pathElements ++= pyFiles.split(",")
45+
pathElements ++= Option(pyFiles).getOrElse("").split(",")
4646
pathElements += PythonUtils.sparkPythonPath
4747
pathElements += sys.env.getOrElse("PYTHONPATH", "")
4848
val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*)

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 41 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -41,10 +41,10 @@ object SparkSubmit {
4141
private var clusterManager: Int = LOCAL
4242

4343
/**
44-
* A special jar name that indicates the class being run is inside of Spark itself,
45-
* and therefore no user jar is needed.
44+
* Special primary resource names that represent shells rather than application jars.
4645
*/
47-
private val RESERVED_JAR_NAME = "spark-internal"
46+
private val SPARK_SHELL = "spark-shell"
47+
private val PYSPARK_SHELL = "pyspark-shell"
4848

4949
def main(args: Array[String]) {
5050
val appArgs = new SparkSubmitArguments(args)
@@ -71,8 +71,8 @@ object SparkSubmit {
7171
* entries for the child, a list of system properties, a list of env vars
7272
* and the main class for the child
7373
*/
74-
private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String],
75-
ArrayBuffer[String], Map[String, String], String) = {
74+
private[spark] def createLaunchEnv(args: SparkSubmitArguments)
75+
: (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = {
7676
if (args.master.startsWith("local")) {
7777
clusterManager = LOCAL
7878
} else if (args.master.startsWith("yarn")) {
@@ -121,24 +121,30 @@ object SparkSubmit {
121121
printErrorAndExit("Cannot currently run driver on the cluster in Mesos")
122122
}
123123

124-
// If we're running a Python app, set the Java class to run to be our PythonRunner, add
125-
// Python files to deployment list, and pass the main file and Python path to PythonRunner
124+
// If we're running a python app, set the main class to our specific python runner
126125
if (isPython) {
127126
if (deployOnCluster) {
128127
printErrorAndExit("Cannot currently run Python driver programs on cluster")
129128
}
130-
args.mainClass = "org.apache.spark.deploy.PythonRunner"
131-
args.files = mergeFileLists(args.files, args.pyFiles, args.primaryResource)
129+
if (args.primaryResource == PYSPARK_SHELL) {
130+
args.mainClass = "py4j.GatewayServer"
131+
args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0")
132+
} else {
133+
// If a python file is provided, add it to the child arguments and list of files to deploy.
134+
// Usage: PythonAppRunner <main python file> <extra python files> [app arguments]
135+
args.mainClass = "org.apache.spark.deploy.PythonRunner"
136+
args.childArgs = ArrayBuffer(args.primaryResource, args.pyFiles) ++ args.childArgs
137+
args.files = mergeFileLists(args.files, args.primaryResource)
138+
}
132139
val pyFiles = Option(args.pyFiles).getOrElse("")
133-
args.childArgs = ArrayBuffer(args.primaryResource, pyFiles) ++ args.childArgs
134-
args.primaryResource = RESERVED_JAR_NAME
140+
args.files = mergeFileLists(args.files, pyFiles)
135141
sysProps("spark.submit.pyFiles") = pyFiles
136142
}
137143

138144
// If we're deploying into YARN, use yarn.Client as a wrapper around the user class
139145
if (!deployOnCluster) {
140146
childMainClass = args.mainClass
141-
if (args.primaryResource != RESERVED_JAR_NAME) {
147+
if (isUserJar(args.primaryResource)) {
142148
childClasspath += args.primaryResource
143149
}
144150
} else if (clusterManager == YARN) {
@@ -219,7 +225,7 @@ object SparkSubmit {
219225
// For python files, the primary resource is already distributed as a regular file
220226
if (!isYarnCluster && !isPython) {
221227
var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq())
222-
if (args.primaryResource != RESERVED_JAR_NAME) {
228+
if (isUserJar(args.primaryResource)) {
223229
jars = jars ++ Seq(args.primaryResource)
224230
}
225231
sysProps.put("spark.jars", jars.mkString(","))
@@ -293,7 +299,7 @@ object SparkSubmit {
293299
}
294300

295301
private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) {
296-
val localJarFile = new File(new URI(localJar).getPath())
302+
val localJarFile = new File(new URI(localJar).getPath)
297303
if (!localJarFile.exists()) {
298304
printWarning(s"Jar $localJar does not exist, skipping.")
299305
}
@@ -302,6 +308,27 @@ object SparkSubmit {
302308
loader.addURL(url)
303309
}
304310

311+
/**
312+
* Return whether the given primary resource represents a user jar.
313+
*/
314+
private def isUserJar(primaryResource: String): Boolean = {
315+
!isShell(primaryResource) && !isPython(primaryResource)
316+
}
317+
318+
/**
319+
* Return whether the given primary resource represents a shell.
320+
*/
321+
private def isShell(primaryResource: String): Boolean = {
322+
primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL
323+
}
324+
325+
/**
326+
* Return whether the given primary resource requires running python.
327+
*/
328+
private[spark] def isPython(primaryResource: String): Boolean = {
329+
primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL
330+
}
331+
305332
/**
306333
* Merge a sequence of comma-separated file lists, some of which may be null to indicate
307334
* no files, into a single comma-separated string.

core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -298,11 +298,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) {
298298
case v =>
299299
primaryResource = v
300300
inSparkOpts = false
301-
isPython = v.endsWith(".py")
301+
isPython = SparkSubmit.isPython(v)
302302
parse(tail)
303303
}
304304
} else {
305-
childArgs += value
305+
if (!value.isEmpty) {
306+
childArgs += value
307+
}
306308
parse(tail)
307309
}
308310

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1101,7 +1101,7 @@ private[spark] object Utils extends Logging {
11011101
* Strip the directory from a path name
11021102
*/
11031103
def stripDirectory(path: String): String = {
1104-
path.split(File.separator).last
1104+
new File(path).getName
11051105
}
11061106

11071107
/**

python/pyspark/java_gateway.py

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,12 @@
1818
import os
1919
import sys
2020
import signal
21+
import shlex
2122
import platform
2223
from subprocess import Popen, PIPE
2324
from threading import Thread
2425
from py4j.java_gateway import java_import, JavaGateway, GatewayClient
2526

26-
2727
def launch_gateway():
2828
SPARK_HOME = os.environ["SPARK_HOME"]
2929

@@ -34,9 +34,11 @@ def launch_gateway():
3434
# Launch the Py4j gateway using Spark's run command so that we pick up the
3535
# proper classpath and settings from spark-env.sh
3636
on_windows = platform.system() == "Windows"
37-
script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
38-
command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
39-
"--die-on-broken-pipe", "0"]
37+
script = "./bin/spark-submit.cmd" if on_windows else "./bin/spark-submit"
38+
submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS")
39+
submit_args = submit_args if submit_args is not None else ""
40+
submit_args = shlex.split(submit_args)
41+
command = [os.path.join(SPARK_HOME, script), "pyspark-shell"] + submit_args
4042
if not on_windows:
4143
# Don't send ctrl-c / SIGINT to the Java gateway:
4244
def preexec_func():

python/pyspark/shell.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@
4040
if os.environ.get("SPARK_EXECUTOR_URI"):
4141
SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"])
4242

43-
sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files)
43+
sc = SparkContext(appName="PySparkShell", pyFiles=add_files)
4444

4545
print("""Welcome to
4646
____ __

0 commit comments

Comments
 (0)