Commit graph

63 commits

Author SHA1 Message Date
Benjamin Peterson 7013eea11c [SPARK-23522][PYTHON] always use sys.exit over builtin exit
The exit() builtin is only for interactive use. applications should use sys.exit().

## What changes were proposed in this pull request?

All usage of the builtin `exit()` function is replaced by `sys.exit()`.

## How was this patch tested?

I ran `python/run-tests`.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Benjamin Peterson <benjamin@python.org>

Closes #20682 from benjaminp/sys-exit.
2018-03-08 20:38:34 +09:00
Li Jin 2cb23a8f51 [SPARK-23011][SQL][PYTHON] Support alternative function form with group aggregate pandas UDF
## What changes were proposed in this pull request?

This PR proposes to support an alternative function from with group aggregate pandas UDF.

The current form:
```
def foo(pdf):
    return ...
```
Takes a single arg that is a pandas DataFrame.

With this PR, an alternative form is supported:
```
def foo(key, pdf):
    return ...
```
The alternative form takes two argument - a tuple that presents the grouping key, and a pandas DataFrame represents the data.

## How was this patch tested?

GroupbyApplyTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20295 from icexelloss/SPARK-23011-groupby-apply-key.
2018-03-08 20:29:07 +09:00
hyukjinkwon c338c8cf82 [SPARK-23352][PYTHON] Explicitly specify supported types in Pandas UDFs
## What changes were proposed in this pull request?

This PR targets to explicitly specify supported types in Pandas UDFs.
The main change here is to add a deduplicated and explicit type checking in `returnType` ahead with documenting this; however, it happened to fix multiple things.

1. Currently, we don't support `BinaryType` in Pandas UDFs, for example, see:

    ```python
    from pyspark.sql.functions import pandas_udf
    pudf = pandas_udf(lambda x: x, "binary")
    df = spark.createDataFrame([[bytearray(1)]])
    df.select(pudf("_1")).show()
    ```
    ```
    ...
    TypeError: Unsupported type in conversion to Arrow: BinaryType
    ```

    We can document this behaviour for its guide.

2. Also, the grouped aggregate Pandas UDF fails fast on `ArrayType` but seems we can support this case.

    ```python
    from pyspark.sql.functions import pandas_udf, PandasUDFType
    foo = pandas_udf(lambda v: v.mean(), 'array<double>', PandasUDFType.GROUPED_AGG)
    df = spark.range(100).selectExpr("id", "array(id) as value")
    df.groupBy("id").agg(foo("value")).show()
    ```

    ```
    ...
     NotImplementedError: ArrayType, StructType and MapType are not supported with PandasUDFType.GROUPED_AGG
    ```

3. Since we can check the return type ahead, we can fail fast before actual execution.

    ```python
    # we can fail fast at this stage because we know the schema ahead
    pandas_udf(lambda x: x, BinaryType())
    ```

## How was this patch tested?

Manually tested and unit tests for `BinaryType` and `ArrayType(...)` were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20531 from HyukjinKwon/pudf-cleanup.
2018-02-12 20:49:36 +09:00
gatorsmile 7a2ada223e [SPARK-23261][PYSPARK] Rename Pandas UDFs
## What changes were proposed in this pull request?
Rename the public APIs and names of pandas udfs.

- `PANDAS SCALAR UDF` -> `SCALAR PANDAS UDF`
- `PANDAS GROUP MAP UDF` -> `GROUPED MAP PANDAS UDF`
- `PANDAS GROUP AGG UDF` -> `GROUPED AGG PANDAS UDF`

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20428 from gatorsmile/renamePandasUDFs.
2018-01-30 21:55:55 +09:00
Li Jin b2ce17b4c9 [SPARK-22274][PYTHON][SQL] User-defined aggregation functions with pandas udf (full shuffle)
## What changes were proposed in this pull request?

Add support for using pandas UDFs with groupby().agg().

This PR introduces a new type of pandas UDF - group aggregate pandas UDF. This type of UDF defines a transformation of multiple pandas Series -> a scalar value. Group aggregate pandas UDFs can be used with groupby().agg(). Note group aggregate pandas UDF doesn't support partial aggregation, i.e., a full shuffle is required.

