04e44b37cc
This PR update PySpark to support Python 3 (tested with 3.4). Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped. TODO: ec2/spark-ec2.py is not fully tested with python3. Author: Davies Liu <davies@databricks.com> Author: twneale <twneale@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Closes #5173 from davies/python3 and squashes the following commits: d7d6323 [Davies Liu] fix tests 6c52a98 [Davies Liu] fix mllib test 99e334f [Davies Liu] update timeout b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 cafd5ec [Davies Liu] adddress comments from @mengxr bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 179fc8d [Davies Liu] tuning flaky tests 8c8b957 [Davies Liu] fix ResourceWarning in Python 3 5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 4006829 [Davies Liu] fix test 2fc0066 [Davies Liu] add python3 path 71535e9 [Davies Liu] fix xrange and divide 5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ed498c8 [Davies Liu] fix compatibility with python 3 820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ad7c374 [Davies Liu] fix mllib test and warning ef1fc2f [Davies Liu] fix tests 4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 59bb492 [Davies Liu] fix tests 1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ca0fdd3 [Davies Liu] fix code style 9563a15 [Davies Liu] add imap back for python 2 0b1ec04 [Davies Liu] make python examples work with Python 3 d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 a716d34 [Davies Liu] test with python 3.4 f1700e8 [Davies Liu] fix test in python3 671b1db [Davies Liu] fix test in python3 692ff47 [Davies Liu] fix flaky test 7b9699f [Davies Liu] invalidate import cache for Python 3.3+ 9c58497 [Davies Liu] fix kill worker 309bfbf [Davies Liu] keep compatibility 5707476 [Davies Liu] cleanup, fix hash of string in 3.3+ 8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 f53e1f0 [Davies Liu] fix tests 70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3 a39167e [Davies Liu] support customize class in __main__ 814c77b [Davies Liu] run unittests with python 3 7f4476e [Davies Liu] mllib tests passed d737924 [Davies Liu] pass ml tests 375ea17 [Davies Liu] SQL tests pass 6cc42a9 [Davies Liu] rename 431a8de [Davies Liu] streaming tests pass 78901a7 [Davies Liu] fix hash of serializer in Python 3 24b2f2e [Davies Liu] pass all RDD tests 35f48fe [Davies Liu] run future again 1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py 6e3c21d [Davies Liu] make cloudpickle work with Python3 2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run 1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out 7354371 [twneale] buffer --> memoryview I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work. b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?). f40d925 [twneale] xrange --> range e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206 79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper 2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3 854be27 [Josh Rosen] Run `futurize` on Python code: 7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py.
181 lines
6 KiB
Python
181 lines
6 KiB
Python
#
|
|
# 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 numbers
|
|
import os
|
|
import signal
|
|
import select
|
|
import socket
|
|
import sys
|
|
import traceback
|
|
import time
|
|
import gc
|
|
from errno import EINTR, EAGAIN
|
|
from socket import AF_INET, SOCK_STREAM, SOMAXCONN
|
|
from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT
|
|
|
|
from pyspark.worker import main as worker_main
|
|
from pyspark.serializers import read_int, write_int
|
|
|
|
|
|
def compute_real_exit_code(exit_code):
|
|
# SystemExit's code can be integer or string, but os._exit only accepts integers
|
|
if isinstance(exit_code, numbers.Integral):
|
|
return exit_code
|
|
else:
|
|
return 1
|
|
|
|
|
|
def worker(sock):
|
|
"""
|
|
Called by a worker process after the fork().
|
|
"""
|
|
signal.signal(SIGHUP, SIG_DFL)
|
|
signal.signal(SIGCHLD, SIG_DFL)
|
|
signal.signal(SIGTERM, SIG_DFL)
|
|
# restore the handler for SIGINT,
|
|
# it's useful for debugging (show the stacktrace before exit)
|
|
signal.signal(SIGINT, signal.default_int_handler)
|
|
|
|
# Read the socket using fdopen instead of socket.makefile() because the latter
|
|
# seems to be very slow; note that we need to dup() the file descriptor because
|
|
# otherwise writes also cause a seek that makes us miss data on the read side.
|
|
infile = os.fdopen(os.dup(sock.fileno()), "rb", 65536)
|
|
outfile = os.fdopen(os.dup(sock.fileno()), "wb", 65536)
|
|
exit_code = 0
|
|
try:
|
|
worker_main(infile, outfile)
|
|
except SystemExit as exc:
|
|
exit_code = compute_real_exit_code(exc.code)
|
|
finally:
|
|
try:
|
|
outfile.flush()
|
|
except Exception:
|
|
pass
|
|
return exit_code
|
|
|
|
|
|
def manager():
|
|
# Create a new process group to corral our children
|
|
os.setpgid(0, 0)
|
|
|
|
# Create a listening socket on the AF_INET loopback interface
|
|
listen_sock = socket.socket(AF_INET, SOCK_STREAM)
|
|
listen_sock.bind(('127.0.0.1', 0))
|
|
listen_sock.listen(max(1024, SOMAXCONN))
|
|
listen_host, listen_port = listen_sock.getsockname()
|
|
|
|
# re-open stdin/stdout in 'wb' mode
|
|
stdin_bin = os.fdopen(sys.stdin.fileno(), 'rb', 4)
|
|
stdout_bin = os.fdopen(sys.stdout.fileno(), 'wb', 4)
|
|
write_int(listen_port, stdout_bin)
|
|
stdout_bin.flush()
|
|
|
|
def shutdown(code):
|
|
signal.signal(SIGTERM, SIG_DFL)
|
|
# Send SIGHUP to notify workers of shutdown
|
|
os.kill(0, SIGHUP)
|
|
exit(code)
|
|
|
|
def handle_sigterm(*args):
|
|
shutdown(1)
|
|
signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM
|
|
signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP
|
|
signal.signal(SIGCHLD, SIG_IGN)
|
|
|
|
reuse = os.environ.get("SPARK_REUSE_WORKER")
|
|
|
|
# Initialization complete
|
|
try:
|
|
while True:
|
|
try:
|
|
ready_fds = select.select([0, listen_sock], [], [], 1)[0]
|
|
except select.error as ex:
|
|
if ex[0] == EINTR:
|
|
continue
|
|
else:
|
|
raise
|
|
|
|
if 0 in ready_fds:
|
|
try:
|
|
worker_pid = read_int(stdin_bin)
|
|
except EOFError:
|
|
# Spark told us to exit by closing stdin
|
|
shutdown(0)
|
|
try:
|
|
os.kill(worker_pid, signal.SIGKILL)
|
|
except OSError:
|
|
pass # process already died
|
|
|
|
if listen_sock in ready_fds:
|
|
try:
|
|
sock, _ = listen_sock.accept()
|
|
except OSError as e:
|
|
if e.errno == EINTR:
|
|
continue
|
|
raise
|
|
|
|
# Launch a worker process
|
|
try:
|
|
pid = os.fork()
|
|
except OSError as e:
|
|
if e.errno in (EAGAIN, EINTR):
|
|
time.sleep(1)
|
|
pid = os.fork() # error here will shutdown daemon
|
|
else:
|
|
outfile = sock.makefile(mode='wb')
|
|
write_int(e.errno, outfile) # Signal that the fork failed
|
|
outfile.flush()
|
|
outfile.close()
|
|
sock.close()
|
|
continue
|
|
|
|
if pid == 0:
|
|
# in child process
|
|
listen_sock.close()
|
|
try:
|
|
# Acknowledge that the fork was successful
|
|
outfile = sock.makefile(mode="wb")
|
|
write_int(os.getpid(), outfile)
|
|
outfile.flush()
|
|
outfile.close()
|
|
while True:
|
|
code = worker(sock)
|
|
if not reuse or code:
|
|
# wait for closing
|
|
try:
|
|
while sock.recv(1024):
|
|
pass
|
|
except Exception:
|
|
pass
|
|
break
|
|
gc.collect()
|
|
except:
|
|
traceback.print_exc()
|
|
os._exit(1)
|
|
else:
|
|
os._exit(0)
|
|
else:
|
|
sock.close()
|
|
|
|
finally:
|
|
shutdown(1)
|
|
|
|
|
|
if __name__ == '__main__':
|
|
manager()
|