Commit graph

132 commits

Author SHA1 Message Date
Alexander Shorin 71352c94ad [SPARK-18523][PYSPARK] Make SparkContext.stop more reliable
## 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.
2016-11-28 18:28:24 -08:00
hyukjinkwon 933a6548d4
[SPARK-18447][DOCS] Fix the markdown for Note:/NOTE:/Note that across 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.
2016-11-22 11:40:18 +00:00
Sean Owen dc4c600986 [SPARK-18138][DOCS] Document that Java 7, Python 2.6, Scala 2.10, Hadoop < 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.
2016-11-03 17:27:23 -07:00
Jeff Zhang 5b77e66dd6 [SPARK-17387][PYSPARK] Creating SparkContext() from python without 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.
2016-10-11 14:56:26 -07:00
Holden Karau 90d5754212
[SPARK-16861][PYSPARK][CORE] Refactor PySpark accumulator API on top of 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.
2016-09-23 09:44:30 +01:00
Yanbo Liang d3b8869763 [SPARK-17585][PYSPARK][CORE] PySpark SparkContext.addFile supports adding 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.
2016-09-21 01:37:03 -07:00
Adrian Petrescu 4a426ff8ae
[SPARK-17437] Add uiWebUrl to JavaSparkContext and pyspark.SparkContext
## 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.
2016-09-20 10:49:02 +01:00
Sami Jaktholm b5bfcddbfb [SPARK-17525][PYTHON] Remove SparkContext.clearFiles() from the PySpark API 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 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.
2016-09-14 09:38:30 +01:00
Yin Huai 0923c4f567 [SPARK-16224] [SQL] [PYSPARK] SparkSession builder's configs need to be set 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.
2016-06-28 07:54:44 -07:00
peng.zhang f4fd7432fb [SPARK-16125][YARN] Fix not test yarn cluster mode correctly in YarnClusterSuite
## 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.
2016-06-24 08:28:32 +01:00
Dongjoon Hyun cdce4e62a5 [SPARK-15031][EXAMPLE] Use SparkSession in Scala/Python/Java example.
## 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.
2016-05-04 14:31:36 -07:00
Holden Karau 00288ea2a4 [SPARK-13687][PYTHON] Cleanup PySpark parallelize temporary files
## 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.
2016-04-10 02:34:54 +01:00
Shixiong Zhu 1e6648d62f [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming
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.
2016-01-06 12:03:01 -08:00
Shixiong Zhu 047a31bb10 [SPARK-12617] [PYSPARK] Clean up the leak sockets of Py4J
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.
2016-01-05 13:10:46 -08:00
Davies Liu ef3f047c07 [SPARK-12132] [PYSPARK] raise KeyboardInterrupt inside SIGINT handler
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.
2015-12-07 11:00:25 -08:00
Davies Liu bd70244b3c [SPARK-11463] [PYSPARK] only install signal in main thread
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.
2015-11-10 22:46:17 -08:00
Davies Liu 232d7f8d42 [SPARK-11114][PYSPARK] add getOrCreate for SparkContext/SQLContext in Python
Also added SQLContext.newSession()

Author: Davies Liu <davies@databricks.com>

Closes #9122 from davies/py_create.
2015-10-19 16:18:20 -07:00
Ashwin Shankar 2e572c4135 [SPARK-8170] [PYTHON] Add signal handler to trap Ctrl-C in pyspark and 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.
2015-10-12 11:06:21 -07:00
Icaro Medeiros c373866774 [PYTHON] Fixed typo in exception message
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.
2015-09-11 21:46:52 +01:00
MechCoder 52c60537a2 [MINOR] [SQL] Fix sphinx warnings in PySpark SQL
Author: MechCoder <manojkumarsivaraj334@gmail.com>

Closes #8171 from MechCoder/sql_sphinx.
2015-08-20 10:05:31 -07:00
Josh Rosen b217230f2a [SPARK-9144] Remove DAGScheduler.runLocallyWithinThread and 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
2015-07-22 21:04:04 -07:00
Lianhui Wang 49351c7f59 [SPARK-8646] PySpark does not run on YARN if master not provided in command line
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
2015-07-16 19:31:45 -07:00
Vladimir Vladimirov 492dca3a73 [SPARK-8528] Expose SparkContext.applicationId in PySpark
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
2015-06-29 12:03:41 -07:00
zsxwing 0fc4b96f3e [SPARK-8373] [PYSPARK] Add emptyRDD to pyspark and fix the issue when 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
2015-06-17 13:59:39 -07:00
Ted Blackman e505460599 [SPARK-8116][PYSPARK] Allow sc.range() to take a single argument.
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>
2015-06-04 22:22:01 -07:00
Holden Karau 6b18cdc1b1 [SPARK-7711] Add a startTime property to match the corresponding one in Scala
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
2015-05-21 14:08:57 -07:00
Daoyuan Wang c2437de189 [SPARK-7150] SparkContext.range() and SQLContext.range()
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()
2015-05-18 21:43:12 -07:00
Davies Liu 32fbd297dd [SPARK-6216] [PYSPARK] check python version of worker with driver
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
2015-05-18 12:55:13 -07:00
Holden Karau ae98eec730 [SPARK-3444] Provide an easy way to change log level
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
2015-05-01 18:02:51 -07:00
Davies Liu ab9128fb7e [SPARK-6949] [SQL] [PySpark] Support Date/Timestamp in Column expression
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
2015-04-21 00:08:18 -07:00
Elisey Zanko 77176619a9 [SPARK-6661] Python type errors should print type, not object
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
2015-04-20 10:44:09 -07:00
Davies Liu 04e44b37cc [SPARK-4897] [PySpark] Python 3 support
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.
2015-04-16 16:20:57 -07:00
Davies Liu 8767565cef [SPARK-6194] [SPARK-677] [PySpark] fix memory leak in collect()
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()
2015-03-09 16:24:06 -07:00
Davies Liu 445a755b88 [SPARK-4172] [PySpark] Progress API in Python
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
2015-02-17 13:36:43 -08:00
Davies Liu fc4eb9505a [SPARK-5864] [PySpark] support .jar as python package
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
2015-02-17 12:05:06 -08:00
Marcelo Vanzin 52f5754f45 Make sure only owner can read / write to directories created for the job.
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>
2015-02-02 14:01:32 -08:00
Yandu Oppacher 3bead67d59 [SPARK-4387][PySpark] Refactoring python profiling code to make it extensible
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
2015-01-28 13:48:06 -08:00
Josh Rosen cef1f092a6 [SPARK-5063] More helpful error messages for several invalid operations
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
2015-01-23 17:53:15 -08:00
Davies Liu 3c8650c12a [SPARK-5224] [PySpark] improve performance of parallelize list/ndarray
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
2015-01-15 11:40:41 -08:00
lewuathe 3cd516191b [SPARK-4822] Use sphinx tags for Python doc annotations
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
2014-12-17 17:31:24 -08:00
Davies Liu 6cf507685e [SPARK-4548] []SPARK-4517] improve performance of python broadcast
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
2014-11-24 17:17:03 -08:00
Davies Liu 4a377aff2d [SPARK-3721] [PySpark] broadcast objects larger than 2G
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
2014-11-18 16:17:51 -08:00
Xiangrui Meng abd581752f [SPARK-4398][PySpark] specialize sc.parallelize(xrange)
`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)
2014-11-14 12:43:17 -08:00
Davies Liu b41a39e240 [SPARK-4186] add binaryFiles and binaryRecords in Python
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
2014-11-06 00:22:19 -08:00
Davies Liu e4f42631a6 [SPARK-3886] [PySpark] simplify serializer, use AutoBatchedSerializer by 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.
2014-11-03 23:56:14 -08:00
Davies Liu 809c785bcc [SPARK-2652] [PySpark] donot use KyroSerializer as default serializer
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
2014-10-23 23:58:00 -07:00
Davies Liu 091d32c52e [SPARK-3971] [MLLib] [PySpark] hotfix: Customized pickler should work in 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
2014-10-16 14:56:50 -07:00
giwa 69c67abaa9 [SPARK-2377] Python API for Streaming
This patch brings Python API for Streaming.

This patch is based on work from @giwa

Author: giwa <ugw.gi.world@gmail.com>
Author: Ken Takagiwa <ken@Kens-MacBook-Pro.local>
Author: Davies Liu <davies.liu@gmail.com>
Author: Ken Takagiwa <ken@kens-mbp.gateway.sonic.net>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Ken <ugw.gi.world@gmail.com>
Author: Ken Takagiwa <ugw.gi.world@gmail.com>
Author: Matthew Farrellee <matt@redhat.com>

Closes #2538 from davies/streaming and squashes the following commits:

64561e4 [Davies Liu] fix tests
331ecce [Davies Liu] fix example
3e2492b [Davies Liu] change updateStateByKey() to easy API
182be73 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
02d0575 [Davies Liu] add wrapper for foreachRDD()
bebeb4a [Davies Liu] address all comments
6db00da [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
8380064 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
52c535b [Davies Liu] remove fix for sum()
e108ec1 [Davies Liu]  address comments
37fe06f [Davies Liu] use random port for callback server
d05871e [Davies Liu] remove reuse of PythonRDD
be5e5ff [Davies Liu] merge branch of env, make tests stable.
8071541 [Davies Liu] Merge branch 'env' into streaming
c7bbbce [Davies Liu] fix sphinx docs
6bb9d91 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
4d0ea8b [Davies Liu] clear reference of SparkEnv after stop
54bd92b [Davies Liu] improve tests
c2b31cb [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
7a88f9f [Davies Liu] rollback RDD.setContext(), use textFileStream() to test checkpointing
bd8a4c2 [Davies Liu] fix scala style
7797c70 [Davies Liu] refactor
ff88bec [Davies Liu] rename RDDFunction to TransformFunction
d328aca [Davies Liu] fix serializer in queueStream
6f0da2f [Davies Liu] recover from checkpoint
fa7261b [Davies Liu] refactor
a13ff34 [Davies Liu] address comments
8466916 [Davies Liu] support checkpoint
9a16bd1 [Davies Liu] change number of partitions during tests
b98d63f [Davies Liu] change private[spark] to private[python]
eed6e2a [Davies Liu] rollback not needed changes
e00136b [Davies Liu] address comments
069a94c [Davies Liu] fix the number of partitions during window()
338580a [Davies Liu] change _first(), _take(), _collect() as private API
19797f9 [Davies Liu] clean up
6ebceca [Davies Liu] add more tests
c40c52d [Davies Liu] change first(), take(n) to has the same behavior as RDD
98ac6c2 [Davies Liu] support ssc.transform()
b983f0f [Davies Liu] address comments
847f9b9 [Davies Liu] add more docs, add first(), take()
e059ca2 [Davies Liu] move check of window into Python
fce0ef5 [Davies Liu] rafactor of foreachRDD()
7001b51 [Davies Liu] refactor of queueStream()
26ea396 [Davies Liu] refactor
74df565 [Davies Liu] fix print and docs
b32774c [Davies Liu] move java_import into streaming
604323f [Davies Liu] enable streaming tests
c499ba0 [Davies Liu] remove Time and Duration
3f0fb4b [Davies Liu] refactor fix tests
c28f520 [Davies Liu] support updateStateByKey
d357b70 [Davies Liu] support windowed dstream
bd13026 [Davies Liu] fix examples
eec401e [Davies Liu] refactor, combine TransformedRDD, fix reuse PythonRDD, fix union
9a57685 [Davies Liu] fix python style
bd27874 [Davies Liu] fix scala style
7339be0 [Davies Liu] delete tests
7f53086 [Davies Liu] support transform(), refactor and cleanup
df098fc [Davies Liu] Merge branch 'master' into giwa
550dfd9 [giwa] WIP fixing 1.1 merge
5cdb6fa [giwa] changed for SCCallSiteSync
e685853 [giwa] meged with rebased 1.1 branch
2d32a74 [giwa] added some StreamingContextTestSuite
4a59e1e [giwa] WIP:added more test for StreamingContext
8ffdbf1 [giwa] added atexit to handle callback server
d5f5fcb [giwa] added comment for StreamingContext.sparkContext
63c881a [giwa] added StreamingContext.sparkContext
d39f102 [giwa] added StreamingContext.remember
d542743 [giwa] clean up code
2fdf0de [Matthew Farrellee] Fix scalastyle errors
c0a06bc [giwa] delete not implemented functions
f385976 [giwa] delete inproper comments
b0f2015 [giwa] added comment in dstream._test_output
bebb3f3 [giwa] remove the last brank line
fbed8da [giwa] revert pom.xml
8ed93af [giwa] fixed explanaiton
066ba90 [giwa] revert pom.xml
fa4af88 [giwa] remove duplicated import
6ae3caa [giwa] revert pom.xml
7dc7391 [giwa] fixed typo
62dc7a3 [giwa] clean up exmples
f04882c [giwa] clen up examples
b171ec3 [giwa] fixed pep8 violation
f198d14 [giwa] clean up code
3166d31 [giwa] clean up
c00e091 [giwa] change test case not to use awaitTermination
e80647e [giwa] adopted the latest compression way of python command
58e41ff [giwa] merge with master
455e5af [giwa] removed wasted print in DStream
af336b7 [giwa] add comments
ddd4ee1 [giwa] added TODO coments
99ce042 [giwa] added saveAsTextFiles and saveAsPickledFiles
2a06cdb [giwa] remove waste duplicated code
c5ecfc1 [giwa] basic function test cases are passed
8dcda84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
795b2cd [giwa] broke something
1e126bf [giwa] WIP: solved partitioned and None is not recognized
f67cf57 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
953deb0 [giwa] edited the comment to add more precise description
af610d3 [giwa] removed unnesessary changes
c1d546e [giwa] fixed PEP-008 violation
99410be [giwa] delete waste file
b3b0362 [giwa] added basic operation test cases
9cde7c9 [giwa] WIP added test case
bd3ba53 [giwa] WIP
5c04a5f [giwa] WIP: added PythonTestInputStream
019ef38 [giwa] WIP
1934726 [giwa] update comment
376e3ac [giwa] WIP
932372a [giwa] clean up dstream.py
0b09cff [giwa] added stop in StreamingContext
92e333e [giwa] implemented reduce and count function in Dstream
1b83354 [giwa] Removed the waste line
88f7506 [Ken Takagiwa] Kill py4j callback server properly
54b5358 [Ken Takagiwa] tried to restart callback server
4f07163 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
fe02547 [Ken Takagiwa] remove waste file
2ad7bd3 [Ken Takagiwa] clean up codes
6197a11 [Ken Takagiwa] clean up code
eb4bf48 [Ken Takagiwa] fix map function
98c2a00 [Ken Takagiwa] added count operation but this implementation need double check
58591d2 [Ken Takagiwa] reduceByKey is working
0df7111 [Ken Takagiwa] delete old file
f485b1d [Ken Takagiwa] fied input of socketTextDStream
dd6de81 [Ken Takagiwa] initial commit for socketTextStream
247fd74 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
4bcb318 [Ken Takagiwa] implementing transform function in Python
38adf95 [Ken Takagiwa] added reducedByKey not working yet
66fcfff [Ken Takagiwa] modify dstream.py to fix indent error
41886c2 [Ken Takagiwa] comment PythonDStream.PairwiseDStream
0b99bec [Ken] initial commit for pySparkStreaming
c214199 [giwa] added testcase for combineByKey
5625bdc [giwa] added gorupByKey testcase
10ab87b [giwa] added sparkContext as input parameter in StreamingContext
10b5b04 [giwa] removed wasted print in DStream
e54f986 [giwa] add comments
16aa64f [giwa] added TODO coments
74535d4 [giwa] added saveAsTextFiles and saveAsPickledFiles
f76c182 [giwa] remove waste duplicated code
18c8723 [giwa] modified streaming test case to add coment
13fb44c [giwa] basic function test cases are passed
3000b2b [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
ff14070 [giwa] broke something
bcdec33 [giwa] WIP: solved partitioned and None is not recognized
270a9e1 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
bb10956 [giwa] edited the comment to add more precise description
253a863 [giwa] removed unnesessary changes
3d37822 [giwa] fixed PEP-008 violation
f21cab3 [giwa] delete waste file
878bad7 [giwa] added basic operation test cases
ce2acd2 [giwa] WIP added test case
9ad6855 [giwa] WIP
1df77f5 [giwa] WIP: added PythonTestInputStream
1523b66 [giwa] WIP
8a0fbbc [giwa] update comment
fe648e3 [giwa] WIP
29c2bc5 [giwa] initial commit for testcase
4d40d63 [giwa] clean up dstream.py
c462bb3 [giwa] added stop in StreamingContext
d2c01ba [giwa] clean up examples
3c45cd2 [giwa] implemented reduce and count function in Dstream
b349649 [giwa] Removed the waste line
3b498e1 [Ken Takagiwa] Kill py4j callback server properly
84a9668 [Ken Takagiwa] tried to restart callback server
9ab8952 [Tathagata Das] Added extra line.
05e991b [Tathagata Das] Added missing file
b1d2a30 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
678e854 [Ken Takagiwa] remove waste file
0a8bbbb [Ken Takagiwa] clean up codes
bab31c1 [Ken Takagiwa] clean up code
72b9738 [Ken Takagiwa] fix map function
d3ee86a [Ken Takagiwa] added count operation but this implementation need double check
15feea9 [Ken Takagiwa] edit python sparkstreaming example
6f98e50 [Ken Takagiwa] reduceByKey is working
c455c8d [Ken Takagiwa] added reducedByKey not working yet
dc6995d [Ken Takagiwa] delete old file
b31446a [Ken Takagiwa] fixed typo of network_workdcount.py
ccfd214 [Ken Takagiwa] added doctest for pyspark.streaming.duration
0d1b954 [Ken Takagiwa] fied input of socketTextDStream
f746109 [Ken Takagiwa] initial commit for socketTextStream
bb7ccf3 [Ken Takagiwa] remove unused import in python
224fc5e [Ken Takagiwa] add empty line
d2099d8 [Ken Takagiwa] sorted the import following Spark coding convention
5bac7ec [Ken Takagiwa] revert streaming/pom.xml
e1df940 [Ken Takagiwa] revert pom.xml
494cae5 [Ken Takagiwa] remove not implemented DStream functions in python
17a74c6 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
1a0f065 [Ken Takagiwa] implementing transform function in Python
d7b4d6f [Ken Takagiwa] added reducedByKey not working yet
87438e2 [Ken Takagiwa] modify dstream.py to fix indent error
b406252 [Ken Takagiwa] comment PythonDStream.PairwiseDStream
454981d [Ken] initial commit for pySparkStreaming
150b94c [giwa] added some StreamingContextTestSuite
f7bc8f9 [giwa] WIP:added more test for StreamingContext
ee50c5a [giwa] added atexit to handle callback server
fdc9125 [giwa] added comment for StreamingContext.sparkContext
f5bfb70 [giwa] added StreamingContext.sparkContext
da09768 [giwa] added StreamingContext.remember
d68b568 [giwa] clean up code
4afa390 [giwa] clean up code
1fd6bc7 [Ken Takagiwa] Merge pull request #2 from mattf/giwa-master
d9d59fe [Matthew Farrellee] Fix scalastyle errors
67473a9 [giwa] delete not implemented functions
c97377c [giwa] delete inproper comments
2ea769e [giwa] added comment in dstream._test_output
3b27bd4 [giwa] remove the last brank line
acfcaeb [giwa] revert pom.xml
93f7637 [giwa] fixed explanaiton
50fd6f9 [giwa] revert pom.xml
4f82c89 [giwa] remove duplicated import
9d1de23 [giwa] revert pom.xml
7339df2 [giwa] fixed typo
9c85e48 [giwa] clean up exmples
24f95db [giwa] clen up examples
0d30109 [giwa] fixed pep8 violation
b7dab85 [giwa] improve test case
583e66d [giwa] move tests for streaming inside streaming directory
1d84142 [giwa] remove unimplement test
f0ea311 [giwa] clean up code
171edeb [giwa] clean up
4dedd2d [giwa] change test case not to use awaitTermination
268a6a5 [giwa] Changed awaitTermination not to call awaitTermincation in Scala. Just use time.sleep instread
09a28bf [giwa] improve testcases
58150f5 [giwa] Changed the test case to focus the test operation
199e37f [giwa] adopted the latest compression way of python command
185fdbf [giwa] merge with master
f1798c4 [giwa] merge with master
e70f706 [giwa] added testcase for combineByKey
e162822 [giwa] added gorupByKey testcase
97742fe [giwa] added sparkContext as input parameter in StreamingContext
14d4c0e [giwa] removed wasted print in DStream
6d8190a [giwa] add comments
4aa99e4 [giwa] added TODO coments
e9fab72 [giwa] added saveAsTextFiles and saveAsPickledFiles
94f2b65 [giwa] remove waste duplicated code
580fbc2 [giwa] modified streaming test case to add coment
99e4bb3 [giwa] basic function test cases are passed
7051a84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
35933e1 [giwa] broke something
9767712 [giwa] WIP: solved partitioned and None is not recognized
4f2d7e6 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
33c0f94d [giwa] edited the comment to add more precise description
774f18d [giwa] removed unnesessary changes
3a671cc [giwa] remove export PYSPARK_PYTHON in spark submit
8efa266 [giwa] fixed PEP-008 violation
fa75d71 [giwa] delete waste file
7f96294 [giwa] added basic operation test cases
3dda31a [giwa] WIP added test case
1f68b78 [giwa] WIP
c05922c [giwa] WIP: added PythonTestInputStream
1fd12ae [giwa] WIP
c880a33 [giwa] update comment
5d22c92 [giwa] WIP
ea4b06b [giwa] initial commit for testcase
5a9b525 [giwa] clean up dstream.py
79c5809 [giwa] added stop in StreamingContext
189dcea [giwa] clean up examples
b8d7d24 [giwa] implemented reduce and count function in Dstream
b6468e6 [giwa] Removed the waste line
b47b5fd [Ken Takagiwa] Kill py4j callback server properly
19ddcdd [Ken Takagiwa] tried to restart callback server
c9fc124 [Tathagata Das] Added extra line.
4caae3f [Tathagata Das] Added missing file
4eff053 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
5e822d4 [Ken Takagiwa] remove waste file
aeaf8a5 [Ken Takagiwa] clean up codes
9fa249b [Ken Takagiwa] clean up code
05459c6 [Ken Takagiwa] fix map function
a9f4ecb [Ken Takagiwa] added count operation but this implementation need double check
d1ee6ca [Ken Takagiwa] edit python sparkstreaming example
0b8b7d0 [Ken Takagiwa] reduceByKey is working
d25d5cf [Ken Takagiwa] added reducedByKey not working yet
7f7c5d1 [Ken Takagiwa] delete old file
967dc26 [Ken Takagiwa] fixed typo of network_workdcount.py
57fb740 [Ken Takagiwa] added doctest for pyspark.streaming.duration
4b69fb1 [Ken Takagiwa] fied input of socketTextDStream
02f618a [Ken Takagiwa] initial commit for socketTextStream
4ce4058 [Ken Takagiwa] remove unused import in python
856d98e [Ken Takagiwa] add empty line
490e338 [Ken Takagiwa] sorted the import following Spark coding convention
5594bd4 [Ken Takagiwa] revert pom.xml
2adca84 [Ken Takagiwa] remove not implemented DStream functions in python
e551e13 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
3758175 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
c5518b4 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
dcf243f [Ken Takagiwa] implementing transform function in Python
9af03f4 [Ken Takagiwa] added reducedByKey not working yet
6e0d9c7 [Ken Takagiwa] modify dstream.py to fix indent error
e497b9b [Ken Takagiwa] comment PythonDStream.PairwiseDStream
5c3a683 [Ken] initial commit for pySparkStreaming
665bfdb [giwa] added testcase for combineByKey
a3d2379 [giwa] added gorupByKey testcase
636090a [giwa] added sparkContext as input parameter in StreamingContext
e7ebb08 [giwa] removed wasted print in DStream
d8b593b [giwa] add comments
ea9c873 [giwa] added TODO coments
89ae38a [giwa] added saveAsTextFiles and saveAsPickledFiles
e3033fc [giwa] remove waste duplicated code
a14c7e1 [giwa] modified streaming test case to add coment
536def4 [giwa] basic function test cases are passed
2112638 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
080541a [giwa] broke something
0704b86 [giwa] WIP: solved partitioned and None is not recognized
90a6484 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
a65f302 [giwa] edited the comment to add more precise description
bdde697 [giwa] removed unnesessary changes
e8c7bfc [giwa] remove export PYSPARK_PYTHON in spark submit
3334169 [giwa] fixed PEP-008 violation
db0a303 [giwa] delete waste file
2cfd3a0 [giwa] added basic operation test cases
90ae568 [giwa] WIP added test case
a120d07 [giwa] WIP
f671cdb [giwa] WIP: added PythonTestInputStream
56fae45 [giwa] WIP
e35e101 [giwa] Merge branch 'master' into testcase
ba5112d [giwa] update comment
28aa56d [giwa] WIP
fb08559 [giwa] initial commit for testcase
a613b85 [giwa] clean up dstream.py
c40c0ef [giwa] added stop in StreamingContext
31e4260 [giwa] clean up examples
d2127d6 [giwa] implemented reduce and count function in Dstream
48f7746 [giwa] Removed the waste line
0f83eaa [Ken Takagiwa] delete py4j 0.8.1
1679808 [Ken Takagiwa] Kill py4j callback server properly
f96cd4e [Ken Takagiwa] tried to restart callback server
fe86198 [Ken Takagiwa] add py4j 0.8.2.1 but server is not launched
1064fe0 [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark
28c6620 [Ken Takagiwa] Implemented DStream.foreachRDD in the Python API using Py4J callback server
85b0fe1 [Ken Takagiwa] Merge pull request #1 from tdas/python-foreach
54e2e8c [Tathagata Das] Added extra line.
e185338 [Tathagata Das] Added missing file
a778d4b [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
cc2092b [Ken Takagiwa] remove waste file
d042ac6 [Ken Takagiwa] clean up codes
84a021f [Ken Takagiwa] clean up code
bd20e17 [Ken Takagiwa] fix map function
d01a125 [Ken Takagiwa] added count operation but this implementation need double check
7d05109 [Ken Takagiwa] merge with remote branch
ae464e0 [Ken Takagiwa] edit python sparkstreaming example
04af046 [Ken Takagiwa] reduceByKey is working
3b6d7b0 [Ken Takagiwa] implementing transform function in Python
571d52d [Ken Takagiwa] added reducedByKey not working yet
5720979 [Ken Takagiwa] delete old file
e604fcb [Ken Takagiwa] fixed typo of network_workdcount.py
4b7c08b [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark
ce7d426 [Ken Takagiwa] added doctest for pyspark.streaming.duration
a8c9fd5 [Ken Takagiwa] fixed for socketTextStream
a61fa9e [Ken Takagiwa] fied input of socketTextDStream
1e84f41 [Ken Takagiwa] initial commit for socketTextStream
6d012f7 [Ken Takagiwa] remove unused import in python
25d30d5 [Ken Takagiwa] add empty line
6e0a64a [Ken Takagiwa] sorted the import following Spark coding convention
fa4a7fc [Ken Takagiwa] revert streaming/pom.xml
8f8202b [Ken Takagiwa] revert streaming pom.xml
c9d79dd [Ken Takagiwa] revert pom.xml
57e3e52 [Ken Takagiwa] remove not implemented DStream functions in python
0a516f5 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
a7a0b5c [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
72bfc66 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
69e9cd3 [Ken Takagiwa] implementing transform function in Python
94a0787 [Ken Takagiwa] added reducedByKey not working yet
88068cf [Ken Takagiwa] modify dstream.py to fix indent error
1367be5 [Ken Takagiwa] comment PythonDStream.PairwiseDStream
eb2b3ba [Ken] Merge remote-tracking branch 'upstream/master'
d8e51f9 [Ken] initial commit for pySparkStreaming
2014-10-12 02:46:56 -07:00
Davies Liu 72f36ee571 [SPARK-3886] [PySpark] use AutoBatchedSerializer by default
Use AutoBatchedSerializer by default, which will choose the proper batch size based on size of serialized objects, let the size of serialized batch fall in into  [64k - 640k].

In JVM, the serializer will also track the objects in batch to figure out duplicated objects, larger batch may cause OOM in JVM.

Author: Davies Liu <davies.liu@gmail.com>

Closes #2740 from davies/batchsize and squashes the following commits:

52cdb88 [Davies Liu] update docs
185f2b9 [Davies Liu] use AutoBatchedSerializer by default
2014-10-10 14:14:05 -07:00
Davies Liu 798ed22c28 [SPARK-3412] [PySpark] Replace Epydoc with Sphinx to generate Python API docs
Retire Epydoc, use Sphinx to generate API docs.

Refine Sphinx docs, also convert some docstrings into Sphinx style.

It looks like:
![api doc](https://cloud.githubusercontent.com/assets/40902/4538272/9e2d4f10-4dec-11e4-8d96-6e45a8fe51f9.png)

Author: Davies Liu <davies.liu@gmail.com>

Closes #2689 from davies/docs and squashes the following commits:

bf4a0a5 [Davies Liu] fix links
3fb1572 [Davies Liu] fix _static in jekyll
65a287e [Davies Liu] fix scripts and logo
8524042 [Davies Liu] Merge branch 'master' of github.com:apache/spark into docs
d5b874a [Davies Liu] Merge branch 'master' of github.com:apache/spark into docs
4bc1c3c [Davies Liu] refactor
746d0b6 [Davies Liu] @param -> :param
240b393 [Davies Liu] replace epydoc with sphinx doc
2014-10-07 18:09:27 -07:00