Skip to content

Commit

Permalink
[SPARK-4897] [PySpark] Python 3 support
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
Davies Liu authored and JoshRosen committed Apr 16, 2015
1 parent 55f553a commit 04e44b3
Show file tree
Hide file tree
Showing 91 changed files with 1,398 additions and 1,396 deletions.
1 change: 1 addition & 0 deletions bin/pyspark
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py"
if [[ -n "$SPARK_TESTING" ]]; then
unset YARN_CONF_DIR
unset HADOOP_CONF_DIR
export PYTHONHASHSEED=0
if [[ -n "$PYSPARK_DOC_TEST" ]]; then
exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1
else
Expand Down
3 changes: 3 additions & 0 deletions bin/spark-submit
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@

SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"

# disable randomized hash for string in Python 3.3+
export PYTHONHASHSEED=0

# Only define a usage function if an upstream script hasn't done so.
if ! type -t usage >/dev/null 2>&1; then
usage() {
Expand Down
3 changes: 3 additions & 0 deletions bin/spark-submit2.cmd
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,9 @@ rem
rem This is the entry point for running Spark submit. To avoid polluting the
rem environment, it just launches a new cmd to do the real work.

rem disable randomized hash for string in Python 3.3+
set PYTHONHASHSEED=0

set CLASS=org.apache.spark.deploy.SparkSubmit
call %~dp0spark-class2.cmd %CLASS% %*
set SPARK_ERROR_LEVEL=%ERRORLEVEL%
Expand Down
2 changes: 2 additions & 0 deletions dev/run-tests
Original file line number Diff line number Diff line change
Expand Up @@ -235,6 +235,8 @@ echo "========================================================================="

CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS

# add path for python 3 in jenkins
export PATH="${PATH}:/home/anaonda/envs/py3k/bin"
./python/run-tests

echo ""
Expand Down
2 changes: 1 addition & 1 deletion dev/run-tests-jenkins
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}"
# GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :(
SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}"

TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout
TESTS_TIMEOUT="150m" # format: http://linux.die.net/man/1/timeout

# Array to capture all tests to run on the pull request. These tests are held under the
#+ dev/tests/ directory.
Expand Down
262 changes: 131 additions & 131 deletions ec2/spark_ec2.py

Large diffs are not rendered by default.

