Skip to content

Commit 0cfda84

Browse files
committed
[SPARK-2313] Use socket to communicate GatewayServer port back to Python driver
This patch changes PySpark so that the GatewayServer's port is communicated back to the Python process that launches it over a local socket instead of a pipe. The old pipe-based approach was brittle and could fail if `spark-submit` printed unexpected to stdout. To accomplish this, I wrote a custom `PythonGatewayServer.main()` function to use in place of Py4J's `GatewayServer.main()`. Closes #3424. Author: Josh Rosen <joshrosen@databricks.com> Closes #4603 from JoshRosen/SPARK-2313 and squashes the following commits: 6a7740b [Josh Rosen] Remove EchoOutputThread since it's no longer needed 0db501f [Josh Rosen] Use select() so that we don't block if GatewayServer dies. 9bdb4b6 [Josh Rosen] Handle case where getListeningPort returns -1 3fb7ed1 [Josh Rosen] Remove stdout=PIPE 2458934 [Josh Rosen] Use underscore to mark env var. as private d12c95d [Josh Rosen] Use Logging and Utils.tryOrExit() e5f9730 [Josh Rosen] Wrap everything in a giant try-block 2f70689 [Josh Rosen] Use stdin PIPE to share fate with driver 8bf956e [Josh Rosen] Initial cut at passing Py4J gateway port back to driver via socket
1 parent c01c4eb commit 0cfda84

File tree

3 files changed

+97
-43
lines changed

3 files changed

+97
-43
lines changed
Lines changed: 64 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,64 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.api.python
19+
20+
import java.io.DataOutputStream
21+
import java.net.Socket
22+
23+
import py4j.GatewayServer
24+
25+
import org.apache.spark.Logging
26+
import org.apache.spark.util.Utils
27+
28+
/**
29+
* Process that starts a Py4J GatewayServer on an ephemeral port and communicates the bound port
30+
* back to its caller via a callback port specified by the caller.
31+
*
32+
* This process is launched (via SparkSubmit) by the PySpark driver (see java_gateway.py).
33+
*/
34+
private[spark] object PythonGatewayServer extends Logging {
35+
def main(args: Array[String]): Unit = Utils.tryOrExit {
36+
// Start a GatewayServer on an ephemeral port
37+
val gatewayServer: GatewayServer = new GatewayServer(null, 0)
38+
gatewayServer.start()
39+
val boundPort: Int = gatewayServer.getListeningPort
40+
if (boundPort == -1) {
41+
logError("GatewayServer failed to bind; exiting")
42+
System.exit(1)
43+
} else {
44+
logDebug(s"Started PythonGatewayServer on port $boundPort")
45+
}
46+
47+
// Communicate the bound port back to the caller via the caller-specified callback port
48+
val callbackHost = sys.env("_PYSPARK_DRIVER_CALLBACK_HOST")
49+
val callbackPort = sys.env("_PYSPARK_DRIVER_CALLBACK_PORT").toInt
50+
logDebug(s"Communicating GatewayServer port to Python driver at $callbackHost:$callbackPort")
51+
val callbackSocket = new Socket(callbackHost, callbackPort)
52+
val dos = new DataOutputStream(callbackSocket.getOutputStream)
53+
dos.writeInt(boundPort)
54+
dos.close()
55+
callbackSocket.close()
56+
57+
// Exit on EOF or broken pipe to ensure that this process dies when the Python driver dies:
58+
while (System.in.read() != -1) {
59+
// Do nothing
60+
}
61+
logDebug("Exiting due to broken pipe from Python driver")
62+
System.exit(0)
63+
}
64+
}

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

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,6 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver}
3939

4040
import org.apache.spark.SPARK_VERSION
4141
import org.apache.spark.deploy.rest._
42-
import org.apache.spark.executor._
4342
import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils}
4443

