aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/context.py
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-20360][PYTHON] reprs for interpretersKyle Kelley2017-04-181-0/+26
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Establishes a very minimal `_repr_html_` for PySpark's `SparkContext`. ## How was this patch tested? nteract: ![screen shot 2017-04-17 at 3 41 29 pm](https://cloud.githubusercontent.com/assets/836375/25107701/d57090ba-2385-11e7-8147-74bc2c50a41b.png) Jupyter: ![screen shot 2017-04-17 at 3 53 19 pm](https://cloud.githubusercontent.com/assets/836375/25107725/05bf1fe8-2386-11e7-93e1-07a20c917dde.png) Hydrogen: ![screen shot 2017-04-17 at 3 49 55 pm](https://cloud.githubusercontent.com/assets/836375/25107664/a75e1ddc-2385-11e7-8477-258661833007.png) Author: Kyle Kelley <rgbkrk@gmail.com> Closes #17662 from rgbkrk/repr.
* [SPARK-13330][PYSPARK] PYTHONHASHSEED is not propgated to python workerJeff Zhang2017-02-241-4/+2
| | | | | | | | | | | | ## 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.
* [SPARK-19307][PYSPARK] Make sure user conf is propagated to SparkContext.Marcelo Vanzin2017-01-251-0/+3
| | | | | | | | | | | | 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.
* [SPARK-18523][PYSPARK] Make SparkContext.stop more reliableAlexander Shorin2016-11-281-2/+15
| | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-18447][DOCS] Fix the markdown for `Note:`/`NOTE:`/`Note that` across ↵hyukjinkwon2016-11-221-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Python API documentation ## What changes were proposed in this pull request? It seems in Python, there are - `Note:` - `NOTE:` - `Note that` - `.. note::` This PR proposes to fix those to `.. note::` to be consistent. **Before** <img width="567" alt="2016-11-21 1 18 49" src="https://cloud.githubusercontent.com/assets/6477701/20464305/85144c86-af88-11e6-8ee9-90f584dd856c.png"> <img width="617" alt="2016-11-21 12 42 43" src="https://cloud.githubusercontent.com/assets/6477701/20464263/27be5022-af88-11e6-8577-4bbca7cdf36c.png"> **After** <img width="554" alt="2016-11-21 1 18 42" src="https://cloud.githubusercontent.com/assets/6477701/20464306/8fe48932-af88-11e6-83e1-fc3cbf74407d.png"> <img width="628" alt="2016-11-21 12 42 51" src="https://cloud.githubusercontent.com/assets/6477701/20464264/2d3e156e-af88-11e6-93f3-cab8d8d02983.png"> ## How was this patch tested? The notes were found via ```bash grep -r "Note: " . grep -r "NOTE: " . grep -r "Note that " . ``` And then fixed one by one comparing with API documentation. After that, manually tested via `make html` under `./python/docs`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15947 from HyukjinKwon/SPARK-18447.
* [SPARK-18138][DOCS] Document that Java 7, Python 2.6, Scala 2.10, Hadoop < ↵Sean Owen2016-11-031-0/+4
| | | | | | | | | | | | | | | | 2.6 are deprecated in Spark 2.1.0 ## 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.
* [SPARK-17387][PYSPARK] Creating SparkContext() from python without ↵Jeff Zhang2016-10-111-4/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | spark-submit ignores user conf ## 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.
* [SPARK-16861][PYSPARK][CORE] Refactor PySpark accumulator API on top of ↵Holden Karau2016-09-231-3/+2
| | | | | | | | | | | | | | | | Accumulator V2 ## 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.
* [SPARK-17585][PYSPARK][CORE] PySpark SparkContext.addFile supports adding ↵Yanbo Liang2016-09-211-2/+5
| | | | | | | | | | | | | | files recursively ## 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.
* [SPARK-17437] Add uiWebUrl to JavaSparkContext and pyspark.SparkContextAdrian Petrescu2016-09-201-0/+5
| | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-17525][PYTHON] Remove SparkContext.clearFiles() from the PySpark API ↵Sami Jaktholm2016-09-141-8/+0
| | | | | | | | | | | | | | | | as it was removed from the Scala API prior to Spark 2.0.0 ## 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 8ce645d4eeda203cf5e100c4bdba2d71edd44e6a. 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.
* [SPARK-16224] [SQL] [PYSPARK] SparkSession builder's configs need to be set ↵Yin Huai2016-06-281-0/+2
| | | | | | | | | | | | | | to the existing Scala SparkContext's SparkConf ## 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.
* [SPARK-16125][YARN] Fix not test yarn cluster mode correctly in YarnClusterSuitepeng.zhang2016-06-241-4/+0
| | | | | | | | | | | | | | | | ## 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.
* [SPARK-15031][EXAMPLE] Use SparkSession in Scala/Python/Java example.Dongjoon Hyun2016-05-041-0/+5
| | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-13687][PYTHON] Cleanup PySpark parallelize temporary filesHolden Karau2016-04-101-9/+13
| | | | | | | | | | | | | | ## 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.
* [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to StreamingShixiong Zhu2016-01-061-61/+0
| | | | | | | | 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.
* [SPARK-12617] [PYSPARK] Clean up the leak sockets of Py4JShixiong Zhu2016-01-051-0/+61
| | | | | | | | 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.
* [SPARK-12132] [PYSPARK] raise KeyboardInterrupt inside SIGINT handlerDavies Liu2015-12-071-0/+1
| | | | | | | | | | | | | | | | | | | 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.
* [SPARK-11463] [PYSPARK] only install signal in main threadDavies Liu2015-11-101-1/+4
| | | | | | | | 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.
* [SPARK-11114][PYSPARK] add getOrCreate for SparkContext/SQLContext in PythonDavies Liu2015-10-191-2/+14
| | | | | | | | Also added SQLContext.newSession() Author: Davies Liu <davies@databricks.com> Closes #9122 from davies/py_create.
* [SPARK-8170] [PYTHON] Add signal handler to trap Ctrl-C in pyspark and ↵Ashwin Shankar2015-10-121-0/+7
| | | | | | | | | | cancel all running jobs 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.
* [PYTHON] Fixed typo in exception messageIcaro Medeiros2015-09-111-1/+1
| | | | | | | | 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.
* [MINOR] [SQL] Fix sphinx warnings in PySpark SQLMechCoder2015-08-201-4/+4
| | | | | | Author: MechCoder <manojkumarsivaraj334@gmail.com> Closes #8171 from MechCoder/sql_sphinx.
* [SPARK-9144] Remove DAGScheduler.runLocallyWithinThread and ↵Josh Rosen2015-07-221-2/+1
| | | | | | | | | | | | | | | | | | | | | | | | spark.localExecution.enabled 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
* [SPARK-8646] PySpark does not run on YARN if master not provided in command lineLianhui Wang2015-07-161-0/+5
| | | | | | | | | | | 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
* [SPARK-8528] Expose SparkContext.applicationId in PySparkVladimir Vladimirov2015-06-291-0/+15
| | | | | | | | | | | | | | | | 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
* [SPARK-8373] [PYSPARK] Add emptyRDD to pyspark and fix the issue when ↵zsxwing2015-06-171-0/+6
| | | | | | | | | | | | | calling sum on an empty RDD 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
* [SPARK-8116][PYSPARK] Allow sc.range() to take a single argument.Ted Blackman2015-06-041-2/+12
| | | | | | | | | | | 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 f02af7c8f7f43e4cfe3c412d2b5ea4128669ce22) Signed-off-by: Reynold Xin <rxin@databricks.com>
* [SPARK-7711] Add a startTime property to match the corresponding one in ScalaHolden Karau2015-05-211-0/+5
| | | | | | | | | | 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
* [SPARK-7150] SparkContext.range() and SQLContext.range()Daoyuan Wang2015-05-181-0/+16
| | | | | | | | | | | | | | | | | | | | | | 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()
* [SPARK-6216] [PYSPARK] check python version of worker with driverDavies Liu2015-05-181-0/+1
| | | | | | | | | | | | 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-3444] Provide an easy way to change log levelHolden Karau2015-05-011-0/+7
| | | | | | | | | | | | | | | 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
* [SPARK-6949] [SQL] [PySpark] Support Date/Timestamp in Column expressionDavies Liu2015-04-211-5/+1
| | | | | | | | | | | | | | | | | | | | | | 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
* [SPARK-6661] Python type errors should print type, not objectElisey Zanko2015-04-201-1/+1
| | | | | | | | 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
* [SPARK-4897] [PySpark] Python 3 supportDavies Liu2015-04-161-14/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-6194] [SPARK-677] [PySpark] fix memory leak in collect()Davies Liu2015-03-091-7/+6
| | | | | | | | | | | | | | | | | 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()
* [SPARK-4172] [PySpark] Progress API in PythonDavies Liu2015-02-171-0/+7
| | | | | | | | | | | | | | | | | 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
* [SPARK-5864] [PySpark] support .jar as python packageDavies Liu2015-02-171-2/+4
| | | | | | | | | | | | 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
* Make sure only owner can read / write to directories created for the job.Marcelo Vanzin2015-02-021-1/+2
| | | | | | | 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>
* [SPARK-4387][PySpark] Refactoring python profiling code to make it extensibleYandu Oppacher2015-01-281-32/+14
| | | | | | | | | | | | | | | | | | | | | | | | 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-5063] More helpful error messages for several invalid operationsJosh Rosen2015-01-231-0/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | 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
* [SPARK-5224] [PySpark] improve performance of parallelize list/ndarrayDavies Liu2015-01-151-1/+1
| | | | | | | | | | | | | | | | | | | 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
* [SPARK-4822] Use sphinx tags for Python doc annotationslewuathe2014-12-171-2/+2
| | | | | | | | | | | | 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
* [SPARK-4548] []SPARK-4517] improve performance of python broadcastDavies Liu2014-11-241-10/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-2/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | 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-4398][PySpark] specialize sc.parallelize(xrange)Xiangrui Meng2014-11-141-4/+21
| | | | | | | | | | | | | `sc.parallelize(range(1 << 20), 1).count()` may take 15 seconds to finish and the rdd object stores the entire list, making task size very large. This PR adds a specialized version for xrange. JoshRosen davies Author: Xiangrui Meng <meng@databricks.com> Closes #3264 from mengxr/SPARK-4398 and squashes the following commits: 8953c41 [Xiangrui Meng] follow davies' suggestion cbd58e3 [Xiangrui Meng] specialize sc.parallelize(xrange)
* [SPARK-4186] add binaryFiles and binaryRecords in PythonDavies Liu2014-11-061-1/+31
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | add binaryFiles() and binaryRecords() in Python ``` binaryFiles(self, path, minPartitions=None): :: Developer API :: Read a directory of binary files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI as a byte array. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file. Note: Small files are preferred, large file is also allowable, but may cause bad performance. binaryRecords(self, path, recordLength): Load data from a flat binary file, assuming each record is a set of numbers with the specified numerical format (see ByteBuffer), and the number of bytes per record is constant. :param path: Directory to the input data files :param recordLength: The length at which to split the records ``` Author: Davies Liu <davies@databricks.com> Closes #3078 from davies/binary and squashes the following commits: cd0bdbd [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary 3aa349b [Davies Liu] add experimental notes 24e84b6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary 5ceaa8a [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary 1900085 [Davies Liu] bugfix bb22442 [Davies Liu] add binaryFiles and binaryRecords in Python
* [SPARK-3886] [PySpark] simplify serializer, use AutoBatchedSerializer by ↵Davies Liu2014-11-031-38/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | default. This PR simplify serializer, always use batched serializer (AutoBatchedSerializer as default), even batch size is 1. Author: Davies Liu <davies@databricks.com> This patch had conflicts when merged, resolved by Committer: Josh Rosen <joshrosen@databricks.com> Closes #2920 from davies/fix_autobatch and squashes the following commits: e544ef9 [Davies Liu] revert unrelated change 6880b14 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch 1d557fc [Davies Liu] fix tests 8180907 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch 76abdce [Davies Liu] clean up 53fa60b [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch d7ac751 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch 2cc2497 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch b4292ce [Davies Liu] fix bug in master d79744c [Davies Liu] recover hive tests be37ece [Davies Liu] refactor eb3938d [Davies Liu] refactor serializer in scala 8d77ef2 [Davies Liu] simplify serializer, use AutoBatchedSerializer by default.
* [SPARK-2652] [PySpark] donot use KyroSerializer as default serializerDavies Liu2014-10-231-1/+0
| | | | | | | | | | | | KyroSerializer can not serialize customized class without registered explicitly, use it as default serializer in PySpark will introduce some regression in MLlib. cc mengxr Author: Davies Liu <davies@databricks.com> Closes #2916 from davies/revert and squashes the following commits: 43eb6d3 [Davies Liu] donot use KyroSerializer as default serializer
* [SPARK-3971] [MLLib] [PySpark] hotfix: Customized pickler should work in ↵Davies Liu2014-10-161-2/+0
| | | | | | | | | | | | | | | | cluster mode Customized pickler should be registered before unpickling, but in executor, there is no way to register the picklers before run the tasks. So, we need to register the picklers in the tasks itself, duplicate the javaToPython() and pythonToJava() in MLlib, call SerDe.initialize() before pickling or unpickling. Author: Davies Liu <davies.liu@gmail.com> Closes #2830 from davies/fix_pickle and squashes the following commits: 0c85fb9 [Davies Liu] revert the privacy change 6b94e15 [Davies Liu] use JavaConverters instead of JavaConversions 0f02050 [Davies Liu] hotfix: Customized pickler does not work in cluster