2018-11-14 01:51:11 -05:00
|
|
|
#
|
|
|
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
|
|
|
# contributor license agreements. See the NOTICE file distributed with
|
|
|
|
# this work for additional information regarding copyright ownership.
|
|
|
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
|
|
|
# (the "License"); you may not use this file except in compliance with
|
|
|
|
# the License. You may obtain a copy of the License at
|
|
|
|
#
|
|
|
|
# http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
#
|
|
|
|
# Unless required by applicable law or agreed to in writing, software
|
|
|
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
# See the License for the specific language governing permissions and
|
|
|
|
# limitations under the License.
|
|
|
|
#
|
|
|
|
|
|
|
|
import datetime
|
|
|
|
import os
|
|
|
|
import threading
|
|
|
|
import time
|
|
|
|
import unittest
|
|
|
|
import warnings
|
2020-11-18 07:18:19 -05:00
|
|
|
from distutils.version import LooseVersion
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-08-27 21:39:21 -04:00
|
|
|
from pyspark import SparkContext, SparkConf
|
2019-08-27 04:30:06 -04:00
|
|
|
from pyspark.sql import Row, SparkSession
|
2019-06-04 13:10:27 -04:00
|
|
|
from pyspark.sql.functions import udf
|
2020-08-30 22:23:31 -04:00
|
|
|
from pyspark.sql.types import StructType, StringType, IntegerType, LongType, \
|
2020-11-18 07:18:19 -05:00
|
|
|
FloatType, DoubleType, DecimalType, DateType, TimestampType, BinaryType, StructField, ArrayType
|
2018-11-14 01:51:11 -05:00
|
|
|
from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \
|
|
|
|
pandas_requirement_message, pyarrow_requirement_message
|
2018-11-14 23:30:52 -05:00
|
|
|
from pyspark.testing.utils import QuietTest
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
if have_pandas:
|
|
|
|
import pandas as pd
|
|
|
|
from pandas.util.testing import assert_frame_equal
|
|
|
|
|
|
|
|
if have_pyarrow:
|
2020-08-08 11:51:57 -04:00
|
|
|
import pyarrow as pa # noqa: F401
|
2019-04-09 18:50:25 -04:00
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
@unittest.skipIf(
|
|
|
|
not have_pandas or not have_pyarrow,
|
2020-09-24 01:15:36 -04:00
|
|
|
pandas_requirement_message or pyarrow_requirement_message) # type: ignore
|
2018-11-14 01:51:11 -05:00
|
|
|
class ArrowTests(ReusedSQLTestCase):
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def setUpClass(cls):
|
|
|
|
from datetime import date, datetime
|
|
|
|
from decimal import Decimal
|
|
|
|
super(ArrowTests, cls).setUpClass()
|
|
|
|
cls.warnings_lock = threading.Lock()
|
|
|
|
|
|
|
|
# Synchronize default timezone between Python and Java
|
|
|
|
cls.tz_prev = os.environ.get("TZ", None) # save current tz if set
|
|
|
|
tz = "America/Los_Angeles"
|
|
|
|
os.environ["TZ"] = tz
|
|
|
|
time.tzset()
|
|
|
|
|
|
|
|
cls.spark.conf.set("spark.sql.session.timeZone", tz)
|
2019-06-02 21:01:37 -04:00
|
|
|
|
|
|
|
# Test fallback
|
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.enabled", "false")
|
|
|
|
assert cls.spark.conf.get("spark.sql.execution.arrow.pyspark.enabled") == "false"
|
2018-11-14 01:51:11 -05:00
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.enabled", "true")
|
2019-06-02 21:01:37 -04:00
|
|
|
assert cls.spark.conf.get("spark.sql.execution.arrow.pyspark.enabled") == "true"
|
|
|
|
|
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.fallback.enabled", "true")
|
|
|
|
assert cls.spark.conf.get("spark.sql.execution.arrow.pyspark.fallback.enabled") == "true"
|
2018-11-14 01:51:11 -05:00
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.fallback.enabled", "false")
|
2019-06-02 21:01:37 -04:00
|
|
|
assert cls.spark.conf.get("spark.sql.execution.arrow.pyspark.fallback.enabled") == "false"
|
|
|
|
|
|
|
|
# Enable Arrow optimization in this tests.
|
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.pyspark.enabled", "true")
|
|
|
|
# Disable fallback by default to easily detect the failures.
|
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.pyspark.fallback.enabled", "false")
|
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
cls.schema = StructType([
|
|
|
|
StructField("1_str_t", StringType(), True),
|
|
|
|
StructField("2_int_t", IntegerType(), True),
|
|
|
|
StructField("3_long_t", LongType(), True),
|
|
|
|
StructField("4_float_t", FloatType(), True),
|
|
|
|
StructField("5_double_t", DoubleType(), True),
|
|
|
|
StructField("6_decimal_t", DecimalType(38, 18), True),
|
|
|
|
StructField("7_date_t", DateType(), True),
|
2019-04-22 06:30:31 -04:00
|
|
|
StructField("8_timestamp_t", TimestampType(), True),
|
|
|
|
StructField("9_binary_t", BinaryType(), True)])
|
2018-11-14 01:51:11 -05:00
|
|
|
cls.data = [(u"a", 1, 10, 0.2, 2.0, Decimal("2.0"),
|
2019-04-22 06:30:31 -04:00
|
|
|
date(1969, 1, 1), datetime(1969, 1, 1, 1, 1, 1), bytearray(b"a")),
|
2018-11-14 01:51:11 -05:00
|
|
|
(u"b", 2, 20, 0.4, 4.0, Decimal("4.0"),
|
2019-04-22 06:30:31 -04:00
|
|
|
date(2012, 2, 2), datetime(2012, 2, 2, 2, 2, 2), bytearray(b"bb")),
|
2018-11-14 01:51:11 -05:00
|
|
|
(u"c", 3, 30, 0.8, 6.0, Decimal("6.0"),
|
2019-04-22 06:30:31 -04:00
|
|
|
date(2100, 3, 3), datetime(2100, 3, 3, 3, 3, 3), bytearray(b"ccc")),
|
[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-17 22:48:10 -05:00
|
|
|
(u"d", 4, 40, 1.0, 8.0, Decimal("8.0"),
|
2019-04-22 06:30:31 -04:00
|
|
|
date(2262, 4, 12), datetime(2262, 3, 3, 3, 3, 3), bytearray(b"dddd"))]
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def tearDownClass(cls):
|
|
|
|
del os.environ["TZ"]
|
|
|
|
if cls.tz_prev is not None:
|
|
|
|
os.environ["TZ"] = cls.tz_prev
|
|
|
|
time.tzset()
|
|
|
|
super(ArrowTests, cls).tearDownClass()
|
|
|
|
|
|
|
|
def create_pandas_data_frame(self):
|
|
|
|
import numpy as np
|
|
|
|
data_dict = {}
|
|
|
|
for j, name in enumerate(self.schema.names):
|
|
|
|
data_dict[name] = [self.data[i][j] for i in range(len(self.data))]
|
|
|
|
# need to convert these to numpy types first
|
|
|
|
data_dict["2_int_t"] = np.int32(data_dict["2_int_t"])
|
|
|
|
data_dict["4_float_t"] = np.float32(data_dict["4_float_t"])
|
|
|
|
return pd.DataFrame(data=data_dict)
|
|
|
|
|
|
|
|
def test_toPandas_fallback_enabled(self):
|
2020-11-18 07:18:19 -05:00
|
|
|
ts = datetime.datetime(2015, 11, 1, 0, 30)
|
2019-06-02 21:01:37 -04:00
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.fallback.enabled": True}):
|
2020-11-18 07:18:19 -05:00
|
|
|
schema = StructType([StructField("a", ArrayType(TimestampType()), True)])
|
|
|
|
df = self.spark.createDataFrame([([ts],)], schema=schema)
|
2018-11-14 01:51:11 -05:00
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.warnings_lock:
|
|
|
|
with warnings.catch_warnings(record=True) as warns:
|
|
|
|
# we want the warnings to appear even if this test is run from a subclass
|
|
|
|
warnings.simplefilter("always")
|
|
|
|
pdf = df.toPandas()
|
|
|
|
# Catch and check the last UserWarning.
|
|
|
|
user_warns = [
|
|
|
|
warn.message for warn in warns if isinstance(warn.message, UserWarning)]
|
|
|
|
self.assertTrue(len(user_warns) > 0)
|
|
|
|
self.assertTrue(
|
2020-07-13 22:22:44 -04:00
|
|
|
"Attempting non-optimization" in str(user_warns[-1]))
|
2020-11-18 07:18:19 -05:00
|
|
|
assert_frame_equal(pdf, pd.DataFrame({"a": [[ts]]}))
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_toPandas_fallback_disabled(self):
|
2020-11-18 07:18:19 -05:00
|
|
|
schema = StructType([StructField("a", ArrayType(TimestampType()), True)])
|
2018-11-14 01:51:11 -05:00
|
|
|
df = self.spark.createDataFrame([(None,)], schema=schema)
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.warnings_lock:
|
|
|
|
with self.assertRaisesRegexp(Exception, 'Unsupported type'):
|
|
|
|
df.toPandas()
|
|
|
|
|
|
|
|
def test_null_conversion(self):
|
|
|
|
df_null = self.spark.createDataFrame([tuple([None for _ in range(len(self.data[0]))])] +
|
|
|
|
self.data)
|
|
|
|
pdf = df_null.toPandas()
|
|
|
|
null_counts = pdf.isnull().sum().tolist()
|
|
|
|
self.assertTrue(all([c == 1 for c in null_counts]))
|
|
|
|
|
|
|
|
def _toPandas_arrow_toggle(self, df):
|
2019-06-02 21:01:37 -04:00
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}):
|
2018-11-14 01:51:11 -05:00
|
|
|
pdf = df.toPandas()
|
|
|
|
|
|
|
|
pdf_arrow = df.toPandas()
|
|
|
|
|
|
|
|
return pdf, pdf_arrow
|
|
|
|
|
|
|
|
def test_toPandas_arrow_toggle(self):
|
|
|
|
df = self.spark.createDataFrame(self.data, schema=self.schema)
|
|
|
|
pdf, pdf_arrow = self._toPandas_arrow_toggle(df)
|
|
|
|
expected = self.create_pandas_data_frame()
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(expected, pdf)
|
|
|
|
assert_frame_equal(expected, pdf_arrow)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_toPandas_respect_session_timezone(self):
|
|
|
|
df = self.spark.createDataFrame(self.data, schema=self.schema)
|
|
|
|
|
2020-01-16 21:44:49 -05:00
|
|
|
timezone = "America/Los_Angeles"
|
|
|
|
with self.sql_conf({"spark.sql.session.timeZone": timezone}):
|
2018-11-14 01:51:11 -05:00
|
|
|
pdf_la, pdf_arrow_la = self._toPandas_arrow_toggle(df)
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf_arrow_la, pdf_la)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2020-01-16 21:44:49 -05:00
|
|
|
timezone = "America/New_York"
|
|
|
|
with self.sql_conf({"spark.sql.session.timeZone": timezone}):
|
2018-11-14 01:51:11 -05:00
|
|
|
pdf_ny, pdf_arrow_ny = self._toPandas_arrow_toggle(df)
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf_arrow_ny, pdf_ny)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
self.assertFalse(pdf_ny.equals(pdf_la))
|
|
|
|
|
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?
This PR proposes to move pandas related functionalities into pandas package. Namely:
```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py # Conversion between pandas <> PySpark DataFrames
├── functions.py # pandas_udf
├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py # Type utils between pandas <> PyArrow
└── utils.py # Version requirement checks
```
In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:
```python
class PandasMapOpsMixin(object):
def mapInPandas(self, ...):
...
return ...
# other Pandas <> PySpark APIs
```
```python
class DataFrame(PandasMapOpsMixin):
# other DataFrame APIs equivalent to Scala side.
```
Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.
### Why are the changes needed?
There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.
Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests should cover. Also, I manually built the PySpark API documentation and checked.
Closes #27109 from HyukjinKwon/pandas-refactoring.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
|
|
|
from pyspark.sql.pandas.types import _check_series_convert_timestamps_local_tz
|
2018-11-14 01:51:11 -05:00
|
|
|
pdf_la_corrected = pdf_la.copy()
|
|
|
|
for field in self.schema:
|
|
|
|
if isinstance(field.dataType, TimestampType):
|
|
|
|
pdf_la_corrected[field.name] = _check_series_convert_timestamps_local_tz(
|
|
|
|
pdf_la_corrected[field.name], timezone)
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf_ny, pdf_la_corrected)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_pandas_round_trip(self):
|
|
|
|
pdf = self.create_pandas_data_frame()
|
|
|
|
df = self.spark.createDataFrame(self.data, schema=self.schema)
|
|
|
|
pdf_arrow = df.toPandas()
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf_arrow, pdf)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_filtered_frame(self):
|
|
|
|
df = self.spark.range(3).toDF("i")
|
|
|
|
pdf = df.filter("i < 0").toPandas()
|
|
|
|
self.assertEqual(len(pdf.columns), 1)
|
|
|
|
self.assertEqual(pdf.columns[0], "i")
|
|
|
|
self.assertTrue(pdf.empty)
|
2019-05-22 00:21:26 -04:00
|
|
|
|
|
|
|
def test_no_partition_frame(self):
|
|
|
|
schema = StructType([StructField("field1", StringType(), True)])
|
|
|
|
df = self.spark.createDataFrame(self.sc.emptyRDD(), schema)
|
|
|
|
pdf = df.toPandas()
|
|
|
|
self.assertEqual(len(pdf.columns), 1)
|
|
|
|
self.assertEqual(pdf.columns[0], "field1")
|
|
|
|
self.assertTrue(pdf.empty)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-06-04 13:10:27 -04:00
|
|
|
def test_propagates_spark_exception(self):
|
|
|
|
df = self.spark.range(3).toDF("i")
|
|
|
|
|
|
|
|
def raise_exception():
|
|
|
|
raise Exception("My error")
|
|
|
|
exception_udf = udf(raise_exception, IntegerType())
|
|
|
|
df = df.withColumn("error", exception_udf())
|
|
|
|
with QuietTest(self.sc):
|
2019-06-26 16:05:41 -04:00
|
|
|
with self.assertRaisesRegexp(Exception, 'My error'):
|
2019-06-04 13:10:27 -04:00
|
|
|
df.toPandas()
|
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
def _createDataFrame_toggle(self, pdf, schema=None):
|
2019-06-02 21:01:37 -04:00
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}):
|
2018-11-14 01:51:11 -05:00
|
|
|
df_no_arrow = self.spark.createDataFrame(pdf, schema=schema)
|
|
|
|
|
|
|
|
df_arrow = self.spark.createDataFrame(pdf, schema=schema)
|
|
|
|
|
|
|
|
return df_no_arrow, df_arrow
|
|
|
|
|
|
|
|
def test_createDataFrame_toggle(self):
|
|
|
|
pdf = self.create_pandas_data_frame()
|
|
|
|
df_no_arrow, df_arrow = self._createDataFrame_toggle(pdf, schema=self.schema)
|
|
|
|
self.assertEquals(df_no_arrow.collect(), df_arrow.collect())
|
|
|
|
|
|
|
|
def test_createDataFrame_respect_session_timezone(self):
|
|
|
|
from datetime import timedelta
|
|
|
|
pdf = self.create_pandas_data_frame()
|
2020-01-16 21:44:49 -05:00
|
|
|
timezone = "America/Los_Angeles"
|
|
|
|
with self.sql_conf({"spark.sql.session.timeZone": timezone}):
|
2018-11-14 01:51:11 -05:00
|
|
|
df_no_arrow_la, df_arrow_la = self._createDataFrame_toggle(pdf, schema=self.schema)
|
|
|
|
result_la = df_no_arrow_la.collect()
|
|
|
|
result_arrow_la = df_arrow_la.collect()
|
|
|
|
self.assertEqual(result_la, result_arrow_la)
|
|
|
|
|
2020-01-16 21:44:49 -05:00
|
|
|
timezone = "America/New_York"
|
|
|
|
with self.sql_conf({"spark.sql.session.timeZone": timezone}):
|
2018-11-14 01:51:11 -05:00
|
|
|
df_no_arrow_ny, df_arrow_ny = self._createDataFrame_toggle(pdf, schema=self.schema)
|
|
|
|
result_ny = df_no_arrow_ny.collect()
|
|
|
|
result_arrow_ny = df_arrow_ny.collect()
|
|
|
|
self.assertEqual(result_ny, result_arrow_ny)
|
|
|
|
|
|
|
|
self.assertNotEqual(result_ny, result_la)
|
|
|
|
|
|
|
|
# Correct result_la by adjusting 3 hours difference between Los Angeles and New York
|
|
|
|
result_la_corrected = [Row(**{k: v - timedelta(hours=3) if k == '8_timestamp_t' else v
|
|
|
|
for k, v in row.asDict().items()})
|
|
|
|
for row in result_la]
|
|
|
|
self.assertEqual(result_ny, result_la_corrected)
|
|
|
|
|
|
|
|
def test_createDataFrame_with_schema(self):
|
|
|
|
pdf = self.create_pandas_data_frame()
|
|
|
|
df = self.spark.createDataFrame(pdf, schema=self.schema)
|
|
|
|
self.assertEquals(self.schema, df.schema)
|
|
|
|
pdf_arrow = df.toPandas()
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf_arrow, pdf)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_createDataFrame_with_incorrect_schema(self):
|
|
|
|
pdf = self.create_pandas_data_frame()
|
|
|
|
fields = list(self.schema)
|
[SPARK-33073][PYTHON] Improve error handling on Pandas to Arrow conversion failures
### What changes were proposed in this pull request?
This improves error handling when a failure in conversion from Pandas to Arrow occurs. And fixes tests to be compatible with upcoming Arrow 2.0.0 release.
### Why are the changes needed?
Current tests will fail with Arrow 2.0.0 because of a change in error message when the schema is invalid. For these cases, the current error message also includes information on disabling safe conversion config, which is mainly meant for floating point truncation and overflow. The tests have been updated to use a message that is show for past Arrow versions, and upcoming.
If the user enters an invalid schema, the error produced by pyarrow is not consistent and either `TypeError` or `ArrowInvalid`, with the latter being caught, and raised as a `RuntimeError` with the extra info.
The error handling is improved by:
- narrowing the exception type to `TypeError`s, which `ArrowInvalid` is a subclass and what is raised on safe conversion failures.
- The exception is only raised with additional information on disabling "spark.sql.execution.pandas.convertToArrowArraySafely" if it is enabled in the first place.
- The original exception is chained to better show it to the user.
### Does this PR introduce _any_ user-facing change?
Yes, the error re-raised changes from a RuntimeError to a ValueError, which better categorizes this type of error and in-line with the original Arrow error.
### How was this patch tested?
Existing tests, using pyarrow 1.0.1 and 2.0.0-snapshot
Closes #29951 from BryanCutler/arrow-better-handle-pandas-errors-SPARK-33073.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-06 05:11:24 -04:00
|
|
|
fields[5], fields[6] = fields[6], fields[5] # swap decimal with date
|
2018-11-14 01:51:11 -05:00
|
|
|
wrong_schema = StructType(fields)
|
[SPARK-33073][PYTHON] Improve error handling on Pandas to Arrow conversion failures
### What changes were proposed in this pull request?
This improves error handling when a failure in conversion from Pandas to Arrow occurs. And fixes tests to be compatible with upcoming Arrow 2.0.0 release.
### Why are the changes needed?
Current tests will fail with Arrow 2.0.0 because of a change in error message when the schema is invalid. For these cases, the current error message also includes information on disabling safe conversion config, which is mainly meant for floating point truncation and overflow. The tests have been updated to use a message that is show for past Arrow versions, and upcoming.
If the user enters an invalid schema, the error produced by pyarrow is not consistent and either `TypeError` or `ArrowInvalid`, with the latter being caught, and raised as a `RuntimeError` with the extra info.
The error handling is improved by:
- narrowing the exception type to `TypeError`s, which `ArrowInvalid` is a subclass and what is raised on safe conversion failures.
- The exception is only raised with additional information on disabling "spark.sql.execution.pandas.convertToArrowArraySafely" if it is enabled in the first place.
- The original exception is chained to better show it to the user.
### Does this PR introduce _any_ user-facing change?
Yes, the error re-raised changes from a RuntimeError to a ValueError, which better categorizes this type of error and in-line with the original Arrow error.
### How was this patch tested?
Existing tests, using pyarrow 1.0.1 and 2.0.0-snapshot
Closes #29951 from BryanCutler/arrow-better-handle-pandas-errors-SPARK-33073.
Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-10-06 05:11:24 -04:00
|
|
|
with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": False}):
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.assertRaisesRegexp(Exception, "[D|d]ecimal.*got.*date"):
|
|
|
|
self.spark.createDataFrame(pdf, schema=wrong_schema)
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_createDataFrame_with_names(self):
|
|
|
|
pdf = self.create_pandas_data_frame()
|
|
|
|
new_names = list(map(str, range(len(self.schema.fieldNames()))))
|
|
|
|
# Test that schema as a list of column names gets applied
|
|
|
|
df = self.spark.createDataFrame(pdf, schema=list(new_names))
|
|
|
|
self.assertEquals(df.schema.fieldNames(), new_names)
|
|
|
|
# Test that schema as tuple of column names gets applied
|
|
|
|
df = self.spark.createDataFrame(pdf, schema=tuple(new_names))
|
|
|
|
self.assertEquals(df.schema.fieldNames(), new_names)
|
|
|
|
|
|
|
|
def test_createDataFrame_column_name_encoding(self):
|
|
|
|
pdf = pd.DataFrame({u'a': [1]})
|
|
|
|
columns = self.spark.createDataFrame(pdf).columns
|
|
|
|
self.assertTrue(isinstance(columns[0], str))
|
|
|
|
self.assertEquals(columns[0], 'a')
|
|
|
|
columns = self.spark.createDataFrame(pdf, [u'b']).columns
|
|
|
|
self.assertTrue(isinstance(columns[0], str))
|
|
|
|
self.assertEquals(columns[0], 'b')
|
|
|
|
|
|
|
|
def test_createDataFrame_with_single_data_type(self):
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.assertRaisesRegexp(ValueError, ".*IntegerType.*not supported.*"):
|
|
|
|
self.spark.createDataFrame(pd.DataFrame({"a": [1]}), schema="int")
|
|
|
|
|
|
|
|
def test_createDataFrame_does_not_modify_input(self):
|
|
|
|
# Some series get converted for Spark to consume, this makes sure input is unchanged
|
|
|
|
pdf = self.create_pandas_data_frame()
|
|
|
|
# Use a nanosecond value to make sure it is not truncated
|
2020-02-10 20:03:01 -05:00
|
|
|
pdf.iloc[0, 7] = pd.Timestamp(1)
|
2018-11-14 01:51:11 -05:00
|
|
|
# Integers with nulls will get NaNs filled with 0 and will be casted
|
2020-02-10 20:03:01 -05:00
|
|
|
pdf.iloc[1, 1] = None
|
2018-11-14 01:51:11 -05:00
|
|
|
pdf_copy = pdf.copy(deep=True)
|
|
|
|
self.spark.createDataFrame(pdf, schema=self.schema)
|
|
|
|
self.assertTrue(pdf.equals(pdf_copy))
|
|
|
|
|
|
|
|
def test_schema_conversion_roundtrip(self):
|
[SPARK-30434][PYTHON][SQL] Move pandas related functionalities into 'pandas' sub-package
### What changes were proposed in this pull request?
This PR proposes to move pandas related functionalities into pandas package. Namely:
```bash
pyspark/sql/pandas
├── __init__.py
├── conversion.py # Conversion between pandas <> PySpark DataFrames
├── functions.py # pandas_udf
├── group_ops.py # Grouped UDF / Cogrouped UDF + groupby.apply, groupby.cogroup.apply
├── map_ops.py # Map Iter UDF + mapInPandas
├── serializers.py # pandas <> PyArrow serializers
├── types.py # Type utils between pandas <> PyArrow
└── utils.py # Version requirement checks
```
In order to separately locate `groupby.apply`, `groupby.cogroup.apply`, `mapInPandas`, `toPandas`, and `createDataFrame(pdf)` under `pandas` sub-package, I had to use a mix-in approach which Scala side uses often by `trait`, and also pandas itself uses this approach (see `IndexOpsMixin` as an example) to group related functionalities. Currently, you can think it's like Scala's self typed trait. See the structure below:
```python
class PandasMapOpsMixin(object):
def mapInPandas(self, ...):
...
return ...
# other Pandas <> PySpark APIs
```
```python
class DataFrame(PandasMapOpsMixin):
# other DataFrame APIs equivalent to Scala side.
```
Yes, This is a big PR but they are mostly just moving around except one case `createDataFrame` which I had to split the methods.
### Why are the changes needed?
There are pandas functionalities here and there and I myself gets lost where it was. Also, when you have to make a change commonly for all of pandas related features, it's almost impossible now.
Also, after this change, `DataFrame` and `SparkSession` become more consistent with Scala side since pandas is specific to Python, and this change separates pandas-specific APIs away from `DataFrame` or `SparkSession`.
### Does this PR introduce any user-facing change?
No.
### How was this patch tested?
Existing tests should cover. Also, I manually built the PySpark API documentation and checked.
Closes #27109 from HyukjinKwon/pandas-refactoring.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-08 20:22:50 -05:00
|
|
|
from pyspark.sql.pandas.types import from_arrow_schema, to_arrow_schema
|
2018-11-14 01:51:11 -05:00
|
|
|
arrow_schema = to_arrow_schema(self.schema)
|
|
|
|
schema_rt = from_arrow_schema(arrow_schema)
|
|
|
|
self.assertEquals(self.schema, schema_rt)
|
|
|
|
|
|
|
|
def test_createDataFrame_with_array_type(self):
|
|
|
|
pdf = pd.DataFrame({"a": [[1, 2], [3, 4]], "b": [[u"x", u"y"], [u"y", u"z"]]})
|
|
|
|
df, df_arrow = self._createDataFrame_toggle(pdf)
|
|
|
|
result = df.collect()
|
|
|
|
result_arrow = df_arrow.collect()
|
|
|
|
expected = [tuple(list(e) for e in rec) for rec in pdf.to_records(index=False)]
|
|
|
|
for r in range(len(expected)):
|
|
|
|
for e in range(len(expected[r])):
|
|
|
|
self.assertTrue(expected[r][e] == result_arrow[r][e] and
|
|
|
|
result[r][e] == result_arrow[r][e])
|
|
|
|
|
|
|
|
def test_toPandas_with_array_type(self):
|
|
|
|
expected = [([1, 2], [u"x", u"y"]), ([3, 4], [u"y", u"z"])]
|
|
|
|
array_schema = StructType([StructField("a", ArrayType(IntegerType())),
|
|
|
|
StructField("b", ArrayType(StringType()))])
|
|
|
|
df = self.spark.createDataFrame(expected, schema=array_schema)
|
|
|
|
pdf, pdf_arrow = self._toPandas_arrow_toggle(df)
|
|
|
|
result = [tuple(list(e) for e in rec) for rec in pdf.to_records(index=False)]
|
|
|
|
result_arrow = [tuple(list(e) for e in rec) for rec in pdf_arrow.to_records(index=False)]
|
|
|
|
for r in range(len(expected)):
|
|
|
|
for e in range(len(expected[r])):
|
|
|
|
self.assertTrue(expected[r][e] == result_arrow[r][e] and
|
|
|
|
result[r][e] == result_arrow[r][e])
|
|
|
|
|
2020-11-18 07:18:19 -05:00
|
|
|
def test_createDataFrame_with_map_type(self):
|
|
|
|
map_data = [{"a": 1}, {"b": 2, "c": 3}, {}, None, {"d": None}]
|
|
|
|
|
|
|
|
pdf = pd.DataFrame({"id": [0, 1, 2, 3, 4], "m": map_data})
|
|
|
|
schema = "id long, m map<string, long>"
|
|
|
|
|
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}):
|
|
|
|
df = self.spark.createDataFrame(pdf, schema=schema)
|
|
|
|
|
|
|
|
if LooseVersion(pa.__version__) < LooseVersion("2.0.0"):
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.assertRaisesRegex(Exception, "MapType.*only.*pyarrow 2.0.0"):
|
|
|
|
self.spark.createDataFrame(pdf, schema=schema)
|
|
|
|
else:
|
|
|
|
df_arrow = self.spark.createDataFrame(pdf, schema=schema)
|
|
|
|
|
|
|
|
result = df.collect()
|
|
|
|
result_arrow = df_arrow.collect()
|
|
|
|
|
|
|
|
self.assertEqual(len(result), len(result_arrow))
|
|
|
|
for row, row_arrow in zip(result, result_arrow):
|
|
|
|
i, m = row
|
|
|
|
_, m_arrow = row_arrow
|
|
|
|
self.assertEqual(m, map_data[i])
|
|
|
|
self.assertEqual(m_arrow, map_data[i])
|
|
|
|
|
|
|
|
def test_toPandas_with_map_type(self):
|
|
|
|
pdf = pd.DataFrame({"id": [0, 1, 2, 3],
|
|
|
|
"m": [{}, {"a": 1}, {"a": 1, "b": 2}, {"a": 1, "b": 2, "c": 3}]})
|
|
|
|
|
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}):
|
|
|
|
df = self.spark.createDataFrame(pdf, schema="id long, m map<string, long>")
|
|
|
|
|
|
|
|
if LooseVersion(pa.__version__) < LooseVersion("2.0.0"):
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.assertRaisesRegex(Exception, "MapType.*only.*pyarrow 2.0.0"):
|
|
|
|
df.toPandas()
|
|
|
|
else:
|
|
|
|
pdf_non, pdf_arrow = self._toPandas_arrow_toggle(df)
|
|
|
|
assert_frame_equal(pdf_arrow, pdf_non)
|
|
|
|
|
|
|
|
def test_toPandas_with_map_type_nulls(self):
|
|
|
|
pdf = pd.DataFrame({"id": [0, 1, 2, 3, 4],
|
|
|
|
"m": [{"a": 1}, {"b": 2, "c": 3}, {}, None, {"d": None}]})
|
|
|
|
|
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}):
|
|
|
|
df = self.spark.createDataFrame(pdf, schema="id long, m map<string, long>")
|
|
|
|
|
|
|
|
if LooseVersion(pa.__version__) < LooseVersion("2.0.0"):
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.assertRaisesRegex(Exception, "MapType.*only.*pyarrow 2.0.0"):
|
|
|
|
df.toPandas()
|
|
|
|
else:
|
|
|
|
pdf_non, pdf_arrow = self._toPandas_arrow_toggle(df)
|
|
|
|
assert_frame_equal(pdf_arrow, pdf_non)
|
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
def test_createDataFrame_with_int_col_names(self):
|
|
|
|
import numpy as np
|
|
|
|
pdf = pd.DataFrame(np.random.rand(4, 2))
|
|
|
|
df, df_arrow = self._createDataFrame_toggle(pdf)
|
|
|
|
pdf_col_names = [str(c) for c in pdf.columns]
|
|
|
|
self.assertEqual(pdf_col_names, df.columns)
|
|
|
|
self.assertEqual(pdf_col_names, df_arrow.columns)
|
|
|
|
|
|
|
|
def test_createDataFrame_fallback_enabled(self):
|
2020-11-18 07:18:19 -05:00
|
|
|
ts = datetime.datetime(2015, 11, 1, 0, 30)
|
2018-11-14 01:51:11 -05:00
|
|
|
with QuietTest(self.sc):
|
2019-06-02 21:01:37 -04:00
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.fallback.enabled": True}):
|
2018-11-14 01:51:11 -05:00
|
|
|
with warnings.catch_warnings(record=True) as warns:
|
|
|
|
# we want the warnings to appear even if this test is run from a subclass
|
|
|
|
warnings.simplefilter("always")
|
|
|
|
df = self.spark.createDataFrame(
|
2020-11-18 07:18:19 -05:00
|
|
|
pd.DataFrame({"a": [[ts]]}), "a: array<timestamp>")
|
2018-11-14 01:51:11 -05:00
|
|
|
# Catch and check the last UserWarning.
|
|
|
|
user_warns = [
|
|
|
|
warn.message for warn in warns if isinstance(warn.message, UserWarning)]
|
|
|
|
self.assertTrue(len(user_warns) > 0)
|
|
|
|
self.assertTrue(
|
2020-07-13 22:22:44 -04:00
|
|
|
"Attempting non-optimization" in str(user_warns[-1]))
|
2020-11-18 07:18:19 -05:00
|
|
|
self.assertEqual(df.collect(), [Row(a=[ts])])
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
def test_createDataFrame_fallback_disabled(self):
|
|
|
|
with QuietTest(self.sc):
|
|
|
|
with self.assertRaisesRegexp(TypeError, 'Unsupported type'):
|
|
|
|
self.spark.createDataFrame(
|
2020-11-18 07:18:19 -05:00
|
|
|
pd.DataFrame({"a": [[datetime.datetime(2015, 11, 1, 0, 30)]]}),
|
|
|
|
"a: array<timestamp>")
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
# Regression test for SPARK-23314
|
|
|
|
def test_timestamp_dst(self):
|
|
|
|
# Daylight saving time for Los Angeles for 2015 is Sun, Nov 1 at 2:00 am
|
|
|
|
dt = [datetime.datetime(2015, 11, 1, 0, 30),
|
|
|
|
datetime.datetime(2015, 11, 1, 1, 30),
|
|
|
|
datetime.datetime(2015, 11, 1, 2, 30)]
|
|
|
|
pdf = pd.DataFrame({'time': dt})
|
|
|
|
|
|
|
|
df_from_python = self.spark.createDataFrame(dt, 'timestamp').toDF('time')
|
|
|
|
df_from_pandas = self.spark.createDataFrame(pdf)
|
|
|
|
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf, df_from_python.toPandas())
|
|
|
|
assert_frame_equal(pdf, df_from_pandas.toPandas())
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-06-24 14:15:21 -04:00
|
|
|
# Regression test for SPARK-28003
|
|
|
|
def test_timestamp_nat(self):
|
|
|
|
dt = [pd.NaT, pd.Timestamp('2019-06-11'), None] * 100
|
|
|
|
pdf = pd.DataFrame({'time': dt})
|
|
|
|
df_no_arrow, df_arrow = self._createDataFrame_toggle(pdf)
|
|
|
|
|
|
|
|
assert_frame_equal(pdf, df_no_arrow.toPandas())
|
|
|
|
assert_frame_equal(pdf, df_arrow.toPandas())
|
|
|
|
|
[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 13:07:28 -05:00
|
|
|
def test_toPandas_batch_order(self):
|
|
|
|
|
|
|
|
def delay_first_part(partition_index, iterator):
|
|
|
|
if partition_index == 0:
|
|
|
|
time.sleep(0.1)
|
|
|
|
return iterator
|
|
|
|
|
|
|
|
# Collects Arrow RecordBatches out of order in driver JVM then re-orders in Python
|
|
|
|
def run_test(num_records, num_parts, max_records, use_delay=False):
|
|
|
|
df = self.spark.range(num_records, numPartitions=num_parts).toDF("a")
|
|
|
|
if use_delay:
|
|
|
|
df = df.rdd.mapPartitionsWithIndex(delay_first_part).toDF()
|
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.maxRecordsPerBatch": max_records}):
|
|
|
|
pdf, pdf_arrow = self._toPandas_arrow_toggle(df)
|
2019-04-09 18:50:25 -04:00
|
|
|
assert_frame_equal(pdf, pdf_arrow)
|
[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 13:07:28 -05:00
|
|
|
|
|
|
|
cases = [
|
|
|
|
(1024, 512, 2), # Use large num partitions for more likely collecting out of order
|
|
|
|
(64, 8, 2, True), # Use delay in first partition to force collecting out of order
|
|
|
|
(64, 64, 1), # Test single batch per partition
|
|
|
|
(64, 1, 64), # Test single partition, single batch
|
|
|
|
(64, 1, 8), # Test single partition, multiple batches
|
|
|
|
(30, 7, 2), # Test different sized partitions
|
|
|
|
]
|
|
|
|
|
|
|
|
for case in cases:
|
|
|
|
run_test(*case)
|
|
|
|
|
2020-05-27 20:27:29 -04:00
|
|
|
def test_createDateFrame_with_category_type(self):
|
|
|
|
pdf = pd.DataFrame({"A": [u"a", u"b", u"c", u"a"]})
|
|
|
|
pdf["B"] = pdf["A"].astype('category')
|
|
|
|
category_first_element = dict(enumerate(pdf['B'].cat.categories))[0]
|
|
|
|
|
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": True}):
|
|
|
|
arrow_df = self.spark.createDataFrame(pdf)
|
|
|
|
arrow_type = arrow_df.dtypes[1][1]
|
|
|
|
result_arrow = arrow_df.toPandas()
|
|
|
|
arrow_first_category_element = result_arrow["B"][0]
|
|
|
|
|
|
|
|
with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}):
|
|
|
|
df = self.spark.createDataFrame(pdf)
|
|
|
|
spark_type = df.dtypes[1][1]
|
|
|
|
result_spark = df.toPandas()
|
|
|
|
spark_first_category_element = result_spark["B"][0]
|
|
|
|
|
|
|
|
assert_frame_equal(result_spark, result_arrow)
|
|
|
|
|
|
|
|
# ensure original category elements are string
|
2020-07-13 22:22:44 -04:00
|
|
|
self.assertIsInstance(category_first_element, str)
|
2020-05-27 20:27:29 -04:00
|
|
|
# spark data frame and arrow execution mode enabled data frame type must match pandas
|
2020-05-27 21:27:15 -04:00
|
|
|
self.assertEqual(spark_type, 'string')
|
|
|
|
self.assertEqual(arrow_type, 'string')
|
2020-07-13 22:22:44 -04:00
|
|
|
self.assertIsInstance(arrow_first_category_element, str)
|
|
|
|
self.assertIsInstance(spark_first_category_element, str)
|
2020-05-27 20:27:29 -04:00
|
|
|
|
[SPARK-32098][PYTHON] Use iloc for positional slicing instead of direct slicing in createDataFrame with Arrow
### What changes were proposed in this pull request?
When you use floats are index of pandas, it creates a Spark DataFrame with a wrong results as below when Arrow is enabled:
```bash
./bin/pyspark --conf spark.sql.execution.arrow.pyspark.enabled=true
```
```python
>>> import pandas as pd
>>> spark.createDataFrame(pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.])).show()
+---+
| a|
+---+
| 1|
| 1|
| 2|
+---+
```
This is because direct slicing uses the value as index when the index contains floats:
```python
>>> pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.])[2:]
a
2.0 1
3.0 2
4.0 3
>>> pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.]).iloc[2:]
a
4.0 3
>>> pd.DataFrame({'a': [1,2,3]}, index=[2, 3, 4])[2:]
a
4 3
```
This PR proposes to explicitly use `iloc` to positionally slide when we create a DataFrame from a pandas DataFrame with Arrow enabled.
FWIW, I was trying to investigate why direct slicing refers the index value or the positional index sometimes but I stopped investigating further after reading this https://pandas.pydata.org/pandas-docs/stable/getting_started/10min.html#selection
> While standard Python / Numpy expressions for selecting and setting are intuitive and come in handy for interactive work, for production code, we recommend the optimized pandas data access methods, `.at`, `.iat`, `.loc` and `.iloc`.
### Why are the changes needed?
To create the correct Spark DataFrame from a pandas DataFrame without a data loss.
### Does this PR introduce _any_ user-facing change?
Yes, it is a bug fix.
```bash
./bin/pyspark --conf spark.sql.execution.arrow.pyspark.enabled=true
```
```python
import pandas as pd
spark.createDataFrame(pd.DataFrame({'a': [1,2,3]}, index=[2., 3., 4.])).show()
```
Before:
```
+---+
| a|
+---+
| 1|
| 1|
| 2|
+---+
```
After:
```
+---+
| a|
+---+
| 1|
| 2|
| 3|
+---+
```
### How was this patch tested?
Manually tested and unittest were added.
Closes #28928 from HyukjinKwon/SPARK-32098.
Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2020-06-25 14:04:47 -04:00
|
|
|
def test_createDataFrame_with_float_index(self):
|
|
|
|
# SPARK-32098: float index should not produce duplicated or truncated Spark DataFrame
|
|
|
|
self.assertEqual(
|
|
|
|
self.spark.createDataFrame(
|
|
|
|
pd.DataFrame({'a': [1, 2, 3]}, index=[2., 3., 4.])).distinct().count(), 3)
|
|
|
|
|
2020-07-14 19:44:48 -04:00
|
|
|
def test_no_partition_toPandas(self):
|
|
|
|
# SPARK-32301: toPandas should work from a Spark DataFrame with no partitions
|
|
|
|
# Forward-ported from SPARK-32300.
|
|
|
|
pdf = self.spark.sparkContext.emptyRDD().toDF("col1 int").toPandas()
|
|
|
|
self.assertEqual(len(pdf), 0)
|
|
|
|
self.assertEqual(list(pdf.columns), ["col1"])
|
|
|
|
|
2020-08-10 05:43:41 -04:00
|
|
|
def test_createDataFrame_empty_partition(self):
|
|
|
|
pdf = pd.DataFrame({"c1": [1], "c2": ["string"]})
|
|
|
|
df = self.spark.createDataFrame(pdf)
|
|
|
|
self.assertEqual([Row(c1=1, c2='string')], df.collect())
|
|
|
|
self.assertGreater(self.spark.sparkContext.defaultParallelism, len(pdf))
|
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
|
2019-08-27 04:30:06 -04:00
|
|
|
@unittest.skipIf(
|
|
|
|
not have_pandas or not have_pyarrow,
|
2020-09-24 01:15:36 -04:00
|
|
|
pandas_requirement_message or pyarrow_requirement_message) # type: ignore
|
2019-08-27 04:30:06 -04:00
|
|
|
class MaxResultArrowTests(unittest.TestCase):
|
|
|
|
# These tests are separate as 'spark.driver.maxResultSize' configuration
|
|
|
|
# is a static configuration to Spark context.
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def setUpClass(cls):
|
2019-08-27 21:39:21 -04:00
|
|
|
cls.spark = SparkSession(SparkContext(
|
|
|
|
'local[4]', cls.__name__, conf=SparkConf().set("spark.driver.maxResultSize", "10k")))
|
2019-08-27 04:30:06 -04:00
|
|
|
|
|
|
|
# Explicitly enable Arrow and disable fallback.
|
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.pyspark.enabled", "true")
|
|
|
|
cls.spark.conf.set("spark.sql.execution.arrow.pyspark.fallback.enabled", "false")
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def tearDownClass(cls):
|
|
|
|
if hasattr(cls, "spark"):
|
|
|
|
cls.spark.stop()
|
|
|
|
|
|
|
|
def test_exception_by_max_results(self):
|
|
|
|
with self.assertRaisesRegexp(Exception, "is bigger than"):
|
|
|
|
self.spark.range(0, 10000, 1, 100).toPandas()
|
|
|
|
|
|
|
|
|
2018-11-14 01:51:11 -05:00
|
|
|
class EncryptionArrowTests(ArrowTests):
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
def conf(cls):
|
|
|
|
return super(EncryptionArrowTests, cls).conf().set("spark.io.encryption.enabled", "true")
|
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
2020-08-30 22:23:31 -04:00
|
|
|
from pyspark.sql.tests.test_arrow import * # noqa: F401
|
2018-11-14 01:51:11 -05:00
|
|
|
|
|
|
|
try:
|
2020-09-24 01:15:36 -04:00
|
|
|
import xmlrunner # type: ignore
|
2019-06-23 20:58:17 -04:00
|
|
|
testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2)
|
2018-11-14 01:51:11 -05:00
|
|
|
except ImportError:
|
2018-11-14 23:30:52 -05:00
|
|
|
testRunner = None
|
|
|
|
unittest.main(testRunner=testRunner, verbosity=2)
|