This PR doesn't support group aggregate pandas UDFs that return ArrayType, StructType or MapType. Support for these types is left for future PR.

## How was this patch tested?

GroupbyAggPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19872 from icexelloss/SPARK-22274-groupby-agg.
2018-01-23 14:11:30 +09:00
Takuya UESHIN 64817c423c [SPARK-22395][SQL][PYTHON] Fix the behavior of timestamp values for Pandas to respect session timezone
## What changes were proposed in this pull request?

When converting Pandas DataFrame/Series from/to Spark DataFrame using `toPandas()` or pandas udfs, timestamp values behave to respect Python system timezone instead of session timezone.

For example, let's say we use `"America/Los_Angeles"` as session timezone and have a timestamp value `"1970-01-01 00:00:01"` in the timezone. Btw, I'm in Japan so Python timezone would be `"Asia/Tokyo"`.

The timestamp value from current `toPandas()` will be the following:

```
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
>>> df = spark.createDataFrame([28801], "long").selectExpr("timestamp(value) as ts")
>>> df.show()
+-------------------+
|                 ts|
+-------------------+
|1970-01-01 00:00:01|
+-------------------+

>>> df.toPandas()
                   ts
0 1970-01-01 17:00:01
```

As you can see, the value becomes `"1970-01-01 17:00:01"` because it respects Python timezone.
As we discussed in #18664, we consider this behavior is a bug and the value should be `"1970-01-01 00:00:01"`.

## How was this patch tested?

Added tests and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19607 from ueshin/issues/SPARK-22395.
2017-11-28 16:45:22 +08:00
Li Jin 7d039e0c0a [SPARK-22409] Introduce function type argument in pandas_udf
## What changes were proposed in this pull request?

* Add a "function type" argument to pandas_udf.
* Add a new public enum class `PandasUdfType` in pyspark.sql.functions
* Refactor udf related code from pyspark.sql.functions to pyspark.sql.udf
* Merge "PythonUdfType" and "PythonEvalType" into a single enum class "PythonEvalType"

Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('double', PandasUDFType.SCALAR):
def plus_one(v):
    return v + 1
```

## Design doc
https://docs.google.com/document/d/1KlLaa-xJ3oz28xlEJqXyCAHU3dwFYkFs_ixcUXrJNTc/edit

## How was this patch tested?

Added PandasUDFTests

## TODO:
* [x] Implement proper enum type for `PandasUDFType`
* [x] Update documentation
* [x] Add more tests in PandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19630 from icexelloss/spark-22409-pandas-udf-type.
2017-11-17 16:43:08 +01:00
Takuya UESHIN b8624b06e5 [SPARK-20396][SQL][PYSPARK][FOLLOW-UP] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This is a follow-up of #18732.
This pr modifies `GroupedData.apply()` method to convert pandas udf to grouped udf implicitly.

## How was this patch tested?

Exisiting tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19517 from ueshin/issues/SPARK-20396/fup2.
2017-10-20 12:44:30 -07:00
Li Jin bfc7e1fe1a [SPARK-20396][SQL][PYSPARK] groupby().apply() with pandas udf
## What changes were proposed in this pull request?

This PR adds an apply() function on df.groupby(). apply() takes a pandas udf that is a transformation on `pandas.DataFrame` -> `pandas.DataFrame`.

Static schema
-------------------
```
schema = df.schema

