## What changes were proposed in this pull request?
In Python 2.7 with latest PyArrow and Pandas, the error message seems a bit different with Python 3. This PR simply fixes the test.
```
======================================================================
FAIL: test_createDataFrame_with_incorrect_schema (pyspark.sql.tests.test_arrow.ArrowTests)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/.../spark/python/pyspark/sql/tests/test_arrow.py", line 275, in test_createDataFrame_with_incorrect_schema
self.spark.createDataFrame(pdf, schema=wrong_schema)
AssertionError: "integer.*required.*got.*str" does not match "('Exception thrown when converting pandas.Series (object) to Arrow Array (int32). It can be caused by overflows or other unsafe conversions warned by Arrow. Arrow safe type check can be disabled by using SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion`.', ArrowTypeError('an integer is required',))"
======================================================================
FAIL: test_createDataFrame_with_incorrect_schema (pyspark.sql.tests.test_arrow.EncryptionArrowTests)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/.../spark/python/pyspark/sql/tests/test_arrow.py", line 275, in test_createDataFrame_with_incorrect_schema
self.spark.createDataFrame(pdf, schema=wrong_schema)
AssertionError: "integer.*required.*got.*str" does not match "('Exception thrown when converting pandas.Series (object) to Arrow Array (int32). It can be caused by overflows or other unsafe conversions warned by Arrow. Arrow safe type check can be disabled by using SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion`.', ArrowTypeError('an integer is required',))"
```
## How was this patch tested?
Manually tested.
```
cd python
./run-tests --python-executables=python --modules pyspark-sql
```
Closes#25042 from HyukjinKwon/SPARK-28240.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Currently with `toLocalIterator()` and `toPandas()` with Arrow enabled, if the Spark job being run in the background serving thread errors, it will be caught and sent to Python through the PySpark serializer.
This is not the ideal solution because it is only catch a SparkException, it won't handle an error that occurs in the serializer, and each method has to have it's own special handling to propagate the error.
This PR instead returns the Python Server object along with the serving port and authentication info, so that it allows the Python caller to join with the serving thread. During the call to join, the serving thread Future is completed either successfully or with an exception. In the latter case, the exception will be propagated to Python through the Py4j call.
## How was this patch tested?
Existing tests
Closes#24834 from BryanCutler/pyspark-propagate-server-error-SPARK-27992.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
## 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>
## What changes were proposed in this pull request?
Currently, pretty skipped message added by f7435bec6a mechanism seems not working when xmlrunner is installed apparently.
This PR fixes two things:
1. When `xmlrunner` is installed, seems `xmlrunner` does not respect `vervosity` level in unittests (default is level 1).
So the output looks as below
```
Running tests...
----------------------------------------------------------------------
SSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSSS
----------------------------------------------------------------------
```
So it is not caught by our message detection mechanism.
2. If we manually set the `vervocity` level to `xmlrunner`, it prints messages as below:
```
test_mixed_udf (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
test_mixed_udf_and_sql (pyspark.sql.tests.test_pandas_udf_scalar.ScalarPandasUDFTests) ... SKIP (0.000s)
...
```
This is different in our Jenkins machine:
```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.23.2 must be installed; however, it was not found.'
...
```
Note that last `SKIP` is different. This PR fixes the regular expression to catch `SKIP` case as well.
## How was this patch tested?
Manually tested.
**Before:**
```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (0s)
...
Tests passed in 562 seconds
========================================================================
...
```
**After:**
```
Starting test(python2.7): pyspark....
Finished test(python2.7): pyspark.... (48s) ... 93 tests were skipped
...
Tests passed in 560 seconds
Skipped tests pyspark.... with python2.7:
pyspark...(...) ... SKIP (0.000s)
...
========================================================================
...
```
Closes#24927 from HyukjinKwon/SPARK-28130.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## 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>
## 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>
## What changes were proposed in this pull request?
`spark.sql.execution.arrow.enabled` was added when we add PySpark arrow optimization.
Later, in the current master, SparkR arrow optimization was added and it's controlled by the same configuration `spark.sql.execution.arrow.enabled`.
There look two issues about this:
1. `spark.sql.execution.arrow.enabled` in PySpark was added from 2.3.0 whereas SparkR optimization was added 3.0.0. The stability is different so it's problematic when we change the default value for one of both optimization first.
2. Suppose users want to share some JVM by PySpark and SparkR. They are currently forced to use the optimization for all or none if the configuration is set globally.
This PR proposes two separate configuration groups for PySpark and SparkR about Arrow optimization:
- Deprecate `spark.sql.execution.arrow.enabled`
- Add `spark.sql.execution.arrow.pyspark.enabled` (fallback to `spark.sql.execution.arrow.enabled`)
- Add `spark.sql.execution.arrow.sparkr.enabled`
- Deprecate `spark.sql.execution.arrow.fallback.enabled`
- Add `spark.sql.execution.arrow.pyspark.fallback.enabled ` (fallback to `spark.sql.execution.arrow.fallback.enabled`)
Note that `spark.sql.execution.arrow.maxRecordsPerBatch` is used within JVM side for both.
Note that `spark.sql.execution.arrow.fallback.enabled` was added due to behaviour change. We don't need it in SparkR - SparkR side has the automatic fallback.
## How was this patch tested?
Manually tested and some unittests were added.
Closes#24700 from HyukjinKwon/separate-sparkr-arrow.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/22275 introduced a performance improvement where we send partitions out of order to python and then, as a last step, send the partition order as well.
However, if there are no partitions we will never send the partition order and we will get an "EofError" on the python side.
This PR fixes this by also sending the partition order if there are no partitions present.
## How was this patch tested?
New unit test added.
Closes#24650 from dvogelbacher/dv/fixNoPartitionArrowConversion.
Authored-by: David Vogelbacher <dvogelbacher@palantir.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## 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>
## What changes were proposed in this pull request?
Running PySpark tests with Pandas 0.24.x causes a failure in `test_pandas_udf_grouped_map` test_supported_types:
`ValueError: The truth value of an array with more than one element is ambiguous. Use a.any() or a.all()`
This is because a column is an ArrayType and the method `sqlutils ReusedSQLTestCase.assertPandasEqual ` does not properly check this.
This PR removes `assertPandasEqual` and replaces it with the built-in `pandas.util.testing.assert_frame_equal` which can properly handle columns of ArrayType and also prints out better diff between the DataFrames when an error occurs.
Additionally, imports of pandas and pyarrow were moved to the top of related test files to avoid duplicating the same import many times.
## How was this patch tested?
Existing tests
Closes#24306 from BryanCutler/python-pandas-assert_frame_equal-SPARK-27387.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## 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>
## 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>
## 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>
## What changes were proposed in this pull request?
This PR continues to break down a big large file into smaller files. See https://github.com/apache/spark/pull/23021. It targets to follow https://github.com/numpy/numpy/tree/master/numpy.
Basically this PR proposes to break down `pyspark/tests.py` into ...:
```
pyspark
...
├── testing
...
│ └── utils.py
├── tests
│ ├── __init__.py
│ ├── test_appsubmit.py
│ ├── test_broadcast.py
│ ├── test_conf.py
│ ├── test_context.py
│ ├── test_daemon.py
│ ├── test_join.py
│ ├── test_profiler.py
│ ├── test_rdd.py
│ ├── test_readwrite.py
│ ├── test_serializers.py
│ ├── test_shuffle.py
│ ├── test_taskcontext.py
│ ├── test_util.py
│ └── test_worker.py
...
```
## How was this patch tested?
Existing tests should cover.
`cd python` and .`/run-tests-with-coverage`. Manually checked they are actually being ran.
Each test (not officially) can be ran via:
```bash
SPARK_TESTING=1 ./bin/pyspark pyspark.tests.test_context
```
Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`.
Closes#23033 from HyukjinKwon/SPARK-26036.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
This is the official first attempt to break huge single `tests.py` file - I did it locally before few times and gave up for some reasons. Now, currently it really makes the unittests super hard to read and difficult to check. To me, it even bothers me to to scroll down the big file. It's one single 7000 lines file!
This is not only readability issue. Since one big test takes most of tests time, the tests don't run in parallel fully - although it will costs to start and stop the context.
We could pick up one example and follow. Given my investigation, the current style looks closer to NumPy structure and looks easier to follow. Please see https://github.com/numpy/numpy/tree/master/numpy.
Basically this PR proposes to break down `pyspark/sql/tests.py` into ...:
```bash
pyspark
...
├── sql
...
│ ├── tests # Includes all tests broken down from 'pyspark/sql/tests.py'
│ │ │ # Each matchs to module in 'pyspark/sql'. Additionally, some logical group can
│ │ │ # be added. For instance, 'test_arrow.py', 'test_datasources.py' ...
│ │ ├── __init__.py
│ │ ├── test_appsubmit.py
│ │ ├── test_arrow.py
│ │ ├── test_catalog.py
│ │ ├── test_column.py
│ │ ├── test_conf.py
│ │ ├── test_context.py
│ │ ├── test_dataframe.py
│ │ ├── test_datasources.py
│ │ ├── test_functions.py
│ │ ├── test_group.py
│ │ ├── test_pandas_udf.py
│ │ ├── test_pandas_udf_grouped_agg.py
│ │ ├── test_pandas_udf_grouped_map.py
│ │ ├── test_pandas_udf_scalar.py
│ │ ├── test_pandas_udf_window.py
│ │ ├── test_readwriter.py
│ │ ├── test_serde.py
│ │ ├── test_session.py
│ │ ├── test_streaming.py
│ │ ├── test_types.py
│ │ ├── test_udf.py
│ │ └── test_utils.py
...
├── testing # Includes testing utils that can be used in unittests.
│ ├── __init__.py
│ └── sqlutils.py
...
```
## How was this patch tested?
Existing tests should cover.
`cd python` and `./run-tests-with-coverage`. Manually checked they are actually being ran.
Each test (not officially) can be ran via:
```
SPARK_TESTING=1 ./bin/pyspark pyspark.sql.tests.test_pandas_udf_scalar
```
Note that if you're using Mac and Python 3, you might have to `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`.
Closes#23021 from HyukjinKwon/SPARK-25344.
Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>