15 changes: 8 additions & 7 deletions examples/src/main/python/als.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,8 @@
This example requires numpy (http://www.numpy.org/)
"""
from os.path import realpath
from __future__ import print_function

import sys

import numpy as np
Expand Down Expand Up @@ -57,9 +58,9 @@ def update(i, vec, mat, ratings):
Usage: als [M] [U] [F] [iterations] [partitions]"
"""

print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an
print("""WARN: This is a naive implementation of ALS and is given as an
example. Please use the ALS method found in pyspark.mllib.recommendation for more
conventional use."""
conventional use.""", file=sys.stderr)

sc = SparkContext(appName="PythonALS")
M = int(sys.argv[1]) if len(sys.argv) > 1 else 100
Expand All @@ -68,8 +69,8 @@ def update(i, vec, mat, ratings):
ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5
partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2

print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \
(M, U, F, ITERATIONS, partitions)
print("Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" %
(M, U, F, ITERATIONS, partitions))

R = matrix(rand(M, F)) * matrix(rand(U, F).T)
ms = matrix(rand(M, F))
Expand All @@ -95,7 +96,7 @@ def update(i, vec, mat, ratings):
usb = sc.broadcast(us)

error = rmse(R, ms, us)
print "Iteration %d:" % i
print "\nRMSE: %5.4f\n" % error
print("Iteration %d:" % i)
print("\nRMSE: %5.4f\n" % error)

sc.stop()
9 changes: 6 additions & 3 deletions examples/src/main/python/avro_inputformat.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,12 @@
# limitations under the License.
#

from __future__ import print_function

import sys

from pyspark import SparkContext
from functools import reduce

"""
Read data file users.avro in local Spark distro:
Expand Down Expand Up @@ -49,15 +52,15 @@
"""
if __name__ == "__main__":
if len(sys.argv) != 2 and len(sys.argv) != 3:
print >> sys.stderr, """
print("""
Usage: avro_inputformat <data_file> [reader_schema_file]
Run with example jar:
./bin/spark-submit --driver-class-path /path/to/example/jar \
/path/to/examples/avro_inputformat.py <data_file> [reader_schema_file]
Assumes you have Avro data stored in <data_file>. Reader schema can be optionally specified
in [reader_schema_file].
"""
""", file=sys.stderr)
exit(-1)

path = sys.argv[1]
Expand All @@ -77,6 +80,6 @@
conf=conf)
output = avro_rdd.map(lambda x: x[0]).collect()
for k in output:
print k
print(k)

sc.stop()
8 changes: 5 additions & 3 deletions examples/src/main/python/cassandra_inputformat.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

import sys

from pyspark import SparkContext
Expand Down Expand Up @@ -47,14 +49,14 @@
"""
if __name__ == "__main__":
if len(sys.argv) != 4:
print >> sys.stderr, """
print("""
Usage: cassandra_inputformat <host> <keyspace> <cf>
Run with example jar:
./bin/spark-submit --driver-class-path /path/to/example/jar \
/path/to/examples/cassandra_inputformat.py <host> <keyspace> <cf>
Assumes you have some data in Cassandra already, running on <host>, in <keyspace> and <cf>
"""
""", file=sys.stderr)
exit(-1)

host = sys.argv[1]
Expand All @@ -77,6 +79,6 @@
conf=conf)
output = cass_rdd.collect()
for (k, v) in output:
print (k, v)
print((k, v))

sc.stop()
6 changes: 4 additions & 2 deletions examples/src/main/python/cassandra_outputformat.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

import sys

from pyspark import SparkContext
Expand Down Expand Up @@ -46,7 +48,7 @@
"""
if __name__ == "__main__":
if len(sys.argv) != 7:
print >> sys.stderr, """
print("""
Usage: cassandra_outputformat <host> <keyspace> <cf> <user_id> <fname> <lname>
Run with example jar:
Expand All @@ -60,7 +62,7 @@
... fname text,
... lname text
... );
"""
""", file=sys.stderr)
exit(-1)

host = sys.argv[1]
Expand Down
8 changes: 5 additions & 3 deletions examples/src/main/python/hbase_inputformat.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

import sys

from pyspark import SparkContext
Expand Down Expand Up @@ -47,14 +49,14 @@
"""
if __name__ == "__main__":
if len(sys.argv) != 3:
print >> sys.stderr, """
print("""
Usage: hbase_inputformat <host> <table>
Run with example jar:
./bin/spark-submit --driver-class-path /path/to/example/jar \
/path/to/examples/hbase_inputformat.py <host> <table>
Assumes you have some data in HBase already, running on <host>, in <table>
"""
""", file=sys.stderr)
exit(-1)

host = sys.argv[1]
Expand All @@ -74,6 +76,6 @@
conf=conf)
output = hbase_rdd.collect()
for (k, v) in output:
print (k, v)
print((k, v))

sc.stop()
6 changes: 4 additions & 2 deletions examples/src/main/python/hbase_outputformat.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

import sys

from pyspark import SparkContext
Expand All @@ -40,15 +42,15 @@
"""
if __name__ == "__main__":
if len(sys.argv) != 7:
print >> sys.stderr, """
print("""
Usage: hbase_outputformat <host> <table> <row> <family> <qualifier> <value>
Run with example jar:
./bin/spark-submit --driver-class-path /path/to/example/jar \
/path/to/examples/hbase_outputformat.py <args>
Assumes you have created <table> with column family <family> in HBase
running on <host> already
"""
""", file=sys.stderr)
exit(-1)

host = sys.argv[1]
Expand Down
11 changes: 6 additions & 5 deletions examples/src/main/python/kmeans.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
This example requires NumPy (http://www.numpy.org/).
"""
from __future__ import print_function

import sys

Expand All @@ -47,12 +48,12 @@ def closestPoint(p, centers):
if __name__ == "__main__":

if len(sys.argv) != 4:
print >> sys.stderr, "Usage: kmeans <file> <k> <convergeDist>"
print("Usage: kmeans <file> <k> <convergeDist>", file=sys.stderr)
exit(-1)

print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given
print("""WARN: This is a naive implementation of KMeans Clustering and is given
as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on
how to use MLlib's KMeans implementation."""
how to use MLlib's KMeans implementation.""", file=sys.stderr)

sc = SparkContext(appName="PythonKMeans")
lines = sc.textFile(sys.argv[1])
Expand All @@ -69,13 +70,13 @@ def closestPoint(p, centers):
pointStats = closest.reduceByKey(
lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2))
newPoints = pointStats.map(
lambda (x, (y, z)): (x, y / z)).collect()
lambda xy: (xy[0], xy[1][0] / xy[1][1])).collect()