pandas_udf(schema)
def normalize(df):
    df = df.assign(v1 = (df.v1 - df.v1.mean()) / df.v1.std()
    return df

df.groupBy('id').apply(normalize)
```
Dynamic schema
-----------------------
**This use case is removed from the PR and we will discuss this as a follow up. See discussion https://github.com/apache/spark/pull/18732#pullrequestreview-66583248**

Another example to use pd.DataFrame dtypes as output schema of the udf:

```
sample_df = df.filter(df.id == 1).toPandas()

def foo(df):
      ret = # Some transformation on the input pd.DataFrame
      return ret

foo_udf = pandas_udf(foo, foo(sample_df).dtypes)

df.groupBy('id').apply(foo_udf)
```
In interactive use case, user usually have a sample pd.DataFrame to test function `foo` in their notebook. Having been able to use `foo(sample_df).dtypes` frees user from specifying the output schema of `foo`.

Design doc: https://github.com/icexelloss/spark/blob/pandas-udf-doc/docs/pyspark-pandas-udf.md

## How was this patch tested?
* Added GroupbyApplyTest

Author: Li Jin <ice.xelloss@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>
Author: Bryan Cutler <cutlerb@gmail.com>

Closes #18732 from icexelloss/groupby-apply-SPARK-20396.
2017-10-11 07:32:01 +09:00
Takuya UESHIN 09cbf3df20 [SPARK-22125][PYSPARK][SQL] Enable Arrow Stream format for vectorized UDF.
## What changes were proposed in this pull request?

Currently we use Arrow File format to communicate with Python worker when invoking vectorized UDF but we can use Arrow Stream format.

This pr replaces the Arrow File format with the Arrow Stream format.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19349 from ueshin/issues/SPARK-22125.
2017-09-27 23:21:44 +09:00
Bryan Cutler d8e825e3bc [SPARK-22106][PYSPARK][SQL] Disable 0-parameter pandas_udf and add doctests
## What changes were proposed in this pull request?

This change disables the use of 0-parameter pandas_udfs due to the API being overly complex and awkward, and can easily be worked around by using an index column as an input argument.  Also added doctests for pandas_udfs which revealed bugs for handling empty partitions and using the pandas_udf decorator.

## How was this patch tested?

Reworked existing 0-parameter test to verify error is raised, added doctest for pandas_udf, added new tests for empty partition and decorator usage.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19325 from BryanCutler/arrow-pandas_udf-0-param-remove-SPARK-22106.
2017-09-26 10:54:00 +09:00
Bryan Cutler 27fc536d9a [SPARK-21190][PYSPARK] Python Vectorized UDFs
This PR adds vectorized UDFs to the Python API

**Proposed API**
Introduce a flag to turn on vectorization for a defined UDF, for example:

```
pandas_udf(DoubleType())
def plus(a, b)
    return a + b
```
or

```
plus = pandas_udf(lambda a, b: a + b, DoubleType())
```
Usage is the same as normal UDFs

0-parameter UDFs
pandas_udf functions can declare an optional `**kwargs` and when evaluated, will contain a key "size" that will give the required length of the output.  For example:

```
pandas_udf(LongType())
def f0(**kwargs):
    return pd.Series(1).repeat(kwargs["size"])

df.select(f0())
```

Added new unit tests in pyspark.sql that are enabled if pyarrow and Pandas are available.

- [x] Fix support for promoted types with null values
- [ ] Discuss 0-param UDF API (use of kwargs)
- [x] Add tests for chained UDFs
- [ ] Discuss behavior when pyarrow not installed / enabled
- [ ] Cleanup pydoc and add user docs

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18659 from BryanCutler/arrow-vectorized-udfs-SPARK-21404.
2017-09-22 16:17:50 +08:00
Josh Rosen 8ddbc431d8 [SPARK-20685] Fix BatchPythonEvaluation bug in case of single UDF w/ repeated arg.
## What changes were proposed in this pull request?

There's a latent corner-case bug in PySpark UDF evaluation where executing a `BatchPythonEvaluation` with a single multi-argument UDF where _at least one argument value is repeated_ will crash at execution with a confusing error.

This problem was introduced in #12057: the code there has a fast path for handling a "batch UDF evaluation consisting of a single Python UDF", but that branch incorrectly assumes that a single UDF won't have repeated arguments and therefore skips the code for unpacking arguments from the input row (whose schema may not necessarily match the UDF inputs due to de-duplication of repeated arguments which occurred in the JVM before sending UDF inputs to Python).

This fix here is simply to remove this special-casing: it turns out that the code in the "multiple UDFs" branch just so happens to work for the single-UDF case because Python treats `(x)` as equivalent to `x`, not as a single-argument tuple.

## How was this patch tested?

New regression test in `pyspark.python.sql.tests` module (tested and confirmed that it fails before my fix).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17927 from JoshRosen/SPARK-20685.
2017-05-10 16:50:57 -07:00
Holden Karau 047a9d92ca [SPARK-18576][PYTHON] Add basic TaskContext information to PySpark
## What changes were proposed in this pull request?

Adds basic TaskContext information to PySpark.

## How was this patch tested?

New unit tests to `tests.py` & existing unit tests.

Author: Holden Karau <holden@us.ibm.com>

Closes #16211 from holdenk/SPARK-18576-pyspark-taskcontext.
2016-12-20 15:51:21 -08:00
Liang-Chi Hsieh cc86fcd0d6
[MINOR][PYSPARK] Improve error message when running PySpark with different minor versions
## What changes were proposed in this pull request?

Currently the error message is correct but doesn't provide additional hint to new users. It would be better to hint related configuration to users in the message.

## How was this patch tested?

N/A because it only changes error message.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15822 from viirya/minor-pyspark-worker-errmsg.
2016-11-10 10:23:45 +00:00
Davies Liu f0afafdc5d [SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?

This PR support multiple Python UDFs within single batch, also improve the performance.

```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$

== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
   +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
      +- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
         +- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
            +- OneRowRelation$

== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
   +- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
      +- OneRowRelation$

== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
:     +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
   +- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
      +- Scan OneRowRelation[]
```

## How was this patch tested?

Added new tests.

Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:

N | Before | After  | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s |  3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X

This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).

Author: Davies Liu <davies@databricks.com>

Closes #12057 from davies/multi_udfs.
2016-03-31 16:40:20 -07:00
Davies Liu a7a93a116d [SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?

This PR brings the support for chained Python UDFs, for example

```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```

Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.

For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF#10 AS double(double(1))#9]
:     +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
   +- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
:     +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
   +- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
      +- !BatchPythonEvaluation double(1), [pythonUDF#17]
         +- Scan OneRowRelation[]
```

TODO: will support multiple unrelated Python UDFs in one batch (another PR).

## How was this patch tested?

Added new unit tests for chained UDFs.

Author: Davies Liu <davies@databricks.com>

Closes #12014 from davies/py_udfs.
2016-03-29 15:06:29 -07:00
Davies Liu 693949ba40 [SPARK-8976] [PYSPARK] fix open mode in python3
This bug only happen on Python 3 and Windows.

I tested this manually with python 3 and disable python daemon, no unit test yet.

Author: Davies Liu <davies@databricks.com>

Closes #8181 from davies/open_mode.
2015-08-13 17:33:37 -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
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 4740d6a158 [SPARK-6216] [PySpark] check the python version in worker
Author: Davies Liu <davies@databricks.com>

Closes #5404 from davies/check_version and squashes the following commits:

e559248 [Davies Liu] add tests
ec33b5f [Davies Liu] check the python version in worker
2015-04-10 14:04:53 -07:00
Josh Rosen ee6e3eff02 Revert "[SPARK-5363] [PySpark] check ending mark in non-block way"
This reverts commits ac6fe67e1d and c06e42f2c1.
2015-02-17 07:49:02 -08:00
Davies Liu ac6fe67e1d [SPARK-5363] [PySpark] check ending mark in non-block way
There is chance of dead lock that the Python process is waiting for ending mark from JVM, but which is eaten by corrupted stream.

This PR checks the ending mark from Python in non-block way, so it will not blocked by Python process.

There is a small chance that the ending mark is sent by Python process but not available right now, then Python process will not be used.

cc JoshRosen pwendell

Author: Davies Liu <davies@databricks.com>

Closes #4601 from davies/freeze and squashes the following commits:

e15a8c3 [Davies Liu] update logging
890329c [Davies Liu] Merge branch 'freeze' of github.com:davies/spark into freeze
2bd2228 [Davies Liu] add more logging
656d544 [Davies Liu] Update PythonRDD.scala
05e1085 [Davies Liu] check ending mark in non-block way
2015-02-16 20:32:03 -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
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
Davies Liu e595c8d08a [SPARK-3993] [PySpark] fix bug while reuse worker after take()
After take(), maybe there are some garbage left in the socket, then next task assigned to this worker will hang because of corrupted data.

We should make sure the socket is clean before reuse it, write END_OF_STREAM at the end, and check it after read out all result from python.

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

Closes #2838 from davies/fix_reuse and squashes the following commits:

8872914 [Davies Liu] fix tests
660875b [Davies Liu] fix bug while reuse worker after take()
2014-10-23 17:20:00 -07:00
Davies Liu c5414b6818 [SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:

```
============================================================
Profile of RDD<id=3>
============================================================
         5146507 function calls (5146487 primitive calls) in 71.094 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
  5144576   68.331    0.000   68.331    0.000 statcounter.py:44(merge)
       20    2.735    0.137   71.071    3.554 statcounter.py:33(__init__)
       20    0.017    0.001    0.017    0.001 {cPickle.dumps}
     1024    0.003    0.000    0.003    0.000 t.py:16(<lambda>)
       20    0.001    0.000    0.001    0.000 {reduce}
       21    0.001    0.000    0.001    0.000 {cPickle.loads}
       20    0.001    0.000    0.001    0.000 copy_reg.py:95(_slotnames)
       41    0.001    0.000    0.001    0.000 serializers.py:461(read_int)
       40    0.001    0.000    0.002    0.000 serializers.py:179(_batched)
       62    0.000    0.000    0.000    0.000 {method 'read' of 'file' objects}
       20    0.000    0.000   71.072    3.554 rdd.py:863(<lambda>)
       20    0.000    0.000    0.001    0.000 serializers.py:198(load_stream)
    40/20    0.000    0.000   71.072    3.554 rdd.py:2093(pipeline_func)
       41    0.000    0.000    0.002    0.000 serializers.py:130(load_stream)
       40    0.000    0.000   71.072    1.777 rdd.py:304(func)
       20    0.000    0.000   71.094    3.555 worker.py:82(process)
```

Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as

```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
         284 function calls (276 primitive calls) in 0.001 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
        4    0.000    0.000    0.000    0.000 serializers.py:198(load_stream)
        4    0.000    0.000    0.000    0.000 {reduce}
     12/4    0.000    0.000    0.001    0.000 rdd.py:2092(pipeline_func)
        4    0.000    0.000    0.000    0.000 {cPickle.loads}
        4    0.000    0.000    0.000    0.000 {cPickle.dumps}
      104    0.000    0.000    0.000    0.000 rdd.py:852(<genexpr>)
        8    0.000    0.000    0.000    0.000 serializers.py:461(read_int)
       12    0.000    0.000    0.000    0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".

Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"

This is bugfix of #2351 cc JoshRosen

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

Closes #2556 from davies/profiler and squashes the following commits:

e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
858e74c [Davies Liu] compatitable with python 2.6
7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-30 18:24:57 -07:00
Josh Rosen f872e4fb80 Revert "[SPARK-3478] [PySpark] Profile the Python tasks"
This reverts commit 1aa549ba98.
2014-09-26 14:47:14 -07:00
Davies Liu 1aa549ba98 [SPARK-3478] [PySpark] Profile the Python tasks
This patch add profiling support for PySpark, it will show the profiling results
before the driver exits, here is one example:

```
============================================================
Profile of RDD<id=3>
============================================================
         5146507 function calls (5146487 primitive calls) in 71.094 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
  5144576   68.331    0.000   68.331    0.000 statcounter.py:44(merge)
       20    2.735    0.137   71.071    3.554 statcounter.py:33(__init__)
       20    0.017    0.001    0.017    0.001 {cPickle.dumps}
     1024    0.003    0.000    0.003    0.000 t.py:16(<lambda>)
       20    0.001    0.000    0.001    0.000 {reduce}
       21    0.001    0.000    0.001    0.000 {cPickle.loads}
       20    0.001    0.000    0.001    0.000 copy_reg.py:95(_slotnames)
       41    0.001    0.000    0.001    0.000 serializers.py:461(read_int)
       40    0.001    0.000    0.002    0.000 serializers.py:179(_batched)
       62    0.000    0.000    0.000    0.000 {method 'read' of 'file' objects}
       20    0.000    0.000   71.072    3.554 rdd.py:863(<lambda>)
       20    0.000    0.000    0.001    0.000 serializers.py:198(load_stream)
    40/20    0.000    0.000   71.072    3.554 rdd.py:2093(pipeline_func)
       41    0.000    0.000    0.002    0.000 serializers.py:130(load_stream)
       40    0.000    0.000   71.072    1.777 rdd.py:304(func)
       20    0.000    0.000   71.094    3.555 worker.py:82(process)
```

Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk
by `sc.dump_profiles(path)`, such as

```python
>>> sc._conf.set("spark.python.profile", "true")
>>> rdd = sc.parallelize(range(100)).map(str)
>>> rdd.count()
100
>>> sc.show_profiles()
============================================================
Profile of RDD<id=1>
============================================================
         284 function calls (276 primitive calls) in 0.001 seconds

   Ordered by: internal time, cumulative time

   ncalls  tottime  percall  cumtime  percall filename:lineno(function)
        4    0.000    0.000    0.000    0.000 serializers.py:198(load_stream)
        4    0.000    0.000    0.000    0.000 {reduce}
     12/4    0.000    0.000    0.001    0.000 rdd.py:2092(pipeline_func)
        4    0.000    0.000    0.000    0.000 {cPickle.loads}
        4    0.000    0.000    0.000    0.000 {cPickle.dumps}
      104    0.000    0.000    0.000    0.000 rdd.py:852(<genexpr>)
        8    0.000    0.000    0.000    0.000 serializers.py:461(read_int)
       12    0.000    0.000    0.000    0.000 rdd.py:303(func)
```
The profiling is disabled by default, can be enabled by "spark.python.profile=true".

Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump"

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

Closes #2351 from davies/profiler and squashes the following commits:

7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles()
2b0daf2 [Davies Liu] fix docs
7a56c24 [Davies Liu] bugfix
cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext
fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler
09d02c3 [Davies Liu] Merge branch 'master' into profiler
c23865c [Davies Liu] Merge branch 'master' into profiler
15d6f18 [Davies Liu] add docs for two configs
dadee1a [Davies Liu] add docs string and clear profiles after show or dump
4f8309d [Davies Liu] address comment, add tests
0a5b6eb [Davies Liu] fix Python UDF
4b20494 [Davies Liu] add profile for python
2014-09-26 09:27:42 -07:00
Davies Liu c854b9fcb5 [SPARK-3634] [PySpark] User's module should take precedence over system modules
Python modules added through addPyFile should take precedence over system modules.

This patch put the path for user added module in the front of sys.path (just after '').

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

Closes #2492 from davies/path and squashes the following commits:

4a2af78 [Davies Liu] fix tests
f7ff4da [Davies Liu] ad license header
6b0002f [Davies Liu] add tests
c16c392 [Davies Liu] put addPyFile in front of sys.path
2014-09-24 12:10:09 -07:00
Davies Liu e77fa81a61 [SPARK-3554] [PySpark] use broadcast automatically for large closure
Py4j can not handle large string efficiently, so we should use broadcast for large closure automatically. (Broadcast use local filesystem to pass through data).

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

Closes #2417 from davies/command and squashes the following commits:

fbf4e97 [Davies Liu] bugfix
aefd508 [Davies Liu] use broadcast automatically for large closure
2014-09-18 18:11:48 -07:00
Davies Liu 4e3fbe8cdb [SPARK-3463] [PySpark] aggregate and show spilled bytes in Python
Aggregate the number of bytes spilled into disks during aggregation or sorting, show them in Web UI.

![spilled](https://cloud.githubusercontent.com/assets/40902/4209758/4b995562-386d-11e4-97c1-8e838ee1d4e3.png)

This patch is blocked by SPARK-3465. (It includes a fix for that).

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

Closes #2336 from davies/metrics and squashes the following commits:

e37df38 [Davies Liu] remove outdated comments
1245eb7 [Davies Liu] remove the temporary fix
ebd2f43 [Davies Liu] Merge branch 'master' into metrics
7e4ad04 [Davies Liu] Merge branch 'master' into metrics
fbe9029 [Davies Liu] show spilled bytes in Python in web ui
2014-09-13 22:31:21 -07:00
Davies Liu 2aea0da84c [SPARK-3030] [PySpark] Reuse Python worker
Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts.

This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming.

For a job with broadcast (43M after compress):
```
    b = sc.broadcast(set(range(30000000)))
    print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count()
```
It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks.

It's enabled by default, could be disabled by `spark.python.worker.reuse = false`.

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

Closes #2259 from davies/reuse-worker and squashes the following commits:

f11f617 [Davies Liu] Merge branch 'master' into reuse-worker
3939f20 [Davies Liu] fix bug in serializer in mllib
cf1c55e [Davies Liu] address comments
3133a60 [Davies Liu] fix accumulator with reused worker
760ab1f [Davies Liu] do not reuse worker if there are any exceptions
7abb224 [Davies Liu] refactor: sychronized with itself
ac3206e [Davies Liu] renaming
8911f44 [Davies Liu] synchronized getWorkerBroadcasts()
6325fc1 [Davies Liu] bugfix: bid >= 0
e0131a2 [Davies Liu] fix name of config
583716e [Davies Liu] only reuse completed and not interrupted worker
ace2917 [Davies Liu] kill python worker after timeout
6123d0f [Davies Liu] track broadcasts for each worker
8d2f08c [Davies Liu] reuse python worker
2014-09-13 16:22:04 -07:00
Josh Rosen 1f1819b20f [SPARK-3114] [PySpark] Fix Python UDFs in Spark SQL.
This fixes SPARK-3114, an issue where we inadvertently broke Python UDFs in Spark SQL.

This PR modifiers the test runner script to always run the PySpark SQL tests, irrespective of whether SparkSQL itself has been modified.  It also includes Davies' fix for the bug.

Closes #2026.

Author: Josh Rosen <joshrosen@apache.org>
Author: Davies Liu <davies.liu@gmail.com>

Closes #2027 from JoshRosen/pyspark-sql-fix and squashes the following commits:

9af2708 [Davies Liu] bugfix: disable compression of command
0d8d3a4 [Josh Rosen] Always run Python Spark SQL tests.
2014-08-18 20:42:19 -07:00
Davies Liu 2fc8aca086 [SPARK-1065] [PySpark] improve supporting for large broadcast
Passing large object by py4j is very slow (cost much memory), so pass broadcast objects via files (similar to parallelize()).

Add an option to keep object in driver (it's False by default) to save memory in driver.

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

Closes #1912 from davies/broadcast and squashes the following commits:

e06df4a [Davies Liu] load broadcast from disk in driver automatically
db3f232 [Davies Liu] fix serialization of accumulator
631a827 [Davies Liu] Merge branch 'master' into broadcast
c7baa8c [Davies Liu] compress serrialized broadcast and command
9a7161f [Davies Liu] fix doc tests
e93cf4b [Davies Liu] address comments: add test
6226189 [Davies Liu] improve large broadcast
2014-08-16 16:59:34 -07:00
Davies Liu ccd5ab5f82 [SPARK-2580] [PySpark] keep silent in worker if JVM close the socket
During rdd.take(n), JVM will close the socket if it had got enough data, the Python worker should keep silent in this case.

In the same time, the worker should not print the trackback into stderr if it send the traceback to JVM successfully.

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

Closes #1625 from davies/error and squashes the following commits:

4fbcc6d [Davies Liu] disable log4j during testing when exception is expected.
cc14202 [Davies Liu] keep silent in worker if JVM close the socket
2014-07-29 00:15:45 -07:00
Nicholas Chammas 5d16d5bbfd [SPARK-2470] PEP8 fixes to PySpark
This pull request aims to resolve all outstanding PEP8 violations in PySpark.

Author: Nicholas Chammas <nicholas.chammas@gmail.com>
Author: nchammas <nicholas.chammas@gmail.com>

Closes #1505 from nchammas/master and squashes the following commits:

98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle
cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses
e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break
9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses
22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses
24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest
7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py
8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py
b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py
d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py
aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py
1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py
95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py
a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib
c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py
d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py
81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py
1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py
7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py
ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py
f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py
a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py
f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py
4dd148f [nchammas] Merge pull request #5 from apache/master
f7e4581 [Nicholas Chammas] unrelated pep8 fix
a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently
de7292a [nchammas] Merge pull request #4 from apache/master
2e4fe00 [nchammas] Merge pull request #3 from apache/master
89fde08 [nchammas] Merge pull request #2 from apache/master
69f6e22 [Nicholas Chammas] PEP8 fixes
2627247 [Nicholas Chammas] broke up lines before they hit 100 chars
6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names
69da6cf [nchammas] Merge pull request #1 from apache/master
2014-07-21 22:30:53 -07:00
Bouke van der Bijl 3776f2f283 Add Python includes to path before depickling broadcast values
This fixes https://issues.apache.org/jira/browse/SPARK-1731 by adding the Python includes to the PYTHONPATH before depickling the broadcast values

@airhorns

Author: Bouke van der Bijl <boukevanderbijl@gmail.com>

Closes #656 from bouk/python-includes-before-broadcast and squashes the following commits:

7b0dfe4 [Bouke van der Bijl] Add Python includes to path before depickling broadcast values
2014-05-10 13:02:13 -07:00
Bouke van der Bijl 12738c1aec SPARK-1115: Catch depickling errors
This surroungs the complete worker code in a try/except block so we catch any error that arrives. An example would be the depickling failing for some reason

@JoshRosen

Author: Bouke van der Bijl <boukevanderbijl@gmail.com>

Closes #644 from bouk/catch-depickling-errors and squashes the following commits:

f0f67cc [Bouke van der Bijl] Lol indentation
0e4d504 [Bouke van der Bijl] Surround the complete python worker with the try block
2014-02-26 14:51:21 -08:00
jyotiska 3ff077d489 Fixed minor typo in worker.py
Fixed minor typo in worker.py

Author: jyotiska <jyotiska123@gmail.com>

Closes #630 from jyotiska/pyspark_code and squashes the following commits:

ee44201 [jyotiska] typo fixed in worker.py
2014-02-22 10:09:50 -08:00
Josh Rosen 1381fc72f7 Switch from MUTF8 to UTF8 in PySpark serializers.
This fixes SPARK-1043, a bug introduced in 0.9.0
where PySpark couldn't serialize strings > 64kB.

This fix was written by @tyro89 and @bouk in #512.
This commit squashes and rebases their pull request
in order to fix some merge conflicts.
2014-01-28 20:20:08 -08:00
Matei Zaharia 5741078c46 Log Python exceptions to stderr as well
This helps in case the exception happened while serializing a record to
be sent to Java, leaving the stream to Java in an inconsistent state
where PythonRDD won't be able to read the error.
2014-01-12 00:10:41 -08:00
Josh Rosen 13122ceb8c FramedSerializer: _dumps => dumps, _loads => loads. 2013-11-10 17:53:25 -08:00
Josh Rosen ffa5bedf46 Send PySpark commands as bytes insetad of strings. 2013-11-10 16:46:00 -08:00
Josh Rosen cbb7f04aef Add custom serializer support to PySpark.
For now, this only adds MarshalSerializer, but it lays the groundwork
for other supporting custom serializers.  Many of these mechanisms
can also be used to support deserialization of different data formats
sent by Java, such as data encoded by MsgPack.

This also fixes a bug in SparkContext.union().
2013-11-10 16:45:38 -08:00
Josh Rosen 7d68a81a8e Remove Pickle-wrapping of Java objects in PySpark.
If we support custom serializers, the Python
worker will know what type of input to expect,
so we won't need to wrap Tuple2 and Strings into
pickled tuples and strings.
2013-11-03 11:03:02 -08:00
Josh Rosen a48d88d206 Replace magic lengths with constants in PySpark.
Write the length of the accumulators section up-front rather
than terminating it with a negative length.  I find this
easier to read.
2013-11-03 10:54:24 -08:00
Matei Zaharia 6550e5e60c Allow PySpark to launch worker.py directly on Windows 2013-09-01 18:06:15 -07:00
Andre Schumacher c7e348faec Implementing SPARK-878 for PySpark: adding zip and egg files to context and passing it down to workers which add these to their sys.path 2013-08-16 11:58:20 -07:00