4544
/**
@@ -284,8 +283,7 @@ object SparkSubmit {
284283
// If we're running a python app, set the main class to our specific python runner
285284
if (args.isPython && deployMode == CLIENT) {
286285
if (args.primaryResource == PYSPARK_SHELL) {
287-
args.mainClass = "py4j.GatewayServer"
288-
args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0")
286+
args.mainClass = "org.apache.spark.api.python.PythonGatewayServer"
289287
} else {
290288
// If a python file is provided, add it to the child arguments and list of files to deploy.
291289
// Usage: PythonAppRunner <main python file> <extra python files> [app arguments]

python/pyspark/java_gateway.py

Lines changed: 32 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -17,19 +17,20 @@
1717

1818
import atexit
1919
import os
20-
import sys
20+
import select
2121
import signal
2222
import shlex
23+
import socket
2324
import platform
2425
from subprocess import Popen, PIPE
25-
from threading import Thread
2626
from py4j.java_gateway import java_import, JavaGateway, GatewayClient
2727

28+
from pyspark.serializers import read_int
29+
2830

2931
def launch_gateway():
3032
SPARK_HOME = os.environ["SPARK_HOME"]
3133

32-
gateway_port = -1
3334
if "PYSPARK_GATEWAY_PORT" in os.environ:
3435
gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"])
3536
else:
@@ -41,36 +42,42 @@ def launch_gateway():
4142
submit_args = submit_args if submit_args is not None else ""
4243
submit_args = shlex.split(submit_args)
4344
command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"]
45+
46+
# Start a socket that will be used by PythonGatewayServer to communicate its port to us
47+
callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
48+
callback_socket.bind(('127.0.0.1', 0))
49+
callback_socket.listen(1)
50+
callback_host, callback_port = callback_socket.getsockname()
51+
env = dict(os.environ)
52+
env['_PYSPARK_DRIVER_CALLBACK_HOST'] = callback_host
53+
env['_PYSPARK_DRIVER_CALLBACK_PORT'] = str(callback_port)
54+
55+
# Launch the Java gateway.
56+
# We open a pipe to stdin so that the Java gateway can die when the pipe is broken
4457
if not on_windows:
4558
# Don't send ctrl-c / SIGINT to the Java gateway:
4659
def preexec_func():
4760
signal.signal(signal.SIGINT, signal.SIG_IGN)
48-
env = dict(os.environ)
4961
env["IS_SUBPROCESS"] = "1" # tell JVM to exit after python exits
50-
proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func, env=env)
62+
proc = Popen(command, stdin=PIPE, preexec_fn=preexec_func, env=env)
5163
else:
5264
# preexec_fn not supported on Windows
53-
proc = Popen(command, stdout=PIPE, stdin=PIPE)
65+
proc = Popen(command, stdin=PIPE, env=env)
5466

55-
try:
56-
# Determine which ephemeral port the server started on:
57-
gateway_port = proc.stdout.readline()
58-
gateway_port = int(gateway_port)
59-
except ValueError:
60-
# Grab the remaining lines of stdout
61-
(stdout, _) = proc.communicate()
62-
exit_code = proc.poll()
63-
error_msg = "Launching GatewayServer failed"
64-
error_msg += " with exit code %d!\n" % exit_code if exit_code else "!\n"
65-
error_msg += "Warning: Expected GatewayServer to output a port, but found "
66-
if gateway_port == "" and stdout == "":
67-
error_msg += "no output.\n"
68-
else:
69-
error_msg += "the following:\n\n"
70-
error_msg += "--------------------------------------------------------------\n"
71-
error_msg += gateway_port + stdout
72-
error_msg += "--------------------------------------------------------------\n"
73-
raise Exception(error_msg)
67+
gateway_port = None
68+
# We use select() here in order to avoid blocking indefinitely if the subprocess dies
69+
# before connecting
70+
while gateway_port is None and proc.poll() is None:
71+
timeout = 1 # (seconds)
72+
readable, _, _ = select.select([callback_socket], [], [], timeout)
73+
if callback_socket in readable:
74+
gateway_connection = callback_socket.accept()[0]
75+
# Determine which ephemeral port the server started on:
76+
gateway_port = read_int(gateway_connection.makefile())
77+
gateway_connection.close()
78+
callback_socket.close()
79+
if gateway_port is None:
80+
raise Exception("Java gateway process exited before sending the driver its port number")
7481

7582
# In Windows, ensure the Java child processes do not linger after Python has exited.
7683
# In UNIX-based systems, the child process can kill itself on broken pipe (i.e. when
@@ -88,21 +95,6 @@ def killChild():
8895
Popen(["cmd", "/c", "taskkill", "/f", "/t", "/pid", str(proc.pid)])
8996
atexit.register(killChild)
9097

91-
# Create a thread to echo output from the GatewayServer, which is required
92-
# for Java log output to show up:
93-
class EchoOutputThread(Thread):
94-
95-
def __init__(self, stream):
96-
Thread.__init__(self)
97-
self.daemon = True
98-
self.stream = stream
99-
100-
def run(self):
101-
while True:
102-
line = self.stream.readline()
103-
sys.stderr.write(line)
104-
EchoOutputThread(proc.stdout).start()
105-
10698
# Connect to the gateway
10799
gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False)
108100

0 commit comments

Comments
 (0)