Commit graph

92 commits

Author SHA1 Message Date
Liang-Chi Hsieh 2bc3fff13b [SPARK-29341][PYTHON] Upgrade cloudpickle to 1.0.0
### What changes were proposed in this pull request?

This patch upgrades cloudpickle to 1.0.0 version.

Main changes:

1. cleanup unused functions: 936f16fac8
2. Fix relative imports inside function body: 31ecdd6f57
3. Write kw only arguments to pickle: 6cb4718528

### Why are the changes needed?

We should include new bug fix like 6cb4718528, because users might use such python function in PySpark.

```python
>>> def f(a, *, b=1):
...   return a + b
...
>>> rdd = sc.parallelize([1, 2, 3])
>>> rdd.map(f).collect()
[Stage 0:>                                                        (0 + 12) / 12]19/10/03 00:42:24 ERROR Executor: Exception in task 3.0 in stage 0.0 (TID 3)
org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/spark/python/lib/pyspark.zip/pyspark/worker.py", line 598, in main
    process()
  File "/spark/python/lib/pyspark.zip/pyspark/worker.py", line 590, in process
    serializer.dump_stream(out_iter, outfile)
  File "/spark/python/lib/pyspark.zip/pyspark/serializers.py", line 513, in dump_stream
    vs = list(itertools.islice(iterator, batch))
  File "/spark/python/lib/pyspark.zip/pyspark/util.py", line 99, in wrapper
    return f(*args, **kwargs)
TypeError: f() missing 1 required keyword-only argument: 'b'
```

After:

```python
>>> def f(a, *, b=1):
...   return a + b
...
>>> rdd = sc.parallelize([1, 2, 3])
>>> rdd.map(f).collect()
[2, 3, 4]
```

### Does this PR introduce any user-facing change?

Yes. This fixes two bugs when pickling Python functions.

### How was this patch tested?

Existing tests.

Closes #26009 from viirya/upgrade-cloudpickle.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-03 19:20:51 +09:00
Chris Martin 05988b256e [SPARK-27463][PYTHON] Support Dataframe Cogroup via Pandas UDFs
### What changes were proposed in this pull request?

Adds a new cogroup Pandas UDF.  This allows two grouped dataframes to be cogrouped together and apply a (pandas.DataFrame, pandas.DataFrame) -> pandas.DataFrame UDF to each cogroup.

**Example usage**

```
from pyspark.sql.functions import pandas_udf, PandasUDFType
df1 = spark.createDataFrame(
   [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)],
   ("time", "id", "v1"))

df2 = spark.createDataFrame(
   [(20000101, 1, "x"), (20000101, 2, "y")],
    ("time", "id", "v2"))

pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP)
   def asof_join(l, r):
      return pd.merge_asof(l, r, on="time", by="id")

df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show()

```

        +--------+---+---+---+
        |    time| id| v1| v2|
        +--------+---+---+---+
        |20000101|  1|1.0|  x|
        |20000102|  1|3.0|  x|
        |20000101|  2|2.0|  y|
        |20000102|  2|4.0|  y|
        +--------+---+---+---+

### How was this patch tested?

Added unit test test_pandas_udf_cogrouped_map

Closes #24981 from d80tb7/SPARK-27463-poc-arrow-stream.

Authored-by: Chris Martin <chris@cmartinit.co.uk>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-09-17 17:13:50 -07:00
HyukjinKwon fe75ff8bea [SPARK-28206][PYTHON] Remove the legacy Epydoc in PySpark API documentation
## What changes were proposed in this pull request?

Seems like we used to generate PySpark API documentation by Epydoc almost at the very first place (see 85b8f2c64f).

This fixes an actual issue:

Before:

