You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Nicolas Renkamp (Jira)" <ji...@apache.org> on 2020/02/26 16:27:00 UTC

[jira] [Updated] (SPARK-30961) Arrow enabled: to_pandas with date column fails

     [ https://issues.apache.org/jira/browse/SPARK-30961?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Nicolas Renkamp updated SPARK-30961:
------------------------------------
    Description: 
Hi,

there seems to be a bug in the arrow enabled to_pandas conversion from spark dataframe to pandas dataframe when the dataframe has a column of type DateType. Here is a minimal example to reproduce the issue:
{code:java}
spark = SparkSession.builder.getOrCreate()
is_arrow_enabled = spark.conf.get("spark.sql.execution.arrow.enabled")
print("Arrow optimization is enabled: " + is_arrow_enabled)
spark_df = spark.createDataFrame(
    [['2019-12-06']], 'created_at: string') \
    .withColumn('created_at', F.to_date('created_at'))

# works
spark_df.toPandas()

spark.conf.set("spark.sql.execution.arrow.enabled", 'true')
is_arrow_enabled = spark.conf.get("spark.sql.execution.arrow.enabled")
print("Arrow optimization is enabled: " + is_arrow_enabled)
# raises AttributeError: Can only use .dt accessor with datetimelike values
# series is still of type object, .dt does not exist
spark_df.toPandas(){code}
A fix would be to modify the _check_series_convert_date function in pyspark.sql.types to:
{code:java}
def _check_series_convert_date(series, data_type):
    """
    Cast the series to datetime.date if it's a date type, otherwise returns the original series.    :param series: pandas.Series
    :param data_type: a Spark data type for the series
    """
    from pyspark.sql.utils import require_minimum_pandas_version
    require_minimum_pandas_version()    from pandas import to_datetime
    if type(data_type) == DateType:
        return to_datetime(series).dt.date
    else:
        return series
{code}
Let me know if I should prepare a Pull Request for the 2.4.5 branch.

I have not tested the behavior on master branch.

 

Thanks,

Nicolas

  was:
Hi,

there seems to be a bug in the arrow enabled to_pandas conversion from spark dataframe to pandas dataframe when the dataframe has a column of type DateType. Here is a minimal example to reproduce the issue:
{code:java}
spark = SparkSession.builder.getOrCreate()
is_arrow_enabled = spark.conf.get("spark.sql.execution.arrow.enabled")
print("Arrow optimization is enabled: " + is_arrow_enabled)
spark_df = spark.createDataFrame(
    [['2019-12-06']], 'created_at: string') \
    .withColumn('created_at', F.to_date('created_at'))

# works
spark_df.toPandas()

spark.conf.set("spark.sql.execution.arrow.enabled", 'true')
is_arrow_enabled = spark.conf.get("spark.sql.execution.arrow.enabled")
print("Arrow optimization is enabled: " + is_arrow_enabled)
# raises AttributeError
spark_df.toPandas(){code}
A fix would be to modify the _check_series_convert_date function in pyspark.sql.types to:
{code:java}
def _check_series_convert_date(series, data_type):
    """
    Cast the series to datetime.date if it's a date type, otherwise returns the original series.    :param series: pandas.Series
    :param data_type: a Spark data type for the series
    """
    from pyspark.sql.utils import require_minimum_pandas_version
    require_minimum_pandas_version()    from pandas import to_datetime
    if type(data_type) == DateType:
        return to_datetime(series).dt.date
    else:
        return series
{code}
Let me know if I should prepare a Pull Request for the 2.4.5 branch.

I have not tested the behavior on master branch.

 

Thanks,

Nicolas


> Arrow enabled: to_pandas with date column fails
> -----------------------------------------------
>
>                 Key: SPARK-30961
>                 URL: https://issues.apache.org/jira/browse/SPARK-30961
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark
>    Affects Versions: 2.4.5
>         Environment: Apache Spark 2.4.5
>            Reporter: Nicolas Renkamp
>            Priority: Major
>              Labels: ready-to-commit
>
> Hi,
> there seems to be a bug in the arrow enabled to_pandas conversion from spark dataframe to pandas dataframe when the dataframe has a column of type DateType. Here is a minimal example to reproduce the issue:
> {code:java}
> spark = SparkSession.builder.getOrCreate()
> is_arrow_enabled = spark.conf.get("spark.sql.execution.arrow.enabled")
> print("Arrow optimization is enabled: " + is_arrow_enabled)
> spark_df = spark.createDataFrame(
>     [['2019-12-06']], 'created_at: string') \
>     .withColumn('created_at', F.to_date('created_at'))
> # works
> spark_df.toPandas()
> spark.conf.set("spark.sql.execution.arrow.enabled", 'true')
> is_arrow_enabled = spark.conf.get("spark.sql.execution.arrow.enabled")
> print("Arrow optimization is enabled: " + is_arrow_enabled)
> # raises AttributeError: Can only use .dt accessor with datetimelike values
> # series is still of type object, .dt does not exist
> spark_df.toPandas(){code}
> A fix would be to modify the _check_series_convert_date function in pyspark.sql.types to:
> {code:java}
> def _check_series_convert_date(series, data_type):
>     """
>     Cast the series to datetime.date if it's a date type, otherwise returns the original series.    :param series: pandas.Series
>     :param data_type: a Spark data type for the series
>     """
>     from pyspark.sql.utils import require_minimum_pandas_version
>     require_minimum_pandas_version()    from pandas import to_datetime
>     if type(data_type) == DateType:
>         return to_datetime(series).dt.date
>     else:
>         return series
> {code}
> Let me know if I should prepare a Pull Request for the 2.4.5 branch.
> I have not tested the behavior on master branch.
>  
> Thanks,
> Nicolas



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org