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.
192 lines
6.2 KiB
Python
192 lines
6.2 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.
|
|
#
|
|
|
|
"""
|
|
>>> from pyspark.conf import SparkConf
|
|
>>> from pyspark.context import SparkContext
|
|
>>> conf = SparkConf()
|
|
>>> conf.setMaster("local").setAppName("My app")
|
|
<pyspark.conf.SparkConf object at ...>
|
|
>>> conf.get("spark.master")
|
|
u'local'
|
|
>>> conf.get("spark.app.name")
|
|
u'My app'
|
|
>>> sc = SparkContext(conf=conf)
|
|
>>> sc.master
|
|
u'local'
|
|
>>> sc.appName
|
|
u'My app'
|
|
>>> sc.sparkHome is None
|
|
True
|
|
|
|
>>> conf = SparkConf(loadDefaults=False)
|
|
>>> conf.setSparkHome("/path")
|
|
<pyspark.conf.SparkConf object at ...>
|
|
>>> conf.get("spark.home")
|
|
u'/path'
|
|
>>> conf.setExecutorEnv("VAR1", "value1")
|
|
<pyspark.conf.SparkConf object at ...>
|
|
>>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")])
|
|
<pyspark.conf.SparkConf object at ...>
|
|
>>> conf.get("spark.executorEnv.VAR1")
|
|
u'value1'
|
|
>>> print(conf.toDebugString())
|
|
spark.executorEnv.VAR1=value1
|
|
spark.executorEnv.VAR3=value3
|
|
spark.executorEnv.VAR4=value4
|
|
spark.home=/path
|
|
>>> sorted(conf.getAll(), key=lambda p: p[0])
|
|
[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), \
|
|
(u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')]
|
|
"""
|
|
|
|
__all__ = ['SparkConf']
|
|
|
|
import sys
|
|
import re
|
|
|
|
if sys.version > '3':
|
|
unicode = str
|
|
__doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__)
|
|
|
|
|
|
class SparkConf(object):
|
|
|
|
"""
|
|
Configuration for a Spark application. Used to set various Spark
|
|
parameters as key-value pairs.
|
|
|
|
Most of the time, you would create a SparkConf object with
|
|
C{SparkConf()}, which will load values from C{spark.*} Java system
|
|
properties as well. In this case, any parameters you set directly on
|
|
the C{SparkConf} object take priority over system properties.
|
|
|
|
For unit tests, you can also call C{SparkConf(false)} to skip
|
|
loading external settings and get the same configuration no matter
|
|
what the system properties are.
|
|
|
|
All setter methods in this class support chaining. For example,
|
|
you can write C{conf.setMaster("local").setAppName("My app")}.
|
|
|
|
Note that once a SparkConf object is passed to Spark, it is cloned
|
|
and can no longer be modified by the user.
|
|
"""
|
|
|
|
def __init__(self, loadDefaults=True, _jvm=None, _jconf=None):
|
|
"""
|
|
Create a new Spark configuration.
|
|
|
|
:param loadDefaults: whether to load values from Java system
|
|
properties (True by default)
|
|
:param _jvm: internal parameter used to pass a handle to the
|
|
Java VM; does not need to be set by users
|
|
:param _jconf: Optionally pass in an existing SparkConf handle
|
|
to use its parameters
|
|
"""
|
|
if _jconf:
|
|
self._jconf = _jconf
|
|
else:
|
|
from pyspark.context import SparkContext
|
|
SparkContext._ensure_initialized()
|
|
_jvm = _jvm or SparkContext._jvm
|
|
self._jconf = _jvm.SparkConf(loadDefaults)
|
|
|
|
def set(self, key, value):
|
|
"""Set a configuration property."""
|
|
self._jconf.set(key, unicode(value))
|
|
return self
|
|
|
|
def setIfMissing(self, key, value):
|
|
"""Set a configuration property, if not already set."""
|
|
if self.get(key) is None:
|
|
self.set(key, value)
|
|
return self
|
|
|
|
def setMaster(self, value):
|
|
"""Set master URL to connect to."""
|
|
self._jconf.setMaster(value)
|
|
return self
|
|
|
|
def setAppName(self, value):
|
|
"""Set application name."""
|
|
self._jconf.setAppName(value)
|
|
return self
|
|
|
|
def setSparkHome(self, value):
|
|
"""Set path where Spark is installed on worker nodes."""
|
|
self._jconf.setSparkHome(value)
|
|
return self
|
|
|
|
def setExecutorEnv(self, key=None, value=None, pairs=None):
|
|
"""Set an environment variable to be passed to executors."""
|
|
if (key is not None and pairs is not None) or (key is None and pairs is None):
|
|
raise Exception("Either pass one key-value pair or a list of pairs")
|
|
elif key is not None:
|
|
self._jconf.setExecutorEnv(key, value)
|
|
elif pairs is not None:
|
|
for (k, v) in pairs:
|
|
self._jconf.setExecutorEnv(k, v)
|
|
return self
|
|
|
|
def setAll(self, pairs):
|
|
"""
|
|
Set multiple parameters, passed as a list of key-value pairs.
|
|
|
|
:param pairs: list of key-value pairs to set
|
|
"""
|
|
for (k, v) in pairs:
|
|
self._jconf.set(k, v)
|
|
return self
|
|
|
|
def get(self, key, defaultValue=None):
|
|
"""Get the configured value for some key, or return a default otherwise."""
|
|
if defaultValue is None: # Py4J doesn't call the right get() if we pass None
|
|
if not self._jconf.contains(key):
|
|
return None
|
|
return self._jconf.get(key)
|
|
else:
|
|
return self._jconf.get(key, defaultValue)
|
|
|
|
def getAll(self):
|
|
"""Get all values as a list of key-value pairs."""
|
|
pairs = []
|
|
for elem in self._jconf.getAll():
|
|
pairs.append((elem._1(), elem._2()))
|
|
return pairs
|
|
|
|
def contains(self, key):
|
|
"""Does this configuration contain a given key?"""
|
|
return self._jconf.contains(key)
|
|
|
|
def toDebugString(self):
|
|
"""
|
|
Returns a printable version of the configuration, as a list of
|
|
key=value pairs, one per line.
|
|
"""
|
|
return self._jconf.toDebugString()
|
|
|
|
|
|
def _test():
|
|
import doctest
|
|
(failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS)
|
|
if failure_count:
|
|
exit(-1)
|
|
|
|
|
|
if __name__ == "__main__":
|
|
_test()
|