![Screen Shot 2019-07-05 at 8 20 01 PM](https://user-images.githubusercontent.com/6477701/60720491-e9879180-9f65-11e9-9562-100830a456cd.png)

After:

![Screen Shot 2019-07-05 at 8 20 05 PM](https://user-images.githubusercontent.com/6477701/60720495-ec828200-9f65-11e9-8277-8f689e292cb0.png)

It seems apparently a bug within `epytext` plugin during the conversion between`param` and `:param` syntax. See also [Epydoc syntax](http://epydoc.sourceforge.net/manual-epytext.html).

Actually, Epydoc syntax violates [PEP-257](https://www.python.org/dev/peps/pep-0257/) IIRC and blocks us to enable some rules for doctest linter as well.

We should remove this legacy away and I guess Spark 3 is good timing to do it.

## How was this patch tested?

Manually built the doc and check each.

I had to manually find the Epydoc syntax by `git grep -r "{L"`, for instance.

Closes #25060 from HyukjinKwon/SPARK-28206.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-07-05 10:08:22 -07:00
Li Jin d0fbc4da3b [SPARK-28003][PYTHON] Allow NaT values when creating Spark dataframe from pandas with Arrow
## What changes were proposed in this pull request?

This patch removes `fillna(0)` when creating ArrowBatch from a pandas Series.

With `fillna(0)`, the original code would turn a timestamp type into object type, which pyarrow will complain later:
```
>>> s = pd.Series([pd.NaT, pd.Timestamp('2015-01-01')])
>>> s.dtypes
dtype('<M8[ns]')
>>> s.fillna(0)
0                      0
1    2015-01-01 00:00:00
dtype: object
```

## How was this patch tested?

Added `test_timestamp_nat`

Closes #24844 from icexelloss/SPARK-28003-arrow-nat.

Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-24 11:15:21 -07:00
Bryan Cutler 90f80395af [SPARK-28041][PYTHON] Increase minimum supported Pandas to 0.23.2
## What changes were proposed in this pull request?

This increases the minimum supported version of Pandas to 0.23.2. Using a lower version will raise an error `Pandas >= 0.23.2 must be installed; however, your version was 0.XX`. Also, a workaround for using pyarrow with Pandas 0.19.2 was removed.

## How was this patch tested?

Existing Tests

Closes #24867 from BryanCutler/pyspark-increase-min-pandas-SPARK-28041.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-18 09:10:58 +09:00
HyukjinKwon f984f6acfe Revert "[SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)"
## What changes were proposed in this pull request?

This PR reverts 9c4eb99c52 for the reasons below:

1. An alternative was not considered properly, https://github.com/apache/spark/pull/24734#issuecomment-500101639 https://github.com/apache/spark/pull/24734#issuecomment-500102340 https://github.com/apache/spark/pull/24734#issuecomment-499202982 - I opened a PR https://github.com/apache/spark/pull/24826

2. 9c4eb99c52 fixed timely flushing which behaviour is somewhat hacky and the timing isn't also guaranteed (in case each batch takes longer to process).

3. For pipelining for smaller batches, looks it's better to allow to configure buffer size rather than having another factor to flush

## How was this patch tested?

N/A

Closes #24827 from HyukjinKwon/revert-flush.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-09 08:28:31 -07:00
WeichenXu 9c4eb99c52 [SPARK-27870][SQL][PYSPARK] Flush batch timely for pandas UDF (for improving pandas UDFs pipeline)
## What changes were proposed in this pull request?

Flush batch timely for pandas UDF.

This could improve performance when multiple pandas UDF plans are pipelined.

When batch being flushed in time, downstream pandas UDFs will get pipelined as soon as possible, and pipeline will help hide the donwstream UDFs computation time. For example:

When the first UDF start computing on batch-3, the second pipelined UDF can start computing on batch-2, and the third pipelined UDF can start computing on batch-1.

If we do not flush each batch in time, the donwstream UDF's pipeline will lag behind too much, which may increase the total processing time.

I add flush at two places:
* JVM process feed data into python worker. In jvm side, when write one batch, flush it
* VM process read data from python worker output, In python worker side, when write one batch, flush it

If no flush, the default buffer size for them are both 65536. Especially in the ML case, in order to make realtime prediction, we will make batch size very small. The buffer size is too large for the case, which cause downstream pandas UDF pipeline lag behind too much.

### Note
* This is only applied to pandas scalar UDF.
* Do not flush for each batch. The minimum interval between two flush is 0.1 second. This avoid too frequent flushing when batch size is small. It works like:
```
        last_flush_time = time.time()
        for batch in iterator:
                writer.write_batch(batch)
                flush_time = time.time()
                if self.flush_timely and (flush_time - last_flush_time > 0.1):
                      stream.flush()
                      last_flush_time = flush_time
```

## How was this patch tested?

### Benchmark to make sure the flush do not cause performance regression
#### Test code:
```
numRows = ...
batchSize = ...

spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', str(batchSize))
df = spark.range(1, numRows + 1, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 10

beg_time = time.time()
result = df.select(sum(fp1('a'))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))
```
#### Test Result:

 params        | Consume time (Before) | Consume time (After)
------------ | ----------------------- | ----------------------
numRows=100000000, batchSize=10000 | 23.43s | 24.64s
numRows=100000000, batchSize=1000 | 36.73s | 34.50s
numRows=10000000, batchSize=100 | 35.67s | 32.64s
numRows=1000000, batchSize=10 | 33.60s | 32.11s
numRows=100000, batchSize=1 | 33.36s | 31.82s

### Benchmark pipelined pandas UDF
#### Test code:
```
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
df = spark.range(1, 31, numPartitions=1).select(col('id').alias('a'))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    print("run fp1")
    time.sleep(1)
    return x + 100

pandas_udf("int", PandasUDFType.SCALAR)
def fp2(x, y):
    print("run fp2")
    time.sleep(1)
    return x + y

beg_time = time.time()
result = df.select(sum(fp2(fp1('a'), col('a')))).head()
print("result: " + str(result[0]))
print("consume time: " + str(time.time() - beg_time))

```
#### Test Result:

**Before**: consume time: 63.57s
**After**: consume time: 32.43s
**So the PR improve performance by make downstream UDF get pipelined early.**

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

Closes #24734 from WeichenXu123/improve_pandas_udf_pipeline.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 14:02:43 -07:00
David Vogelbacher f9ca8ab196 [SPARK-27805][PYTHON] Propagate SparkExceptions during toPandas with arrow enabled
## What changes were proposed in this pull request?
Similar to https://github.com/apache/spark/pull/24070, we now propagate SparkExceptions that are encountered during the collect in the java process to the python process.

Fixes https://jira.apache.org/jira/browse/SPARK-27805

## How was this patch tested?
Added a new unit test

Closes #24677 from dvogelbacher/dv/betterErrorMsgWhenUsingArrow.

Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-04 10:10:27 -07:00
Liang-Chi Hsieh d9bcacf94b [SPARK-27629][PYSPARK] Prevent Unpickler from intervening each unpickling
## What changes were proposed in this pull request?

In SPARK-27612, one correctness issue was reported. When protocol 4 is used to pickle Python objects, we found that unpickled objects were wrong. A temporary fix was proposed by not using highest protocol.

It was found that Opcodes.MEMOIZE was appeared in the opcodes in protocol 4. It is suspect to this issue.

A deeper dive found that Opcodes.MEMOIZE stores objects into internal map of Unpickler object. We use single Unpickler object to unpickle serialized Python bytes. Stored objects intervenes next round of unpickling, if the map is not cleared.

We has two options:

1. Continues to reuse Unpickler, but calls its close after each unpickling.
2. Not to reuse Unpickler and create new Unpickler object in each unpickling.

This patch takes option 1.

## How was this patch tested?

Passing the test added in SPARK-27612 (#24519).

Closes #24521 from viirya/SPARK-27629.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 13:21:08 +09:00
HyukjinKwon 5c479243de [SPARK-27612][PYTHON] Use Python's default protocol instead of highest protocol
## What changes were proposed in this pull request?

This PR partially reverts https://github.com/apache/spark/pull/20691

After we changed the Python protocol to highest ones, seems like it introduced a correctness bug. This potentially affects all Python related code paths.

I suspect a bug related to Pryolite (maybe opcodes `MEMOIZE`, `FRAME` and/or our `RowPickler`). I would like to stick to default protocol for now and investigate the issue separately.

I will separately investigate later to bring highest protocol back.

## How was this patch tested?

Unittest was added.

```bash
./run-tests --python-executables=python3.7 --testname "pyspark.sql.tests.test_serde SerdeTests.test_int_array_serialization"
```

Closes #24519 from HyukjinKwon/SPARK-27612.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-03 14:40:13 +09:00
Bryan Cutler d36cce18e2 [SPARK-27276][PYTHON][SQL] Increase minimum version of pyarrow to 0.12.1 and remove prior workarounds
## What changes were proposed in this pull request?

This increases the minimum support version of pyarrow to 0.12.1 and removes workarounds in pyspark to remain compatible with prior versions. This means that users will need to have at least pyarrow 0.12.1 installed and available in the cluster or an `ImportError` will be raised to indicate an upgrade is needed.

## How was this patch tested?

Existing tests using:
Python 2.7.15, pyarrow 0.12.1, pandas 0.24.2
Python 3.6.7, pyarrow 0.12.1, pandas 0.24.0

Closes #24298 from BryanCutler/arrow-bump-min-pyarrow-SPARK-27276.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-22 19:30:31 +09:00
Takuya UESHIN 594be7a911 [SPARK-27240][PYTHON] Use pandas DataFrame for struct type argument in Scalar Pandas UDF.
## What changes were proposed in this pull request?

Now that we support returning pandas DataFrame for struct type in Scalar Pandas UDF.

If we chain another Pandas UDF after the Scalar Pandas UDF returning pandas DataFrame, the argument of the chained UDF will be pandas DataFrame, but currently we don't support pandas DataFrame as an argument of Scalar Pandas UDF. That means there is an inconsistency between the chained UDF and the single UDF.

We should support taking pandas DataFrame for struct type argument in Scalar Pandas UDF to be consistent.
Currently pyarrow >=0.11 is supported.

## How was this patch tested?

Modified and added some tests.

Closes #24177 from ueshin/issues/SPARK-27240/structtype_argument.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-03-25 11:26:09 -07:00
Bryan Cutler be08b415da [SPARK-27163][PYTHON] Cleanup and consolidate Pandas UDF functionality
## What changes were proposed in this pull request?

This change is a cleanup and consolidation of 3 areas related to Pandas UDFs:

1) `ArrowStreamPandasSerializer` now inherits from `ArrowStreamSerializer` and uses the base class `dump_stream`, `load_stream` to create Arrow reader/writer and send Arrow record batches.  `ArrowStreamPandasSerializer` makes the conversions to/from Pandas and converts to Arrow record batch iterators. This change removed duplicated creation of Arrow readers/writers.

2) `createDataFrame` with Arrow now uses `ArrowStreamPandasSerializer` instead of doing its own conversions from Pandas to Arrow and sending record batches through `ArrowStreamSerializer`.

3) Grouped Map UDFs now reuse existing logic in `ArrowStreamPandasSerializer` to send Pandas DataFrame results as a `StructType` instead of separating each column from the DataFrame. This makes the code a little more consistent with the Python worker, but does require that the returned StructType column is flattened out in `FlatMapGroupsInPandasExec` in Scala.

## How was this patch tested?

Existing tests and ran tests with pyarrow 0.12.0

Closes #24095 from BryanCutler/arrow-refactor-cleanup-UDFs.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-21 17:44:51 +09:00
Bryan Cutler ddc2052ebd [SPARK-23836][PYTHON] Add support for StructType return in Scalar Pandas UDF
## What changes were proposed in this pull request?

This change adds support for returning StructType from a scalar Pandas UDF, where the return value of the function is a pandas.DataFrame. Nested structs are not supported and an error will be raised, child types can be any other type currently supported.

## How was this patch tested?

Added additional unit tests to `test_pandas_udf_scalar`

Closes #23900 from BryanCutler/pyspark-support-scalar_udf-StructType-SPARK-23836.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-03-07 08:52:24 -08:00
Takuya UESHIN 4a4e7aeca7 [SPARK-26887][SQL][PYTHON][NS] Create datetime.date directly instead of creating datetime64 as intermediate data.
## What changes were proposed in this pull request?

Currently `DataFrame.toPandas()` with arrow enabled or `ArrowStreamPandasSerializer` for pandas UDF with pyarrow<0.12 creates `datetime64[ns]` type series as intermediate data and then convert to `datetime.date` series, but the intermediate `datetime64[ns]` might cause an overflow even if the date is valid.

```
>>> import datetime
>>>
>>> t = [datetime.date(2262, 4, 12), datetime.date(2263, 4, 12)]
>>>
>>> df = spark.createDataFrame(t, 'date')
>>> df.show()
+----------+
|     value|
+----------+
|2262-04-12|
|2263-04-12|
+----------+

>>>
>>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
>>>
>>> df.toPandas()
        value
0  1677-09-21
1  1678-09-21
```

We should avoid creating such intermediate data and create `datetime.date` series directly instead.

## How was this patch tested?

Modified some tests to include the date which overflow caused by the intermediate conversion.
Run tests with pyarrow 0.8, 0.10, 0.11, 0.12 in my local environment.

Closes #23795 from ueshin/issues/SPARK-26887/date_as_object.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-18 11:48:10 +08:00
Boris Shminke 75ea89ad94 [SPARK-18161][PYTHON] Update cloudpickle to v0.6.1
## What changes were proposed in this pull request?

In this PR we've done two things:
1) updated the Spark's copy of cloudpickle to 0.6.1 (current stable)
The main reason Spark stayed with cloudpickle 0.4.x was that the default pickle protocol was changed in later versions.

2) started using pickle.HIGHEST_PROTOCOL for both Python 2 and Python 3 for serializers and broadcast
[Pyrolite](https://github.com/irmen/Pyrolite) has such Pickle protocol version support: reading: 0,1,2,3,4; writing: 2.

## How was this patch tested?

Jenkins tests.

Authors: Sloane Simmons, Boris Shminke

This contribution is original work of Sloane Simmons and Boris Shminke and they licensed it to the project under the project's open source license.

Closes #20691 from inpefess/pickle_protocol_4.

Lead-authored-by: Boris Shminke <boris@shminke.me>
Co-authored-by: singularperturbation <sloanes.k@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-02 10:49:45 +08:00
Bryan Cutler 16990f9299 [SPARK-26566][PYTHON][SQL] Upgrade Apache Arrow to version 0.12.0
## What changes were proposed in this pull request?

Upgrade Apache Arrow to version 0.12.0. This includes the Java artifacts and fixes to enable usage with pyarrow 0.12.0

Version 0.12.0 includes the following selected fixes/improvements relevant to Spark users:

* Safe cast fails from numpy float64 array with nans to integer, ARROW-4258
* Java, Reduce heap usage for variable width vectors, ARROW-4147
* Binary identity cast not implemented, ARROW-4101
* pyarrow open_stream deprecated, use ipc.open_stream, ARROW-4098
* conversion to date object no longer needed, ARROW-3910
* Error reading IPC file with no record batches, ARROW-3894
* Signed to unsigned integer cast yields incorrect results when type sizes are the same, ARROW-3790
* from_pandas gives incorrect results when converting floating point to bool, ARROW-3428
* Import pyarrow fails if scikit-learn is installed from conda (boost-cpp / libboost issue), ARROW-3048
* Java update to official Flatbuffers version 1.9.0, ARROW-3175

complete list [here](https://issues.apache.org/jira/issues/?jql=project%20%3D%20ARROW%20AND%20status%20in%20(Resolved%2C%20Closed)%20AND%20fixVersion%20%3D%200.12.0)

PySpark requires the following fixes to work with PyArrow 0.12.0

* Encrypted pyspark worker fails due to ChunkedStream missing closed property
* pyarrow now converts dates as objects by default, which causes error because type is assumed datetime64
* ArrowTests fails due to difference in raised error message
* pyarrow.open_stream deprecated
* tests fail because groupby adds index column with duplicate name

## How was this patch tested?

Ran unit tests with pyarrow versions 0.8.0, 0.10.0, 0.11.1, 0.12.0

Closes #23657 from BryanCutler/arrow-upgrade-012.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-29 14:18:45 +08:00
Liang-Chi Hsieh f92d276653 [SPARK-25811][PYSPARK] Raise a proper error when unsafe cast is detected by PyArrow
## What changes were proposed in this pull request?

Since 0.11.0, PyArrow supports to raise an error for unsafe cast ([PR](https://github.com/apache/arrow/pull/2504)). We should use it to raise a proper error for pandas udf users when such cast is detected.

Added a SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion` to disable Arrow safe type check.

## How was this patch tested?

Added test and manually test.

Closes #22807 from viirya/SPARK-25811.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-22 14:54:41 +08:00
Takuya UESHIN 8edae94fa7 [SPARK-26355][PYSPARK] Add a workaround for PyArrow 0.11.
## What changes were proposed in this pull request?

In PyArrow 0.11, there is a API breaking change.

- [ARROW-1949](https://issues.apache.org/jira/browse/ARROW-1949) - [Python/C++] Add option to Array.from_pandas and pyarrow.array to perform unsafe casts.

This causes test failures in `ScalarPandasUDFTests.test_vectorized_udf_null_(byte|short|int|long)`:

```
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 377, in main
    process()
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 372, in process
    serializer.dump_stream(func(split_index, iterator), outfile)
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 317, in dump_stream
    batch = _create_batch(series, self._timezone)
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 286, in _create_batch
    arrs = [create_array(s, t) for s, t in series]
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 284, in create_array
    return pa.Array.from_pandas(s, mask=mask, type=t)
  File "pyarrow/array.pxi", line 474, in pyarrow.lib.Array.from_pandas
    return array(obj, mask=mask, type=type, safe=safe, from_pandas=True,
  File "pyarrow/array.pxi", line 169, in pyarrow.lib.array
    return _ndarray_to_array(values, mask, type, from_pandas, safe,
  File "pyarrow/array.pxi", line 69, in pyarrow.lib._ndarray_to_array
    check_status(NdarrayToArrow(pool, values, mask, from_pandas,
  File "pyarrow/error.pxi", line 81, in pyarrow.lib.check_status
    raise ArrowInvalid(message)
ArrowInvalid: Floating point value truncated
```

We should add a workaround to support PyArrow 0.11.

## How was this patch tested?

In my local environment.

Closes #23305 from ueshin/issues/SPARK-26355/pyarrow_0.11.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-13 13:14:59 +08:00
Bryan Cutler ecaa495b1f [SPARK-25274][PYTHON][SQL] In toPandas with Arrow send un-ordered record batches to improve performance
## What changes were proposed in this pull request?

When executing `toPandas` with Arrow enabled, partitions that arrive in the JVM out-of-order must be buffered before they can be send to Python. This causes an excess of memory to be used in the driver JVM and increases the time it takes to complete because data must sit in the JVM waiting for preceding partitions to come in.

This change sends un-ordered partitions to Python as soon as they arrive in the JVM, followed by a list of partition indices so that Python can assemble the data in the correct order. This way, data is not buffered at the JVM and there is no waiting on particular partitions so performance will be increased.

Followup to #21546

## How was this patch tested?

Added new test with a large number of batches per partition, and test that forces a small delay in the first partition. These test that partitions are collected out-of-order and then are are put in the correct order in Python.

## Performance Tests - toPandas

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.

Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
	start = time.time()
	_ = df.toPandas()
	elapsed = time.time() - start
```

Spark config
```
spark.driver.memory 5g
spark.executor.memory 5g
spark.driver.maxResultSize 2g
spark.sql.execution.arrow.enabled true
```

Current Master w/ Arrow stream | This PR
---------------------|------------
5.16207 | 4.342533
5.133671 | 4.399408
5.147513 | 4.468471
5.105243 | 4.36524
5.018685 | 4.373791

Avg Master | Avg This PR
------------------|--------------
5.1134364 | 4.3898886

Speedup of **1.164821449**

Closes #22275 from BryanCutler/arrow-toPandas-oo-batches-SPARK-25274.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2018-12-06 10:07:28 -08:00
Imran Rashid 58419b9267 [PYSPARK] Updates to pyspark broadcast 2018-09-17 14:06:09 -05:00
Bryan Cutler 82c18c240a [SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames
## What changes were proposed in this pull request?

This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled.  Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format.  This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.

Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code.  Here are the details of this change:

**toPandas()**

_Before:_
Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata.  Next a collect is done and an Array of Arrow files is the result.  After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.

_After:_
Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers.  The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts a Spark job with a custom handler that sends Arrow RecordBatches to Python. Partitions arriving in order are sent immediately, and out-of-order partitions are buffered until the ones that precede it come in. This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver.  Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive all data must be buffered in memory until then. This case is no worse that before when doing a full collect.

**createDataFrame()**

_Before:_
A Pandas DataFrame is split into parts and each part is made into an Arrow file.  Then each file is prefixed by the buffer size and written to a temp file.  The temp file is read and each Arrow file is parallelized as a byte array.

_After:_
A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch.  The temp file is read as a stream and the Arrow messages are examined.  If the message is an ArrowRecordBatch, the data is saved as a byte array.  After reading the file, each ArrowRecordBatch is parallelized as a byte array.  This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance ends up a litle better.  It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.

## How was this patch tested?

Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.

## Performance Tests - toPandas

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.

Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
	start = time.time()
	_ = df.toPandas()
	elapsed = time.time() - start
```

Current Master | This PR
---------------------|------------
5.803557 | 5.16207
5.409119 | 5.133671
5.493509 | 5.147513
5.433107 | 5.105243
5.488757 | 5.018685

Avg Master | Avg This PR
------------------|--------------
5.5256098 | 5.1134364

Speedup of **1.08060595**

## Performance Tests - createDataFrame

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.

Test code
```python
def run():
	pdf = pd.DataFrame(np.random.rand(10000000, 10))
	spark.createDataFrame(pdf).first()

for i in range(6):
	start = time.time()
	run()
	elapsed = time.time() - start
	gc.collect()
	print("Run %d: %f" % (i, elapsed))
```

Current Master | This PR
--------------------|----------
6.234608 | 5.665641
6.32144 | 5.3475
6.527859 | 5.370803
6.95089 | 5.479151
6.235046 | 5.529167

Avg Master | Avg This PR
---------------|----------------
6.4539686 | 5.4784524

Speedup of **1.178064192**

## Memory Improvements

**toPandas()**

The most significant improvement is reduction of the upper bound space complexity in the JVM driver.  Before, the entire dataset was collected in the JVM first before sending it to Python.  With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition.  Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches.  The schema is now only send from driver JVM to Python.  Before, multiple Arrow file formats were used that each contained the schema.  This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.

I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available.  Using these settings on a standalone cluster:
```
spark.driver.memory 1g
spark.executor.memory 5g
spark.sql.execution.arrow.enabled true
spark.sql.execution.arrow.fallback.enabled false
spark.sql.execution.arrow.maxRecordsPerBatch 0
spark.driver.maxResultSize 2g
```

Test code:
```python
from pyspark.sql.functions import rand
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
df.toPandas()
```

This makes total data size of 33554432×8×4 = 1073741824

With the current master, it fails with OOM but passes using this PR.

**createDataFrame()**

No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above.  The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.

Closes #21546 from BryanCutler/arrow-toPandas-stream-SPARK-23030.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 15:01:12 +08:00
Xingbo Jiang ad45299d04 [SPARK-25095][PYSPARK] Python support for BarrierTaskContext
## What changes were proposed in this pull request?

Add method `barrier()` and `getTaskInfos()` in python TaskContext, these two methods are only allowed for barrier tasks.

## How was this patch tested?

Add new tests in `tests.py`

Closes #22085 from jiangxb1987/python.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-21 15:54:30 -07:00
Bryan Cutler ed075e1ff6 [SPARK-23874][SQL][PYTHON] Upgrade Apache Arrow to 0.10.0
## What changes were proposed in this pull request?

Upgrade Apache Arrow to 0.10.0

Version 0.10.0 has a number of bug fixes and improvements with the following pertaining directly to usage in Spark:
 * Allow for adding BinaryType support ARROW-2141
 * Bug fix related to array serialization ARROW-1973
 * Python2 str will be made into an Arrow string instead of bytes ARROW-2101
 * Python bytearrays are supported in as input to pyarrow ARROW-2141
 * Java has common interface for reset to cleanup complex vectors in Spark ArrowWriter ARROW-1962
 * Cleanup pyarrow type equality checks ARROW-2423
 * ArrowStreamWriter should not hold references to ArrowBlocks ARROW-2632, ARROW-2645
 * Improved low level handling of messages for RecordBatch ARROW-2704

## How was this patch tested?

existing tests

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21939 from BryanCutler/arrow-upgrade-010.
2018-08-14 17:13:38 -07:00
hyukjinkwon f4772fd26f [SPARK-24976][PYTHON] Allow None for Decimal type conversion (specific to PyArrow 0.9.0)
## What changes were proposed in this pull request?

See [ARROW-2432](https://jira.apache.org/jira/browse/ARROW-2432). Seems using `from_pandas` to convert decimals fails if encounters a value of `None`:

```python
import pyarrow as pa
import pandas as pd
from decimal import Decimal

pa.Array.from_pandas(pd.Series([Decimal('3.14'), None]), type=pa.decimal128(3, 2))
```

**Arrow 0.8.0**

```
<pyarrow.lib.Decimal128Array object at 0x10a572c58>
[
  Decimal('3.14'),
  NA
]
```

**Arrow 0.9.0**

```
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "array.pxi", line 383, in pyarrow.lib.Array.from_pandas
  File "array.pxi", line 177, in pyarrow.lib.array
  File "error.pxi", line 77, in pyarrow.lib.check_status
  File "error.pxi", line 77, in pyarrow.lib.check_status
pyarrow.lib.ArrowInvalid: Error converting from Python objects to Decimal: Got Python object of type NoneType but can only handle these types: decimal.Decimal
```

This PR propose to work around this via Decimal NaN:

```python
pa.Array.from_pandas(pd.Series([Decimal('3.14'), Decimal('NaN')]), type=pa.decimal128(3, 2))
```

```
<pyarrow.lib.Decimal128Array object at 0x10ffd2e68>
[
  Decimal('3.14'),
  NA
]
```

## How was this patch tested?

Manually tested:

```bash
SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests ScalarPandasUDFTests
```

**Before**

```
Traceback (most recent call last):
  File "/.../spark/python/pyspark/sql/tests.py", line 4672, in test_vectorized_udf_null_decimal
    self.assertEquals(df.collect(), res.collect())
  File "/.../spark/python/pyspark/sql/dataframe.py", line 533, in collect
    sock_info = self._jdf.collectToPython()
  File "/.../spark/python/lib/py4j-0.10.7-src.zip/py4j/java_gateway.py", line 1257, in __call__
    answer, self.gateway_client, self.target_id, self.name)
  File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/.../spark/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 328, in get_return_value
    format(target_id, ".", name), value)
Py4JJavaError: An error occurred while calling o51.collectToPython.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 3 in stage 1.0 failed 1 times, most recent failure: Lost task 3.0 in stage 1.0 (TID 7, localhost, executor driver): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/.../spark/python/pyspark/worker.py", line 320, in main
    process()
  File "/.../spark/python/pyspark/worker.py", line 315, in process
    serializer.dump_stream(func(split_index, iterator), outfile)
  File "/.../spark/python/pyspark/serializers.py", line 274, in dump_stream
    batch = _create_batch(series, self._timezone)
  File "/.../spark/python/pyspark/serializers.py", line 243, in _create_batch
    arrs = [create_array(s, t) for s, t in series]
  File "/.../spark/python/pyspark/serializers.py", line 241, in create_array
    return pa.Array.from_pandas(s, mask=mask, type=t)
  File "array.pxi", line 383, in pyarrow.lib.Array.from_pandas
  File "array.pxi", line 177, in pyarrow.lib.array
  File "error.pxi", line 77, in pyarrow.lib.check_status
  File "error.pxi", line 77, in pyarrow.lib.check_status
ArrowInvalid: Error converting from Python objects to Decimal: Got Python object of type NoneType but can only handle these types: decimal.Decimal
```

**After**

```
Running tests...
----------------------------------------------------------------------
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
.......S.............................
----------------------------------------------------------------------
Ran 37 tests in 21.980s
```

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21928 from HyukjinKwon/SPARK-24976.
2018-07-31 17:24:24 -07:00
Ruben Berenguel Montoro 6567fc43ac [PYTHON] Fix typo in serializer exception
## What changes were proposed in this pull request?

Fix typo in exception raised in Python serializer

## How was this patch tested?

No code changes

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

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>

Closes #21566 from rberenguel/fix_typo_pyspark_serializers.
2018-06-15 16:59:00 +08:00
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
Bryan Cutler 9bb239c8b1 [SPARK-23159][PYTHON] Update cloudpickle to v0.4.3
## What changes were proposed in this pull request?

The version of cloudpickle in PySpark was close to version 0.4.0 with some additional backported fixes and some minor additions for Spark related things.  This update removes Spark related changes and matches cloudpickle [v0.4.3](https://github.com/cloudpipe/cloudpickle/releases/tag/v0.4.3):

Changes by updating to 0.4.3 include:
* Fix pickling of named tuples https://github.com/cloudpipe/cloudpickle/pull/113
* Built in type constructors for PyPy compatibility [here](d84980ccaa)
* Fix memoryview support https://github.com/cloudpipe/cloudpickle/pull/122
* Improved compatibility with other cloudpickle versions https://github.com/cloudpipe/cloudpickle/pull/128
* Several cleanups https://github.com/cloudpipe/cloudpickle/pull/121 and [here](c91aaf1104)
* [MRG] Regression on pickling classes from the __main__ module https://github.com/cloudpipe/cloudpickle/pull/149
* BUG: Handle instance methods of builtin types https://github.com/cloudpipe/cloudpickle/pull/154
* Fix <span>#</span>129 : do not silence RuntimeError in dump() https://github.com/cloudpipe/cloudpickle/pull/153

## How was this patch tested?

Existing pyspark.tests using python 2.7.14, 3.5.2, 3.6.3

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20373 from BryanCutler/pyspark-update-cloudpickle-42-SPARK-23159.
2018-03-08 20:19:55 +09:00
Takuya UESHIN 63c5bf13ce [SPARK-23334][SQL][PYTHON] Fix pandas_udf with return type StringType() to handle str type properly in Python 2.
## What changes were proposed in this pull request?

In Python 2, when `pandas_udf` tries to return string type value created in the udf with `".."`, the execution fails. E.g.,

```python
from pyspark.sql.functions import pandas_udf, col
import pandas as pd

df = spark.range(10)
str_f = pandas_udf(lambda x: pd.Series(["%s" % i for i in x]), "string")
df.select(str_f(col('id'))).show()
```

raises the following exception:

```
...

java.lang.AssertionError: assertion failed: Invalid schema from pandas_udf: expected StringType, got BinaryType
	at scala.Predef$.assert(Predef.scala:170)
	at org.apache.spark.sql.execution.python.ArrowEvalPythonExec$$anon$2.<init>(ArrowEvalPythonExec.scala:93)

...
```

Seems like pyarrow ignores `type` parameter for `pa.Array.from_pandas()` and consider it as binary type when the type is string type and the string values are `str` instead of `unicode` in Python 2.

This pr adds a workaround for the case.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20507 from ueshin/issues/SPARK-23334.
2018-02-06 18:30:50 +09:00
Takuya UESHIN a24c03138a [SPARK-23290][SQL][PYTHON] Use datetime.date for date type when converting Spark DataFrame to Pandas DataFrame.
## What changes were proposed in this pull request?

In #18664, there was a change in how `DateType` is being returned to users ([line 1968 in dataframe.py](https://github.com/apache/spark/pull/18664/files#diff-6fc344560230bf0ef711bb9b5573f1faR1968)). This can cause client code which works in Spark 2.2 to fail.
See [SPARK-23290](https://issues.apache.org/jira/browse/SPARK-23290?focusedCommentId=16350917&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16350917) for an example.

This pr modifies to use `datetime.date` for date type as Spark 2.2 does.

## How was this patch tested?

Tests modified to fit the new behavior and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20506 from ueshin/issues/SPARK-23290.
2018-02-06 14:52:25 +08:00
Bryan Cutler 59d52631eb [SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0
## What changes were proposed in this pull request?

Upgrade Spark to Arrow 0.8.0 for Java and Python.  Also includes an upgrade of Netty to 4.1.17 to resolve dependency requirements.

The highlights that pertain to Spark for the update from Arrow versoin 0.4.1 to 0.8.0 include:

* Java refactoring for more simple API
* Java reduced heap usage and streamlined hot code paths
* Type support for DecimalType, ArrayType
* Improved type casting support in Python
* Simplified type checking in Python

## How was this patch tested?

Existing tests

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19884 from BryanCutler/arrow-upgrade-080-SPARK-22324.
2017-12-21 20:43:56 +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
Bryan Cutler 209b9361ac [SPARK-20791][PYSPARK] Use Arrow to create Spark DataFrame from Pandas
## What changes were proposed in this pull request?

This change uses Arrow to optimize the creation of a Spark DataFrame from a Pandas DataFrame. The input df is sliced according to the default parallelism. The optimization is enabled with the existing conf "spark.sql.execution.arrow.enabled" and is disabled by default.

## How was this patch tested?

Added new unit test to create DataFrame with and without the optimization enabled, then compare results.

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

Closes #19459 from BryanCutler/arrow-createDataFrame-from_pandas-SPARK-20791.
2017-11-13 13:16:01 +09:00
Bryan Cutler 17af727e38 [SPARK-21375][PYSPARK][SQL] Add Date and Timestamp support to ArrowConverters for toPandas() Conversion
## What changes were proposed in this pull request?

Adding date and timestamp support with Arrow for `toPandas()` and `pandas_udf`s.  Timestamps are stored in Arrow as UTC and manifested to the user as timezone-naive localized to the Python system timezone.

## How was this patch tested?

Added Scala tests for date and timestamp types under ArrowConverters, ArrowUtils, and ArrowWriter suites.  Added Python tests for `toPandas()` and `pandas_udf`s with date and timestamp types.

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

Closes #18664 from BryanCutler/arrow-date-timestamp-SPARK-21375.
2017-10-26 23:02:46 -07: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
Bryan Cutler 7bf4da8a33 [MINOR] Fixed up pandas_udf related docs and formatting
## What changes were proposed in this pull request?

Fixed some minor issues with pandas_udf related docs and formatting.

## How was this patch tested?

NA

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #19375 from BryanCutler/arrow-pandas_udf-cleanup-minor.
2017-09-28 10:24:51 +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
Andrew Ray 6adf67dd14 [SPARK-21985][PYSPARK] PairDeserializer is broken for double-zipped RDDs
## What changes were proposed in this pull request?
(edited)
Fixes a bug introduced in #16121

In PairDeserializer convert each batch of keys and values to lists (if they do not have `__len__` already) so that we can check that they are the same size. Normally they already are lists so this should not have a performance impact, but this is needed when repeated `zip`'s are done.

## How was this patch tested?

Additional unit test

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #19226 from aray/SPARK-21985.
2017-09-18 02:46:27 +09:00
Bryan Cutler d03aebbe65 [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  Data types except complex, date, timestamp, and decimal  are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
2017-07-10 15:21:03 -07:00
Wenchen Fan 838effb98a Revert "[SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas"
This reverts commit e44697606f.
2017-06-28 14:28:40 +08:00
Bryan Cutler e44697606f [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
2017-06-23 09:01:13 +08:00
hyukjinkwon 20e6280626 [SPARK-19019] [PYTHON] Fix hijacked collections.namedtuple and port cloudpickle changes for PySpark to work with Python 3.6.0
## What changes were proposed in this pull request?

Currently, PySpark does not work with Python 3.6.0.

Running `./bin/pyspark` simply throws the error as below and PySpark does not work at all:

```
Traceback (most recent call last):
  File ".../spark/python/pyspark/shell.py", line 30, in <module>
    import pyspark
  File ".../spark/python/pyspark/__init__.py", line 46, in <module>
    from pyspark.context import SparkContext
  File ".../spark/python/pyspark/context.py", line 36, in <module>
    from pyspark.java_gateway import launch_gateway
  File ".../spark/python/pyspark/java_gateway.py", line 31, in <module>
    from py4j.java_gateway import java_import, JavaGateway, GatewayClient
  File "<frozen importlib._bootstrap>", line 961, in _find_and_load
  File "<frozen importlib._bootstrap>", line 950, in _find_and_load_unlocked
  File "<frozen importlib._bootstrap>", line 646, in _load_unlocked
  File "<frozen importlib._bootstrap>", line 616, in _load_backward_compatible
  File ".../spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 18, in <module>
  File "/usr/local/Cellar/python3/3.6.0/Frameworks/Python.framework/Versions/3.6/lib/python3.6/pydoc.py", line 62, in <module>
    import pkgutil
  File "/usr/local/Cellar/python3/3.6.0/Frameworks/Python.framework/Versions/3.6/lib/python3.6/pkgutil.py", line 22, in <module>
    ModuleInfo = namedtuple('ModuleInfo', 'module_finder name ispkg')
  File ".../spark/python/pyspark/serializers.py", line 394, in namedtuple
    cls = _old_namedtuple(*args, **kwargs)
TypeError: namedtuple() missing 3 required keyword-only arguments: 'verbose', 'rename', and 'module'
```

The root cause seems because some arguments of `namedtuple` are now completely keyword-only arguments from Python 3.6.0 (See https://bugs.python.org/issue25628).

We currently copy this function via `types.FunctionType` which does not set the default values of keyword-only arguments (meaning `namedtuple.__kwdefaults__`) and this seems causing internally missing values in the function (non-bound arguments).

This PR proposes to work around this by manually setting it via `kwargs` as `types.FunctionType` seems not supporting to set this.

Also, this PR ports the changes in cloudpickle for compatibility for Python 3.6.0.

## How was this patch tested?

Manually tested with Python 2.7.6 and Python 3.6.0.

```
./bin/pyspsark
```

, manual creation of `namedtuple` both in local and rdd with Python 3.6.0,

and Jenkins tests for other Python versions.

Also,

```
./run-tests --python-executables=python3.6
```

```
Will test against the following Python executables: ['python3.6']
Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
Finished test(python3.6): pyspark.sql.tests (192s)
Finished test(python3.6): pyspark.accumulators (3s)
Finished test(python3.6): pyspark.mllib.tests (198s)
Finished test(python3.6): pyspark.broadcast (3s)
Finished test(python3.6): pyspark.conf (2s)
Finished test(python3.6): pyspark.context (14s)
Finished test(python3.6): pyspark.ml.classification (21s)
Finished test(python3.6): pyspark.ml.evaluation (11s)
Finished test(python3.6): pyspark.ml.clustering (20s)
Finished test(python3.6): pyspark.ml.linalg.__init__ (0s)
Finished test(python3.6): pyspark.streaming.tests (240s)
Finished test(python3.6): pyspark.tests (240s)
Finished test(python3.6): pyspark.ml.recommendation (19s)
Finished test(python3.6): pyspark.ml.feature (36s)
Finished test(python3.6): pyspark.ml.regression (37s)
Finished test(python3.6): pyspark.ml.tuning (28s)
Finished test(python3.6): pyspark.mllib.classification (26s)
Finished test(python3.6): pyspark.mllib.evaluation (18s)
Finished test(python3.6): pyspark.mllib.clustering (44s)
Finished test(python3.6): pyspark.mllib.linalg.__init__ (0s)
Finished test(python3.6): pyspark.mllib.feature (26s)
Finished test(python3.6): pyspark.mllib.fpm (23s)
Finished test(python3.6): pyspark.mllib.random (8s)
Finished test(python3.6): pyspark.ml.tests (92s)
Finished test(python3.6): pyspark.mllib.stat.KernelDensity (0s)
Finished test(python3.6): pyspark.mllib.linalg.distributed (25s)
Finished test(python3.6): pyspark.mllib.stat._statistics (15s)
Finished test(python3.6): pyspark.mllib.recommendation (24s)
Finished test(python3.6): pyspark.mllib.regression (26s)
Finished test(python3.6): pyspark.profiler (9s)
Finished test(python3.6): pyspark.mllib.tree (16s)
Finished test(python3.6): pyspark.shuffle (1s)
Finished test(python3.6): pyspark.mllib.util (18s)
Finished test(python3.6): pyspark.serializers (11s)
Finished test(python3.6): pyspark.rdd (20s)
Finished test(python3.6): pyspark.sql.conf (8s)
Finished test(python3.6): pyspark.sql.catalog (17s)
Finished test(python3.6): pyspark.sql.column (18s)
Finished test(python3.6): pyspark.sql.context (18s)
Finished test(python3.6): pyspark.sql.group (27s)
Finished test(python3.6): pyspark.sql.dataframe (33s)
Finished test(python3.6): pyspark.sql.functions (35s)
Finished test(python3.6): pyspark.sql.types (6s)
Finished test(python3.6): pyspark.sql.streaming (13s)
Finished test(python3.6): pyspark.streaming.util (0s)
Finished test(python3.6): pyspark.sql.session (16s)
Finished test(python3.6): pyspark.sql.window (4s)
Finished test(python3.6): pyspark.sql.readwriter (35s)
Tests passed in 433 seconds
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16429 from HyukjinKwon/SPARK-19019.
2017-01-17 09:53:20 -08:00
Andrew Ray 3c68944b22 [SPARK-16589] [PYTHON] Chained cartesian produces incorrect number of records
## What changes were proposed in this pull request?

Fixes a bug in the python implementation of rdd cartesian product related to batching that showed up in repeated cartesian products with seemingly random results. The root cause being multiple iterators pulling from the same stream in the wrong order because of logic that ignored batching.

`CartesianDeserializer` and `PairDeserializer` were changed to implement `_load_stream_without_unbatching` and borrow the one line implementation of `load_stream` from `BatchedSerializer`. The default implementation of `_load_stream_without_unbatching` was changed to give consistent results (always an iterable) so that it could be used without additional checks.

`PairDeserializer` no longer extends `CartesianDeserializer` as it was not really proper. If wanted a new common super class could be added.

Both `CartesianDeserializer` and `PairDeserializer` now only extend `Serializer` (which has no `dump_stream` implementation) since they are only meant for *de*serialization.

## How was this patch tested?

Additional unit tests (sourced from #14248) plus one for testing a cartesian with zip.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #16121 from aray/fix-cartesian.
2016-12-08 11:08:12 -08:00
Davies Liu 5520418100 [SPARK-10542] [PYSPARK] fix serialize namedtuple
Author: Davies Liu <davies@databricks.com>

Closes #8707 from davies/fix_namedtuple.
2015-09-14 19:46:34 -07:00
Josh Rosen 41afa16500 [SPARK-8652] [PYSPARK] Check return value for all uses of doctest.testmod()
This patch addresses a critical issue in the PySpark tests:

Several of our Python modules' `__main__` methods call `doctest.testmod()` in order to run doctests but forget to check and handle its return value. As a result, some PySpark test failures can go unnoticed because they will not fail the build.

Fortunately, there was only one test failure which was masked by this bug: a `pyspark.profiler` doctest was failing due to changes in RDD pipelining.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #7032 from JoshRosen/testmod-fix and squashes the following commits:

60dbdc0 [Josh Rosen] Account for int vs. long formatting change in Python 3
8b8d80a [Josh Rosen] Fix failing test.
e6423f9 [Josh Rosen] Check return code for all uses of doctest.testmod().
2015-06-26 08:12:22 -07:00
Kevin Conor fdf63f1249 [SPARK-8339] [PYSPARK] integer division for python 3
Itertools islice requires an integer for the stop argument.  Switching to integer division here prevents a ValueError when vs is evaluated above.

davies

This is my original work, and I license it to the project.

Author: Kevin Conor <kevin@discoverybayconsulting.com>

Closes #6794 from kconor/kconor-patch-1 and squashes the following commits:

da5e700 [Kevin Conor] Integer division for batch size
2015-06-19 00:12:20 -07:00