## What changes were proposed in this pull request?
This change uses Arrow to optimize the creation of a Spark DataFrame from a Pandas DataFrame. The input df is sliced according to the default parallelism. The optimization is enabled with the existing conf "spark.sql.execution.arrow.enabled" and is disabled by default.
## How was this patch tested?
Added new unit test to create DataFrame with and without the optimization enabled, then compare results.
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>
Closes#19459 from BryanCutler/arrow-createDataFrame-from_pandas-SPARK-20791.
## What changes were proposed in this pull request?
This PR proposes to improve error message from:
```
>>> sc.show_profiles()
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1000, in show_profiles
self.profiler_collector.show_profiles()
AttributeError: 'NoneType' object has no attribute 'show_profiles'
>>> sc.dump_profiles("/tmp/abc")
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1005, in dump_profiles
self.profiler_collector.dump_profiles(path)
AttributeError: 'NoneType' object has no attribute 'dump_profiles'
```
to
```
>>> sc.show_profiles()
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1003, in show_profiles
raise RuntimeError("'spark.python.profile' configuration must be set "
RuntimeError: 'spark.python.profile' configuration must be set to 'true' to enable Python profile.
>>> sc.dump_profiles("/tmp/abc")
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File ".../spark/python/pyspark/context.py", line 1012, in dump_profiles
raise RuntimeError("'spark.python.profile' configuration must be set "
RuntimeError: 'spark.python.profile' configuration must be set to 'true' to enable Python profile.
```
## How was this patch tested?
Unit tests added in `python/pyspark/tests.py` and manual tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#19260 from HyukjinKwon/profile-errors.
## What changes were proposed in this pull request?
When using PySpark broadcast variables in a multi-threaded environment, `SparkContext._pickled_broadcast_vars` becomes a shared resource. A race condition can occur when broadcast variables that are pickled from one thread get added to the shared ` _pickled_broadcast_vars` and become part of the python command from another thread. This PR introduces a thread-safe pickled registry using thread local storage so that when python command is pickled (causing the broadcast variable to be pickled and added to the registry) each thread will have their own view of the pickle registry to retrieve and clear the broadcast variables used.
## How was this patch tested?
Added a unit test that causes this race condition using another thread.
Author: Bryan Cutler <cutlerb@gmail.com>
Closes#18695 from BryanCutler/pyspark-bcast-threadsafe-SPARK-12717.
## What changes were proposed in this pull request?
After SPARK-12661, I guess we officially dropped Python 2.6 support. It looks there are few places missing this notes.
I grepped "Python 2.6" and "python 2.6" and the results were below:
```
./core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala: // Unpickle array.array generated by Python 2.6
./docs/index.md:Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0.
./docs/rdd-programming-guide.md:Spark {{site.SPARK_VERSION}} works with Python 2.6+ or Python 3.4+. It can use the standard CPython interpreter,
./docs/rdd-programming-guide.md:Note that support for Python 2.6 is deprecated as of Spark 2.0.0, and may be removed in Spark 2.2.0.
./python/pyspark/context.py: warnings.warn("Support for Python 2.6 is deprecated as of Spark 2.0.0")
./python/pyspark/ml/tests.py: sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
./python/pyspark/mllib/tests.py: sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
./python/pyspark/serializers.py: # On Python 2.6, we can't write bytearrays to streams, so we need to convert them
./python/pyspark/sql/tests.py: sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
./python/pyspark/streaming/tests.py: sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
./python/pyspark/tests.py: sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier')
./python/pyspark/tests.py: # NOTE: dict is used instead of collections.Counter for Python 2.6
./python/pyspark/tests.py: # NOTE: dict is used instead of collections.Counter for Python 2.6
```
This PR only proposes to change visible changes as below:
```
./docs/rdd-programming-guide.md:Spark {{site.SPARK_VERSION}} works with Python 2.6+ or Python 3.4+. It can use the standard CPython interpreter,
./docs/rdd-programming-guide.md:Note that support for Python 2.6 is deprecated as of Spark 2.0.0, and may be removed in Spark 2.2.0.
./python/pyspark/context.py: warnings.warn("Support for Python 2.6 is deprecated as of Spark 2.0.0")
```
This one is already correct:
```
./docs/index.md:Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0.
```
## How was this patch tested?
```bash
grep -r "Python 2.6" .
grep -r "python 2.6" .
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18682 from HyukjinKwon/minor-python.26.
## What changes were proposed in this pull request?
Extend setJobDescription to PySpark and JavaSpark APIs
SPARK-21125
## How was this patch tested?
Testing was done by running a local Spark shell on the built UI. I originally had added a unit test but the PySpark context cannot easily access the Scala Spark Context's private variable with the Job Description key so I omitted the test, due to the simplicity of this addition.
Also ran the existing tests.
# Misc
This contribution is my original work and that I license the work to the project under the project's open source license.
Author: sjarvie <sjarvie@uber.com>
Closes#18332 from sjarvie/add_python_set_job_description.
## What changes were proposed in this pull request?
self.environment will be propagated to executor. Should set PYTHONHASHSEED as long as the python version is greater than 3.3
## How was this patch tested?
Manually tested it.
Author: Jeff Zhang <zjffdu@apache.org>
Closes#11211 from zjffdu/SPARK-13330.
The code was failing to propagate the user conf in the case where the
JVM was already initialized, which happens when a user submits a
python script via spark-submit.
Tested with new unit test and by running a python script in a real cluster.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#16682 from vanzin/SPARK-19307.
## What changes were proposed in this pull request?
This PR fixes SparkContext broken state in which it may fall if spark driver get crashed or killed by OOM.
## How was this patch tested?
1. Start SparkContext;
2. Find Spark driver process and `kill -9` it;
3. Call `sc.stop()`;
4. Create new SparkContext after that;
Without this patch you will crash on step 3 and won't be able to do step 4 without manual reset private attibutes or IPython notebook / shell restart.
Author: Alexander Shorin <kxepal@apache.org>
Closes#15961 from kxepal/18523-make-spark-context-stop-more-reliable.
## What changes were proposed in this pull request?
Document that Java 7, Python 2.6, Scala 2.10, Hadoop < 2.6 are deprecated in Spark 2.1.0. This does not actually implement any of the change in SPARK-18138, just peppers the documentation with notices about it.
## How was this patch tested?
Doc build
Author: Sean Owen <sowen@cloudera.com>
Closes#15733 from srowen/SPARK-18138.
## What changes were proposed in this pull request?
The root cause that we would ignore SparkConf when launching JVM is that SparkConf require JVM to be created first. https://github.com/apache/spark/blob/master/python/pyspark/conf.py#L106
In this PR, I would defer the launching of JVM until SparkContext is created so that we can pass SparkConf to JVM correctly.
## How was this patch tested?
Use the example code in the description of SPARK-17387,
```
$ SPARK_HOME=$PWD PYTHONPATH=python:python/lib/py4j-0.10.3-src.zip python
Python 2.7.12 (default, Jul 1 2016, 15:12:24)
[GCC 5.4.0 20160609] on linux2
Type "help", "copyright", "credits" or "license" for more information.
>>> from pyspark import SparkContext
>>> from pyspark import SparkConf
>>> conf = SparkConf().set("spark.driver.memory", "4g")
>>> sc = SparkContext(conf=conf)
```
And verify the spark.driver.memory is correctly picked up.
```
...op/ -Xmx4g org.apache.spark.deploy.SparkSubmit --conf spark.driver.memory=4g pyspark-shell
```
Author: Jeff Zhang <zjffdu@apache.org>
Closes#14959 from zjffdu/SPARK-17387.
## What changes were proposed in this pull request?
Move the internals of the PySpark accumulator API from the old deprecated API on top of the new accumulator API.
## How was this patch tested?
The existing PySpark accumulator tests (both unit tests and doc tests at the start of accumulator.py).
Author: Holden Karau <holden@us.ibm.com>
Closes#14467 from holdenk/SPARK-16861-refactor-pyspark-accumulator-api.
## What changes were proposed in this pull request?
Users would like to add a directory as dependency in some cases, they can use ```SparkContext.addFile``` with argument ```recursive=true``` to recursively add all files under the directory by using Scala. But Python users can only add file not directory, we should also make it supported.
## How was this patch tested?
Unit test.
Author: Yanbo Liang <ybliang8@gmail.com>
Closes#15140 from yanboliang/spark-17585.
## What changes were proposed in this pull request?
The Scala version of `SparkContext` has a handy field called `uiWebUrl` that tells you which URL the SparkUI spawned by that instance lives at. This is often very useful because the value for `spark.ui.port` in the config is only a suggestion; if that port number is taken by another Spark instance on the same machine, Spark will just keep incrementing the port until it finds a free one. So, on a machine with a lot of running PySpark instances, you often have to start trying all of them one-by-one until you find your application name.
Scala users have a way around this with `uiWebUrl` but Java and Python users do not. This pull request fixes this in the most straightforward way possible, simply propagating this field through the `JavaSparkContext` and into pyspark through the Java gateway.
Please let me know if any additional documentation/testing is needed.
## How was this patch tested?
Existing tests were run to make sure there were no regressions, and a binary distribution was created and tested manually for the correct value of `sc.uiWebPort` in a variety of circumstances.
Author: Adrian Petrescu <apetresc@gmail.com>
Closes#15000 from apetresc/pyspark-uiweburl.
## What changes were proposed in this pull request?
This pull request removes the SparkContext.clearFiles() method from the PySpark API as the method was removed from the Scala API in 8ce645d4ee. Using that method in PySpark leads to an exception as PySpark tries to call the non-existent method on the JVM side.
## How was this patch tested?
Existing tests (though none of them tested this particular method).
Author: Sami Jaktholm <sjakthol@outlook.com>
Closes#15081 from sjakthol/pyspark-sc-clearfiles.
## What changes were proposed in this pull request?
When we create a SparkSession at the Python side, it is possible that a SparkContext has been created. For this case, we need to set configs of the SparkSession builder to the Scala SparkContext's SparkConf (we need to do so because conf changes on a active Python SparkContext will not be propagated to the JVM side). Otherwise, we may create a wrong SparkSession (e.g. Hive support is not enabled even if enableHiveSupport is called).
## How was this patch tested?
New tests and manual tests.
Author: Yin Huai <yhuai@databricks.com>
Closes#13931 from yhuai/SPARK-16224.
## What changes were proposed in this pull request?
Since SPARK-13220(Deprecate "yarn-client" and "yarn-cluster"), YarnClusterSuite doesn't test "yarn cluster" mode correctly.
This pull request fixes it.
## How was this patch tested?
Unit test
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Author: peng.zhang <peng.zhang@xiaomi.com>
Closes#13836 from renozhang/SPARK-16125-test-yarn-cluster-mode.
## What changes were proposed in this pull request?
This PR aims to update Scala/Python/Java examples by replacing `SQLContext` with newly added `SparkSession`.
- Use **SparkSession Builder Pattern** in 154(Scala 55, Java 52, Python 47) files.
- Add `getConf` in Python SparkContext class: `python/pyspark/context.py`
- Replace **SQLContext Singleton Pattern** with **SparkSession Singleton Pattern**:
- `SqlNetworkWordCount.scala`
- `JavaSqlNetworkWordCount.java`
- `sql_network_wordcount.py`
Now, `SQLContexts` are used only in R examples and the following two Python examples. The python examples are untouched in this PR since it already fails some unknown issue.
- `simple_params_example.py`
- `aft_survival_regression.py`
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#12809 from dongjoon-hyun/SPARK-15031.
## What changes were proposed in this pull request?
Eagerly cleanup PySpark's temporary parallelize cleanup files rather than waiting for shut down.
## How was this patch tested?
Unit tests
Author: Holden Karau <holden@us.ibm.com>
Closes#12233 from holdenk/SPARK-13687-cleanup-pyspark-temporary-files.
Move Py4jCallbackConnectionCleaner to Streaming because the callback server starts only in StreamingContext.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10621 from zsxwing/SPARK-12617-2.
This patch added Py4jCallbackConnectionCleaner to clean the leak sockets of Py4J every 30 seconds. This is a workaround before Py4J fixes the leak issue https://github.com/bartdag/py4j/issues/187
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#10579 from zsxwing/SPARK-12617.
Currently, the current line is not cleared by Cltr-C
After this patch
```
>>> asdfasdf^C
Traceback (most recent call last):
File "~/spark/python/pyspark/context.py", line 225, in signal_handler
raise KeyboardInterrupt()
KeyboardInterrupt
```
It's still worse than 1.5 (and before).
Author: Davies Liu <davies@databricks.com>
Closes#10134 from davies/fix_cltrc.
Only install signal in main thread, or it will fail to create context in not-main thread.
Author: Davies Liu <davies@databricks.com>
Closes#9574 from davies/python_signal.
This patch adds a signal handler to trap Ctrl-C and cancels running job.
Author: Ashwin Shankar <ashankar@netflix.com>
Closes#9033 from ashwinshankar77/master.
Just fixing a typo in exception message, raised when attempting to pickle SparkContext.
Author: Icaro Medeiros <icaro.medeiros@gmail.com>
Closes#8724 from icaromedeiros/master.
Spark has an option called spark.localExecution.enabled; according to the docs:
> Enables Spark to run certain jobs, such as first() or take() on the driver, without sending tasks to the cluster. This can make certain jobs execute very quickly, but may require shipping a whole partition of data to the driver.
This feature ends up adding quite a bit of complexity to DAGScheduler, especially in the runLocallyWithinThread method, but as far as I know nobody uses this feature (I searched the mailing list and haven't seen any recent mentions of the configuration nor stacktraces including the runLocally method). As a step towards scheduler complexity reduction, I propose that we remove this feature and all code related to it for Spark 1.5.
This pull request simply brings #7484 up to date.
Author: Josh Rosen <joshrosen@databricks.com>
Author: Reynold Xin <rxin@databricks.com>
Closes#7585 from rxin/remove-local-exec and squashes the following commits:
84bd10e [Reynold Xin] Python fix.
1d9739a [Reynold Xin] Merge pull request #7484 from JoshRosen/remove-localexecution
eec39fa [Josh Rosen] Remove allowLocal(); deprecate user-facing uses of it.
b0835dc [Josh Rosen] Remove local execution code in DAGScheduler
8975d96 [Josh Rosen] Remove local execution tests.
ffa8c9b [Josh Rosen] Remove documentation for configuration
andrewor14 davies vanzin can you take a look at this? thanks
Author: Lianhui Wang <lianhuiwang09@gmail.com>
Closes#7438 from lianhuiwang/SPARK-8646 and squashes the following commits:
cb3f12d [Lianhui Wang] add whitespace
6d874a6 [Lianhui Wang] support pyspark for yarn-client
Use case - we want to log applicationId (YARN in hour case) to request help with troubleshooting from the DevOps
Author: Vladimir Vladimirov <vladimir.vladimirov@magnetic.com>
Closes#6936 from smartkiwi/master and squashes the following commits:
870338b [Vladimir Vladimirov] this would make doctest to run in python3
0eae619 [Vladimir Vladimirov] Scala doesn't use u'...' for unicode literals
14d77a8 [Vladimir Vladimirov] stop using ELLIPSIS
b4ebfc5 [Vladimir Vladimirov] addressed PR feedback - updated docstring
223a32f [Vladimir Vladimirov] fixed test - applicationId is property that returns the string
3221f5a [Vladimir Vladimirov] [SPARK-8528] added documentation for Scala
2cff090 [Vladimir Vladimirov] [SPARK-8528] add applicationId property for SparkContext object in pyspark
This PR fixes the sum issue and also adds `emptyRDD` so that it's easy to create a test case.
Author: zsxwing <zsxwing@gmail.com>
Closes#6826 from zsxwing/python-emptyRDD and squashes the following commits:
b36993f [zsxwing] Update the return type to JavaRDD[T]
71df047 [zsxwing] Add emptyRDD to pyspark and fix the issue when calling sum on an empty RDD
Author: Ted Blackman <ted.blackman@gmail.com>
Closes#6656 from belisarius222/branch-1.4 and squashes the following commits:
747cbc2 [Ted Blackman] [SPARK-8116][PYSPARK] Allow sc.range() to take a single argument.
(cherry picked from commit f02af7c8f7)
Signed-off-by: Reynold Xin <rxin@databricks.com>
Author: Holden Karau <holden@pigscanfly.ca>
Closes#6275 from holdenk/SPARK-771-startTime-is-missing-from-pyspark and squashes the following commits:
06662dc [Holden Karau] add mising blank line for style checks
7a87410 [Holden Karau] add back missing newline
7a7876b [Holden Karau] Add a startTime property to match the corresponding one in the Scala SparkContext
This PR is based on #6081, thanks adrian-wang.
Closes#6081
Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Davies Liu <davies@databricks.com>
Closes#6230 from davies/range and squashes the following commits:
d3ce5fe [Davies Liu] add tests
789eda5 [Davies Liu] add range() in Python
4590208 [Davies Liu] Merge commit 'refs/pull/6081/head' of github.com:apache/spark into range
cbf5200 [Daoyuan Wang] let's add python support in a separate PR
f45e3b2 [Daoyuan Wang] remove redundant toLong
617da76 [Daoyuan Wang] fix safe marge for corner cases
867c417 [Daoyuan Wang] fix
13dbe84 [Daoyuan Wang] update
bd998ba [Daoyuan Wang] update comments
d3a0c1b [Daoyuan Wang] add range api()
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
Add support for changing the log level at run time through the SparkContext. Based on an earlier PR, #2433 includes CR feedback from pwendel & davies
Author: Holden Karau <holden@pigscanfly.ca>
Closes#5791 from holdenk/SPARK-3444-provide-an-easy-way-to-change-log-level-r2 and squashes the following commits:
3bf3be9 [Holden Karau] fix exception
42ba873 [Holden Karau] fix exception
9117244 [Holden Karau] Only allow valid log levels, throw exception if invalid log level.
338d7bf [Holden Karau] rename setLoggingLevel to setLogLevel
fac14a0 [Holden Karau] Fix style errors
d9d03f3 [Holden Karau] Add support for changing the log level at run time through the SparkContext. Based on an earlier PR, #2433 includes CR feedback from @pwendel & @davies
This PR enable auto_convert in JavaGateway, then we could register a converter for a given types, for example, date and datetime.
There are two bugs related to auto_convert, see [1] and [2], we workaround it in this PR.
[1] https://github.com/bartdag/py4j/issues/160
[2] https://github.com/bartdag/py4j/issues/161
cc rxin JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#5570 from davies/py4j_date and squashes the following commits:
eb4fa53 [Davies Liu] fix tests in python 3
d17d634 [Davies Liu] rollback changes in mllib
2e7566d [Davies Liu] convert tuple into ArrayList
ceb3779 [Davies Liu] Update rdd.py
3c373f3 [Davies Liu] support date and datetime by auto_convert
cb094ff [Davies Liu] enable auto convert
Author: Elisey Zanko <elisey.zanko@gmail.com>
Closes#5361 from 31z4/spark-6661 and squashes the following commits:
73c5d79 [Elisey Zanko] Python type errors should print type, not object
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.
Because circular reference between JavaObject and JavaMember, an Java object can not be released until Python GC kick in, then it will cause memory leak in collect(), which may consume lots of memory in JVM.
This PR change the way we sending collected data back into Python from local file to socket, which could avoid any disk IO during collect, also avoid any referrers of Java object in Python.
cc JoshRosen
Author: Davies Liu <davies@databricks.com>
Closes#4923 from davies/fix_collect and squashes the following commits:
d730286 [Davies Liu] address comments
24c92a4 [Davies Liu] fix style
ba54614 [Davies Liu] use socket to transfer data from JVM
9517c8f [Davies Liu] fix memory leak in collect()
This patch bring the pull based progress API into Python, also a example in Python.
Author: Davies Liu <davies@databricks.com>
Closes#3027 from davies/progress_api and squashes the following commits:
b1ba984 [Davies Liu] fix style
d3b9253 [Davies Liu] add tests, mute the exception after stop
4297327 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
969fa9d [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
25590c9 [Davies Liu] update with Java API
360de2d [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
c0f1021 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api
023afb3 [Davies Liu] add Python API and example for progress API
A jar file containing Python sources in it could be used as a Python package, just like zip file.
spark-submit already put the jar file into PYTHONPATH, this patch also put it in the sys.path, then it could be used in Python worker.
Author: Davies Liu <davies@databricks.com>
Closes#4652 from davies/jar and squashes the following commits:
17d3f76 [Davies Liu] support .jar as python package
Whenever a directory is created by the utility method, immediately restrict
its permissions so that only the owner has access to its contents.
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
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
This patch adds more helpful error messages for invalid programs that define nested RDDs, broadcast RDDs, perform actions inside of transformations (e.g. calling `count()` from inside of `map()`), and call certain methods on stopped SparkContexts. Currently, these invalid programs lead to confusing NullPointerExceptions at runtime and have been a major source of questions on the mailing list and StackOverflow.
In a few cases, I chose to log warnings instead of throwing exceptions in order to avoid any chance that this patch breaks programs that worked "by accident" in earlier Spark releases (e.g. programs that define nested RDDs but never run any jobs with them).
In SparkContext, the new `assertNotStopped()` method is used to check whether methods are being invoked on a stopped SparkContext. In some cases, user programs will not crash in spite of calling methods on stopped SparkContexts, so I've only added `assertNotStopped()` calls to methods that always throw exceptions when called on stopped contexts (e.g. by dereferencing a null `dagScheduler` pointer).
Author: Josh Rosen <joshrosen@databricks.com>
Closes#3884 from JoshRosen/SPARK-5063 and squashes the following commits:
a38774b [Josh Rosen] Fix spelling typo
a943e00 [Josh Rosen] Convert two exceptions into warnings in order to avoid breaking user programs in some edge-cases.
2d0d7f7 [Josh Rosen] Fix test to reflect 1.2.1 compatibility
3f0ea0c [Josh Rosen] Revert two unintentional formatting changes
8e5da69 [Josh Rosen] Remove assertNotStopped() calls for methods that were sometimes safe to call on stopped SC's in Spark 1.2
8cff41a [Josh Rosen] IllegalStateException fix
6ef68d0 [Josh Rosen] Fix Python line length issues.
9f6a0b8 [Josh Rosen] Add improved error messages to PySpark.
13afd0f [Josh Rosen] SparkException -> IllegalStateException
8d404f3 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5063
b39e041 [Josh Rosen] Fix BroadcastSuite test which broadcasted an RDD
99cc09f [Josh Rosen] Guard against calling methods on stopped SparkContexts.
34833e8 [Josh Rosen] Add more descriptive error message.
57cc8a1 [Josh Rosen] Add error message when directly broadcasting RDD.
15b2e6b [Josh Rosen] [SPARK-5063] Useful error messages for nested RDDs and actions inside of transformations
After the default batchSize changed to 0 (batched based on the size of object), but parallelize() still use BatchedSerializer with batchSize=1, this PR will use batchSize=1024 for parallelize by default.
Also, BatchedSerializer did not work well with list and numpy.ndarray, this improve BatchedSerializer by using __len__ and __getslice__.
Here is the benchmark for parallelize 1 millions int with list or ndarray:
| before | after | improvements
------- | ------------ | ------------- | -------
list | 11.7 s | 0.8 s | 14x
numpy.ndarray | 32 s | 0.7 s | 40x
Author: Davies Liu <davies@databricks.com>
Closes#4024 from davies/opt_numpy and squashes the following commits:
7618c7c [Davies Liu] improve performance of parallelize list/ndarray
Modify python annotations for sphinx. There is no change to build process from.
https://github.com/apache/spark/blob/master/docs/README.md
Author: lewuathe <lewuathe@me.com>
Closes#3685 from Lewuathe/sphinx-tag-for-pydoc and squashes the following commits:
88a0fd9 [lewuathe] [SPARK-4822] Fix DevelopApi and WARN tags
3d7a398 [lewuathe] [SPARK-4822] Use sphinx tags for Python doc annotations
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
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