aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/worker.py
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-6216] [PYSPARK] check python version of worker with driverDavies Liu2015-05-181-5/+7
| | | | | | | | | | | | This PR revert #5404, change to pass the version of python in driver into JVM, check it in worker before deserializing closure, then it can works with different major version of Python. Author: Davies Liu <davies@databricks.com> Closes #6203 from davies/py_version and squashes the following commits: b8fb76e [Davies Liu] fix test 6ce5096 [Davies Liu] use string for version 47c6278 [Davies Liu] check python version of worker with driver
* [SPARK-4897] [PySpark] Python 3 supportDavies Liu2015-04-161-6/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-6216] [PySpark] check the python version in workerDavies Liu2015-04-101-1/+5
| | | | | | | | | Author: Davies Liu <davies@databricks.com> Closes #5404 from davies/check_version and squashes the following commits: e559248 [Davies Liu] add tests ec33b5f [Davies Liu] check the python version in worker
* Revert "[SPARK-5363] [PySpark] check ending mark in non-block way"Josh Rosen2015-02-171-1/+0
| | | | This reverts commits ac6fe67e1d8bf01ee565f9cc09ad48d88a275829 and c06e42f2c1e5fcf123b466efd27ee4cb53bbed3f.
* [SPARK-5363] [PySpark] check ending mark in non-block wayDavies Liu2015-02-161-0/+1
| | | | | | | | | | | | | | | | | | | | There is chance of dead lock that the Python process is waiting for ending mark from JVM, but which is eaten by corrupted stream. This PR checks the ending mark from Python in non-block way, so it will not blocked by Python process. There is a small chance that the ending mark is sent by Python process but not available right now, then Python process will not be used. cc JoshRosen pwendell Author: Davies Liu <davies@databricks.com> Closes #4601 from davies/freeze and squashes the following commits: e15a8c3 [Davies Liu] update logging 890329c [Davies Liu] Merge branch 'freeze' of github.com:davies/spark into freeze 2bd2228 [Davies Liu] add more logging 656d544 [Davies Liu] Update PythonRDD.scala 05e1085 [Davies Liu] check ending mark in non-block way
* [SPARK-4387][PySpark] Refactoring python profiling code to make it extensibleYandu Oppacher2015-01-281-9/+3
| | | | | | | | | | | | | | | | | | | | | | | | This PR is based on #3255 , fix conflicts and code style. Closes #3255. Author: Yandu Oppacher <yandu.oppacher@jadedpixel.com> Author: Davies Liu <davies@databricks.com> Closes #3901 from davies/refactor-python-profile-code and squashes the following commits: b4a9306 [Davies Liu] fix tests 4b79ce8 [Davies Liu] add docstring for profiler_cls 2700e47 [Davies Liu] use BasicProfiler as default 349e341 [Davies Liu] more refactor 6a5d4df [Davies Liu] refactor and fix tests 31bf6b6 [Davies Liu] fix code style 0864b5d [Yandu Oppacher] Remove unused method 76a6c37 [Yandu Oppacher] Added a profile collector to accumulate the profilers per stage 9eefc36 [Yandu Oppacher] Fix doc 9ace076 [Yandu Oppacher] Refactor of profiler, and moved tests around 8739aff [Yandu Oppacher] Code review fixes 9bda3ec [Yandu Oppacher] Refactor profiler code
* [SPARK-4548] []SPARK-4517] improve performance of python broadcastDavies Liu2014-11-241-7/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Re-implement the Python broadcast using file: 1) serialize the python object using cPickle, write into disks. 2) Create a wrapper in JVM (for the dumped file), it read data from during serialization 3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors 4) During deserialization, writing the data into disk. 5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access. It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor). Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 25.20 | 9.33 | 170.13% | python-broadcast-w-set | 4.13 | 4.50 | -8.35% | Testing with 100 tasks (16 CPUs): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 38.16 | 8.40 | 353.98% python-broadcast-w-set | 23.29 | 9.59 | 142.80% Author: Davies Liu <davies@databricks.com> Closes #3417 from davies/pybroadcast and squashes the following commits: 50a58e0 [Davies Liu] address comments b98de1d [Davies Liu] disable gc while unpickle e5ee6b9 [Davies Liu] support large string 09303b8 [Davies Liu] read all data into memory dde02dd [Davies Liu] improve performance of python broadcast
* [SPARK-3721] [PySpark] broadcast objects larger than 2GDavies Liu2014-11-181-3/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | This patch will bring support for broadcasting objects larger than 2G. pickle, zlib, FrameSerializer and Array[Byte] all can not support objects larger than 2G, so this patch introduce LargeObjectSerializer to serialize broadcast objects, the object will be serialized and compressed into small chunks, it also change the type of Broadcast[Array[Byte]]] into Broadcast[Array[Array[Byte]]]]. Testing for support broadcast objects larger than 2G is slow and memory hungry, so this is tested manually, could be added into SparkPerf. Author: Davies Liu <davies@databricks.com> Author: Davies Liu <davies.liu@gmail.com> Closes #2659 from davies/huge and squashes the following commits: 7b57a14 [Davies Liu] add more tests for broadcast 28acff9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge a2f6a02 [Davies Liu] bug fix 4820613 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge 5875c73 [Davies Liu] address comments 10a349b [Davies Liu] address comments 0c33016 [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge 6182c8f [Davies Liu] Merge branch 'master' into huge d94b68f [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge 2514848 [Davies Liu] address comments fda395b [Davies Liu] Merge branch 'master' of github.com:apache/spark into huge 1c2d928 [Davies Liu] fix scala style 091b107 [Davies Liu] broadcast objects larger than 2G
* [SPARK-3993] [PySpark] fix bug while reuse worker after take()Davies Liu2014-10-231-2/+9
| | | | | | | | | | | | | | After take(), maybe there are some garbage left in the socket, then next task assigned to this worker will hang because of corrupted data. We should make sure the socket is clean before reuse it, write END_OF_STREAM at the end, and check it after read out all result from python. Author: Davies Liu <davies.liu@gmail.com> Author: Davies Liu <davies@databricks.com> Closes #2838 from davies/fix_reuse and squashes the following commits: 8872914 [Davies Liu] fix tests 660875b [Davies Liu] fix bug while reuse worker after take()
* [SPARK-3478] [PySpark] Profile the Python tasksDavies Liu2014-09-301-3/+16
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD<id=3> ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>) 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>) 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD<id=1> ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>) 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" This is bugfix of #2351 cc JoshRosen Author: Davies Liu <davies.liu@gmail.com> Closes #2556 from davies/profiler and squashes the following commits: e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 858e74c [Davies Liu] compatitable with python 2.6 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python
* Revert "[SPARK-3478] [PySpark] Profile the Python tasks"Josh Rosen2014-09-261-16/+3
| | | | This reverts commit 1aa549ba9839565274a12c52fa1075b424f138a6.
* [SPARK-3478] [PySpark] Profile the Python tasksDavies Liu2014-09-261-3/+16
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD<id=3> ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16(<lambda>) 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863(<lambda>) 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD<id=1> ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852(<genexpr>) 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" Author: Davies Liu <davies.liu@gmail.com> Closes #2351 from davies/profiler and squashes the following commits: 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python
* [SPARK-3634] [PySpark] User's module should take precedence over system modulesDavies Liu2014-09-241-2/+9
| | | | | | | | | | | | | | | Python modules added through addPyFile should take precedence over system modules. This patch put the path for user added module in the front of sys.path (just after ''). Author: Davies Liu <davies.liu@gmail.com> Closes #2492 from davies/path and squashes the following commits: 4a2af78 [Davies Liu] fix tests f7ff4da [Davies Liu] ad license header 6b0002f [Davies Liu] add tests c16c392 [Davies Liu] put addPyFile in front of sys.path
* [SPARK-3554] [PySpark] use broadcast automatically for large closureDavies Liu2014-09-181-1/+3
| | | | | | | | | | | Py4j can not handle large string efficiently, so we should use broadcast for large closure automatically. (Broadcast use local filesystem to pass through data). Author: Davies Liu <davies.liu@gmail.com> Closes #2417 from davies/command and squashes the following commits: fbf4e97 [Davies Liu] bugfix aefd508 [Davies Liu] use broadcast automatically for large closure
* [SPARK-3463] [PySpark] aggregate and show spilled bytes in PythonDavies Liu2014-09-131-4/+10
| | | | | | | | | | | | | | | | | | Aggregate the number of bytes spilled into disks during aggregation or sorting, show them in Web UI. ![spilled](https://cloud.githubusercontent.com/assets/40902/4209758/4b995562-386d-11e4-97c1-8e838ee1d4e3.png) This patch is blocked by SPARK-3465. (It includes a fix for that). Author: Davies Liu <davies.liu@gmail.com> Closes #2336 from davies/metrics and squashes the following commits: e37df38 [Davies Liu] remove outdated comments 1245eb7 [Davies Liu] remove the temporary fix ebd2f43 [Davies Liu] Merge branch 'master' into metrics 7e4ad04 [Davies Liu] Merge branch 'master' into metrics fbe9029 [Davies Liu] show spilled bytes in Python in web ui
* [SPARK-3030] [PySpark] Reuse Python workerDavies Liu2014-09-131-2/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu <davies.liu@gmail.com> Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker
* [SPARK-3114] [PySpark] Fix Python UDFs in Spark SQL.Josh Rosen2014-08-181-1/+1
| | | | | | | | | | | | | | | | This fixes SPARK-3114, an issue where we inadvertently broke Python UDFs in Spark SQL. This PR modifiers the test runner script to always run the PySpark SQL tests, irrespective of whether SparkSQL itself has been modified. It also includes Davies' fix for the bug. Closes #2026. Author: Josh Rosen <joshrosen@apache.org> Author: Davies Liu <davies.liu@gmail.com> Closes #2027 from JoshRosen/pyspark-sql-fix and squashes the following commits: 9af2708 [Davies Liu] bugfix: disable compression of command 0d8d3a4 [Josh Rosen] Always run Python Spark SQL tests.
* [SPARK-1065] [PySpark] improve supporting for large broadcastDavies Liu2014-08-161-3/+5
| | | | | | | | | | | | | | | | | | Passing large object by py4j is very slow (cost much memory), so pass broadcast objects via files (similar to parallelize()). Add an option to keep object in driver (it's False by default) to save memory in driver. Author: Davies Liu <davies.liu@gmail.com> Closes #1912 from davies/broadcast and squashes the following commits: e06df4a [Davies Liu] load broadcast from disk in driver automatically db3f232 [Davies Liu] fix serialization of accumulator 631a827 [Davies Liu] Merge branch 'master' into broadcast c7baa8c [Davies Liu] compress serrialized broadcast and command 9a7161f [Davies Liu] fix doc tests e93cf4b [Davies Liu] address comments: add test 6226189 [Davies Liu] improve large broadcast
* [SPARK-2580] [PySpark] keep silent in worker if JVM close the socketDavies Liu2014-07-291-8/+13
| | | | | | | | | | | | | During rdd.take(n), JVM will close the socket if it had got enough data, the Python worker should keep silent in this case. In the same time, the worker should not print the trackback into stderr if it send the traceback to JVM successfully. Author: Davies Liu <davies.liu@gmail.com> Closes #1625 from davies/error and squashes the following commits: 4fbcc6d [Davies Liu] disable log4j during testing when exception is expected. cc14202 [Davies Liu] keep silent in worker if JVM close the socket
* [SPARK-2470] PEP8 fixes to PySparkNicholas Chammas2014-07-211-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Author: nchammas <nicholas.chammas@gmail.com> Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master
* Add Python includes to path before depickling broadcast valuesBouke van der Bijl2014-05-101-7/+7
| | | | | | | | | | | | This fixes https://issues.apache.org/jira/browse/SPARK-1731 by adding the Python includes to the PYTHONPATH before depickling the broadcast values @airhorns Author: Bouke van der Bijl <boukevanderbijl@gmail.com> Closes #656 from bouk/python-includes-before-broadcast and squashes the following commits: 7b0dfe4 [Bouke van der Bijl] Add Python includes to path before depickling broadcast values
* SPARK-1115: Catch depickling errorsBouke van der Bijl2014-02-261-24/+24
| | | | | | | | | | | | | This surroungs the complete worker code in a try/except block so we catch any error that arrives. An example would be the depickling failing for some reason @JoshRosen Author: Bouke van der Bijl <boukevanderbijl@gmail.com> Closes #644 from bouk/catch-depickling-errors and squashes the following commits: f0f67cc [Bouke van der Bijl] Lol indentation 0e4d504 [Bouke van der Bijl] Surround the complete python worker with the try block
* Fixed minor typo in worker.pyjyotiska2014-02-221-1/+1
| | | | | | | | | | Fixed minor typo in worker.py Author: jyotiska <jyotiska123@gmail.com> Closes #630 from jyotiska/pyspark_code and squashes the following commits: ee44201 [jyotiska] typo fixed in worker.py
* Switch from MUTF8 to UTF8 in PySpark serializers.Josh Rosen2014-01-281-4/+4
| | | | | | | | | This fixes SPARK-1043, a bug introduced in 0.9.0 where PySpark couldn't serialize strings > 64kB. This fix was written by @tyro89 and @bouk in #512. This commit squashes and rebases their pull request in order to fix some merge conflicts.
* Log Python exceptions to stderr as wellMatei Zaharia2014-01-121-0/+4
| | | | | | This helps in case the exception happened while serializing a record to be sent to Java, leaving the stream to Java in an inconsistent state where PythonRDD won't be able to read the error.
* FramedSerializer: _dumps => dumps, _loads => loads.Josh Rosen2013-11-101-2/+2
|
* Send PySpark commands as bytes insetad of strings.Josh Rosen2013-11-101-10/+2
|
* Add custom serializer support to PySpark.Josh Rosen2013-11-101-22/+19
| | | | | | | | | For now, this only adds MarshalSerializer, but it lays the groundwork for other supporting custom serializers. Many of these mechanisms can also be used to support deserialization of different data formats sent by Java, such as data encoded by MsgPack. This also fixes a bug in SparkContext.union().
* Remove Pickle-wrapping of Java objects in PySpark.Josh Rosen2013-11-031-5/+9
| | | | | | If we support custom serializers, the Python worker will know what type of input to expect, so we won't need to wrap Tuple2 and Strings into pickled tuples and strings.
* Replace magic lengths with constants in PySpark.Josh Rosen2013-11-031-6/+7
| | | | | | Write the length of the accumulators section up-front rather than terminating it with a negative length. I find this easier to read.
* Allow PySpark to launch worker.py directly on WindowsMatei Zaharia2013-09-011-4/+7
|
* Implementing SPARK-878 for PySpark: adding zip and egg files to context and ↵Andre Schumacher2013-08-161-1/+12
| | | | passing it down to workers which add these to their sys.path
* Add Apache license headers and LICENSE and NOTICE filesMatei Zaharia2013-07-161-0/+17
|
* Fix reporting of PySpark exceptionsJey Kottalam2013-06-211-1/+1
|
* Add tests and fixes for Python daemon shutdownJey Kottalam2013-06-211-0/+2
|
* Prefork Python worker processesJey Kottalam2013-06-211-32/+29
|
* Add Python timing instrumentationJey Kottalam2013-06-211-1/+15
|
* Fix stdout redirection in PySpark.Josh Rosen2013-02-011-2/+3
|
* SPARK-673: Capture and re-throw Python exceptionsPatrick Wendell2013-01-311-2/+8
| | | | | This patch alters the Python <-> executor protocol to pass on exception data when they occur in user Python code.
* Allow PySpark's SparkFiles to be used from driverJosh Rosen2013-01-231-0/+1
| | | | Fix minor documentation formatting issues.
* Fix sys.path bug in PySpark SparkContext.addPyFileJosh Rosen2013-01-221-0/+1
|
* Don't download files to master's working directory.Josh Rosen2013-01-211-0/+3
| | | | | | | This should avoid exceptions caused by existing files with different contents. I also removed some unused code.
* Added accumulators to PySparkMatei Zaharia2013-01-201-1/+6
|
* Add mapPartitionsWithSplit() to PySpark.Josh Rosen2013-01-081-1/+3
|
* Rename top-level 'pyspark' directory to 'python'Josh Rosen2013-01-011-0/+40