moskvax commented on a change in pull request #28743:
URL: https://github.com/apache/spark/pull/28743#discussion_r438051119



##########
File path: python/pyspark/sql/pandas/conversion.py
##########
@@ -394,10 +394,11 @@ def _create_from_pandas_with_arrow(self, pdf, schema, 
timezone):
 
         # Create the Spark schema from list of names passed in with Arrow types
         if isinstance(schema, (list, tuple)):
-            arrow_schema = pa.Schema.from_pandas(pdf, preserve_index=False)
+            inferred_types = [pa.infer_type(s, mask=s.isna(), from_pandas=True)

Review comment:
       pyarrow < 0.17.0 cannot handle either 
([ARROW-8159](https://issues.apache.org/jira/browse/ARROW-8159)). pyarrow 
0.17.x works as long as the columns that contain `pd.NA` values are not 
`object`-dtyped, which is the case by default as of pandas 1.0.4 (cf 
pandas-dev/pandas#32931). `pa.infer_type` can take a mask and thus avoids 
trying to infer the type of `pd.NA` values, which is what causes 
`pa.Schema.from_pandas` to fail here.
   
   `pa.Schema.from_pandas` returns different types from `pa.infer_type` in two 
cases:
   1. `Categorical` arrays
       * `pa.Schema.from_pandas` returns a `DictionaryType`
       * `pa.infer_type` returns the `value_type` of the `DictionaryType`, 
which is what is already used to determine the Spark type of the resulting 
column
   2. `__arrow_array__`-implementing arrays which return a specialised Arrow 
type (`IntervalArray`, `PeriodArray`)
       * `pa.Schema.from_pandas` returns the type of the array returned by 
`__arrow_array__`
       * `pa.infer_type` does not check for `__arrow_array__` and thus fails 
with these arrays, however these types cannot currently be converted to Spark 
types anyway
   
   Neither of these cases cause regressions, which is why I propose replacing 
`pa.Schema.from_pandas` with `pa.infer_type` here.

##########
File path: python/pyspark/sql/pandas/serializers.py
##########
@@ -150,15 +151,22 @@ def _create_batch(self, series):
         series = ((s, None) if not isinstance(s, (list, tuple)) else s for s 
in series)
 
         def create_array(s, t):
-            mask = s.isnull()
+            # Create with __arrow_array__ if the series' backing array 
implements it
+            series_array = getattr(s, 'array', s._values)
+            if hasattr(series_array, "__arrow_array__"):
+                return series_array.__arrow_array__(type=t)
+
             # Ensure timestamp series are in expected form for Spark internal 
representation
             if t is not None and pa.types.is_timestamp(t):
                 s = _check_series_convert_timestamps_internal(s, 
self._timezone)
-            elif type(s.dtype) == pd.CategoricalDtype:
+            elif is_categorical_dtype(s.dtype):
                 # Note: This can be removed once minimum pyarrow version is >= 
0.16.1
                 s = s.astype(s.dtypes.categories.dtype)
             try:
-                array = pa.Array.from_pandas(s, mask=mask, type=t, 
safe=self._safecheck)
+                mask = s.isnull()
+                # pass _ndarray_values to avoid potential failed type checks 
from pandas array types

Review comment:
       This is a workaround for `IntegerArray` in pre-1.0.0 pandas, which did 
not yet implement `__arrow_array__`, so pyarrow expects it to be a NumPy array:
   
   ```pycon
   >>> import pandas as pd
   >>> import pyarrow as pa
   >>> print(pd.__version__, pa.__version__)
   0.25.0 0.17.1
   >>> s = pd.Series(range(3), dtype=pd.Int64Dtype())
   >>> pa.Array.from_pandas(s)
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "pyarrow/array.pxi", line 805, in pyarrow.lib.Array.from_pandas
     File "pyarrow/array.pxi", line 265, in pyarrow.lib.array
     File "pyarrow/types.pxi", line 76, in pyarrow.lib._datatype_to_pep3118
     File "pyarrow/array.pxi", line 64, in pyarrow.lib._ndarray_to_type
     File "pyarrow/error.pxi", line 108, in pyarrow.lib.check_status
   pyarrow.lib.ArrowTypeError: Did not pass numpy.dtype object
   >>> pa.Array.from_pandas(s, type=pa.int64())
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "pyarrow/array.pxi", line 805, in pyarrow.lib.Array.from_pandas
     File "pyarrow/array.pxi", line 265, in pyarrow.lib.array
     File "pyarrow/array.pxi", line 80, in pyarrow.lib._ndarray_to_array
     File "pyarrow/error.pxi", line 85, in pyarrow.lib.check_status
   pyarrow.lib.ArrowInvalid: Input object was not a NumPy array
   >>> pa.Array.from_pandas(s._ndarray_values, type=pa.int64())
   <pyarrow.lib.Int64Array object at 0x7fb88007a980>
   [
     0,
     1,
     2
   ]
   >>>
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to