tempDist = sum(np.sum((kPoints[x] - y) ** 2) for (x, y) in newPoints)

for (x, y) in newPoints:
kPoints[x] = y

print "Final centers: " + str(kPoints)
print("Final centers: " + str(kPoints))

sc.stop()
20 changes: 9 additions & 11 deletions examples/src/main/python/logistic_regression.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,8 @@
In practice, one may prefer to use the LogisticRegression algorithm in
MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py.
"""
from __future__ import print_function

from collections import namedtuple
from math import exp
from os.path import realpath
import sys

import numpy as np
Expand All @@ -42,27 +40,27 @@
def readPointBatch(iterator):
strs = list(iterator)
matrix = np.zeros((len(strs), D + 1))
for i in xrange(len(strs)):
matrix[i] = np.fromstring(strs[i].replace(',', ' '), dtype=np.float32, sep=' ')
for i, s in enumerate(strs):
matrix[i] = np.fromstring(s.replace(',', ' '), dtype=np.float32, sep=' ')
return [matrix]

if __name__ == "__main__":

if len(sys.argv) != 3:
print >> sys.stderr, "Usage: logistic_regression <file> <iterations>"
print("Usage: logistic_regression <file> <iterations>", file=sys.stderr)
exit(-1)

print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is
print("""WARN: This is a naive implementation of Logistic Regression and is
given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py
to see how MLlib's implementation is used."""
to see how MLlib's implementation is used.""", file=sys.stderr)

sc = SparkContext(appName="PythonLR")
points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache()
iterations = int(sys.argv[2])

# Initialize w to a random value
w = 2 * np.random.ranf(size=D) - 1
print "Initial w: " + str(w)
print("Initial w: " + str(w))

# Compute logistic regression gradient for a matrix of data points
def gradient(matrix, w):
Expand All @@ -76,9 +74,9 @@ def add(x, y):
return x

for i in range(iterations):
print "On iteration %i" % (i + 1)
print("On iteration %i" % (i + 1))
w -= points.map(lambda m: gradient(m, w)).reduce(add)

print "Final w: " + str(w)
print("Final w: " + str(w))

sc.stop()
20 changes: 11 additions & 9 deletions examples/src/main/python/ml/simple_text_classification_pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
# limitations under the License.
#

from __future__ import print_function

from pyspark import SparkContext
from pyspark.ml import Pipeline
from pyspark.ml.classification import LogisticRegression
Expand All @@ -37,10 +39,10 @@

# Prepare training documents, which are labeled.
LabeledDocument = Row("id", "text", "label")
training = sc.parallelize([(0L, "a b c d e spark", 1.0),
(1L, "b d", 0.0),
(2L, "spark f g h", 1.0),
(3L, "hadoop mapreduce", 0.0)]) \
training = sc.parallelize([(0, "a b c d e spark", 1.0),
(1, "b d", 0.0),
(2, "spark f g h", 1.0),
(3, "hadoop mapreduce", 0.0)]) \
.map(lambda x: LabeledDocument(*x)).toDF()

# Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr.
Expand All @@ -54,16 +56,16 @@

# Prepare test documents, which are unlabeled.
Document = Row("id", "text")
test = sc.parallelize([(4L, "spark i j k"),
(5L, "l m n"),
(6L, "mapreduce spark"),
(7L, "apache hadoop")]) \
test = sc.parallelize([(4, "spark i j k"),
(5, "l m n"),
(6, "mapreduce spark"),
(7, "apache hadoop")]) \
.map(lambda x: Document(*x)).toDF()

# Make predictions on test documents and print columns of interest.
prediction = model.transform(test)
selected = prediction.select("id", "text", "prediction")
for row in selected.collect():
print row
print(row)

sc.stop()
Loading

0 comments on commit 04e44b3

Please sign in to comment.