You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Sandeep Singh (JIRA)" <ji...@apache.org> on 2016/11/11 08:12:59 UTC

[jira] [Updated] (SPARK-18381) Wrong date conversion between spark and python for dates before 1583

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

Sandeep Singh updated SPARK-18381:
----------------------------------
    Description: 
Dates before 1538 (julian/gregorian calendar transition) are processed incorrectly. 
* With python udf (datetime.strptime), .show() returns wrong dates but .collect() returns correct dates
* With pyspark.sql.functions.to_date, .show() shows correct dates but .collect() returns wrong dates. Additionally, collecting '0001-01-01' returns error when collecting dataframe. 

{code:none}
from pyspark.sql.types import DateType
from pyspark.sql.functions import to_date, udf

strToDate =  udf (lambda x: datetime.strptime(x, '%Y-%m-%d'), DateType())


l = [('0002-01-01', 1), ('1581-01-01', 2), ('1582-01-01', 3), ('1583-01-01', 4), ('1584-01-01', 5), ('2012-01-21', 6)]
l_older = [('0001-01-01', 1)]

test_df = spark.createDataFrame(l, ["date_string", "number"])
test_df_older = spark.createDataFrame(l_older, ["date_string", "number"])

test_df_strptime = test_df.withColumn( "date_cast", strToDate(test_df["date_string"]))
test_df_todate = test_df.withColumn( "date_cast", to_date(test_df["date_string"]))
test_df_older_todate = test_df_older.withColumn( "date_cast", to_date(test_df_older["date_string"]))

test_df_strptime.show()
test_df_todate.show()
print test_df_strptime.collect()
print test_df_todate.collect()
print test_df_older_todate.collect()
{code}
{noformat}
+-----------+------+----------+
|date_string|number| date_cast|
+-----------+------+----------+
| 0002-01-01|     1|0002-01-03|
| 1581-01-01|     2|1580-12-22|
| 1582-01-01|     3|1581-12-22|
| 1583-01-01|     4|1583-01-01|
| 1584-01-01|     5|1584-01-01|
| 2012-01-21|     6|2012-01-21|
+-----------+------+----------+
+-----------+------+----------+
|date_string|number| date_cast|
+-----------+------+----------+
| 0002-01-01|     1|0002-01-01|
| 1581-01-01|     2|1581-01-01|
| 1582-01-01|     3|1582-01-01|
| 1583-01-01|     4|1583-01-01|
| 1584-01-01|     5|1584-01-01|
| 2012-01-21|     6|2012-01-21|
+-----------+------+----------+
[Row(date_string=u'0002-01-01', number=1, date_cast=datetime.date(2, 1, 1)), Row(date_string=u'1581-01-01', number=2, date_cast=datetime.date(1581, 1, 1)), Row(date_string=u'1582-01-01', number=3, date_cast=datetime.date(1582, 1, 1)), Row(date_string=u'1583-01-01', number=4, date_cast=datetime.date(1583, 1, 1)), Row(date_string=u'1584-01-01', number=5, date_cast=datetime.date(1584, 1, 1)), Row(date_string=u'2012-01-21', number=6, date_cast=datetime.date(2012, 1, 21))]
[Row(date_string=u'0002-01-01', number=1, date_cast=datetime.date(1, 12, 30)), Row(date_string=u'1581-01-01', number=2, date_cast=datetime.date(1581, 1, 11)), Row(date_string=u'1582-01-01', number=3, date_cast=datetime.date(1582, 1, 11)), Row(date_string=u'1583-01-01', number=4, date_cast=datetime.date(1583, 1, 1)), Row(date_string=u'1584-01-01', number=5, date_cast=datetime.date(1584, 1, 1)), Row(date_string=u'2012-01-21', number=6, date_cast=datetime.date(2012, 1, 21))]
Traceback (most recent call last):
  File "/tmp/zeppelin_pyspark-6043517212596195478.py", line 267, in <module>
    raise Exception(traceback.format_exc())
Exception: Traceback (most recent call last):
  File "/tmp/zeppelin_pyspark-6043517212596195478.py", line 265, in <module>
    exec(code)
  File "<stdin>", line 15, in <module>
  File "/usr/local/spark/python/pyspark/sql/dataframe.py", line 311, in collect
    return list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
  File "/usr/local/spark/python/pyspark/rdd.py", line 142, in _load_from_socket
    for item in serializer.load_stream(rf):
  File "/usr/local/spark/python/pyspark/serializers.py", line 139, in load_stream
    yield self._read_with_length(stream)
  File "/usr/local/spark/python/pyspark/serializers.py", line 164, in _read_with_length
    return self.loads(obj)
  File "/usr/local/spark/python/pyspark/serializers.py", line 422, in loads
    return pickle.loads(obj)
  File "/usr/local/spark/python/pyspark/sql/types.py", line 1354, in <lambda>
    return lambda *a: dataType.fromInternal(a)
  File "/usr/local/spark/python/pyspark/sql/types.py", line 600, in fromInternal
    values = [f.fromInternal(v) for f, v in zip(self.fields, obj)]
  File "/usr/local/spark/python/pyspark/sql/types.py", line 439, in fromInternal
    return self.dataType.fromInternal(obj)
  File "/usr/local/spark/python/pyspark/sql/types.py", line 176, in fromInternal
    return datetime.date.fromordinal(v + self.EPOCH_ORDINAL)
ValueError: ('ordinal must be >= 1', <function <lambda> at 0x7fa21bf7baa0>, (u'0001-01-01', 1, -719164))
{noformat}


  was:
Dates before 1538 (julian/gregorian calendar transition) are processed incorrectly. 
* With python udf (datetime.strptime), .show() returns wrong dates but .collect() returns correct dates
* With pyspark.sql.functions.to_date, .show() shows correct dates but .collect() returns wrong dates. Additionally, collecting '0001-01-01' returns error when collecting dataframe. 

{code:none}
from pyspark.sql.types import DateType
from pyspark.sql.functions import to_date

strToDate =  udf (lambda x: datetime.strptime(x, '%Y-%m-%d'), DateType())


l = [('0002-01-01', 1), ('1581-01-01', 2), ('1582-01-01', 3), ('1583-01-01', 4), ('1584-01-01', 5), ('2012-01-21', 6)]
l_older = [('0001-01-01', 1)]

test_df = spark.createDataFrame(l, ["date_string", "number"])
test_df_older = spark.createDataFrame(l_older, ["date_string", "number"])

test_df_strptime = test_df.withColumn( "date_cast", strToDate(test_df["date_string"]))
test_df_todate = test_df.withColumn( "date_cast", to_date(test_df["date_string"]))
test_df_older_todate = test_df_older.withColumn( "date_cast", to_date(test_df_older["date_string"]))

test_df_strptime.show()
test_df_todate.show()
print test_df_strptime.collect()
print test_df_todate.collect()
print test_df_older_todate.collect()
{code}
{noformat}
+-----------+------+----------+
|date_string|number| date_cast|
+-----------+------+----------+
| 0002-01-01|     1|0002-01-03|
| 1581-01-01|     2|1580-12-22|
| 1582-01-01|     3|1581-12-22|
| 1583-01-01|     4|1583-01-01|
| 1584-01-01|     5|1584-01-01|
| 2012-01-21|     6|2012-01-21|
+-----------+------+----------+
+-----------+------+----------+
|date_string|number| date_cast|
+-----------+------+----------+
| 0002-01-01|     1|0002-01-01|
| 1581-01-01|     2|1581-01-01|
| 1582-01-01|     3|1582-01-01|
| 1583-01-01|     4|1583-01-01|
| 1584-01-01|     5|1584-01-01|
| 2012-01-21|     6|2012-01-21|
+-----------+------+----------+
[Row(date_string=u'0002-01-01', number=1, date_cast=datetime.date(2, 1, 1)), Row(date_string=u'1581-01-01', number=2, date_cast=datetime.date(1581, 1, 1)), Row(date_string=u'1582-01-01', number=3, date_cast=datetime.date(1582, 1, 1)), Row(date_string=u'1583-01-01', number=4, date_cast=datetime.date(1583, 1, 1)), Row(date_string=u'1584-01-01', number=5, date_cast=datetime.date(1584, 1, 1)), Row(date_string=u'2012-01-21', number=6, date_cast=datetime.date(2012, 1, 21))]
[Row(date_string=u'0002-01-01', number=1, date_cast=datetime.date(1, 12, 30)), Row(date_string=u'1581-01-01', number=2, date_cast=datetime.date(1581, 1, 11)), Row(date_string=u'1582-01-01', number=3, date_cast=datetime.date(1582, 1, 11)), Row(date_string=u'1583-01-01', number=4, date_cast=datetime.date(1583, 1, 1)), Row(date_string=u'1584-01-01', number=5, date_cast=datetime.date(1584, 1, 1)), Row(date_string=u'2012-01-21', number=6, date_cast=datetime.date(2012, 1, 21))]
Traceback (most recent call last):
  File "/tmp/zeppelin_pyspark-6043517212596195478.py", line 267, in <module>
    raise Exception(traceback.format_exc())
Exception: Traceback (most recent call last):
  File "/tmp/zeppelin_pyspark-6043517212596195478.py", line 265, in <module>
    exec(code)
  File "<stdin>", line 15, in <module>
  File "/usr/local/spark/python/pyspark/sql/dataframe.py", line 311, in collect
    return list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
  File "/usr/local/spark/python/pyspark/rdd.py", line 142, in _load_from_socket
    for item in serializer.load_stream(rf):
  File "/usr/local/spark/python/pyspark/serializers.py", line 139, in load_stream
    yield self._read_with_length(stream)
  File "/usr/local/spark/python/pyspark/serializers.py", line 164, in _read_with_length
    return self.loads(obj)
  File "/usr/local/spark/python/pyspark/serializers.py", line 422, in loads
    return pickle.loads(obj)
  File "/usr/local/spark/python/pyspark/sql/types.py", line 1354, in <lambda>
    return lambda *a: dataType.fromInternal(a)
  File "/usr/local/spark/python/pyspark/sql/types.py", line 600, in fromInternal
    values = [f.fromInternal(v) for f, v in zip(self.fields, obj)]
  File "/usr/local/spark/python/pyspark/sql/types.py", line 439, in fromInternal
    return self.dataType.fromInternal(obj)
  File "/usr/local/spark/python/pyspark/sql/types.py", line 176, in fromInternal
    return datetime.date.fromordinal(v + self.EPOCH_ORDINAL)
ValueError: ('ordinal must be >= 1', <function <lambda> at 0x7fa21bf7baa0>, (u'0001-01-01', 1, -719164))
{noformat}



> Wrong date conversion between spark and python for dates before 1583
> --------------------------------------------------------------------
>
>                 Key: SPARK-18381
>                 URL: https://issues.apache.org/jira/browse/SPARK-18381
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark
>    Affects Versions: 2.0.0
>            Reporter: Luca Caniparoli
>
> Dates before 1538 (julian/gregorian calendar transition) are processed incorrectly. 
> * With python udf (datetime.strptime), .show() returns wrong dates but .collect() returns correct dates
> * With pyspark.sql.functions.to_date, .show() shows correct dates but .collect() returns wrong dates. Additionally, collecting '0001-01-01' returns error when collecting dataframe. 
> {code:none}
> from pyspark.sql.types import DateType
> from pyspark.sql.functions import to_date, udf
> strToDate =  udf (lambda x: datetime.strptime(x, '%Y-%m-%d'), DateType())
> l = [('0002-01-01', 1), ('1581-01-01', 2), ('1582-01-01', 3), ('1583-01-01', 4), ('1584-01-01', 5), ('2012-01-21', 6)]
> l_older = [('0001-01-01', 1)]
> test_df = spark.createDataFrame(l, ["date_string", "number"])
> test_df_older = spark.createDataFrame(l_older, ["date_string", "number"])
> test_df_strptime = test_df.withColumn( "date_cast", strToDate(test_df["date_string"]))
> test_df_todate = test_df.withColumn( "date_cast", to_date(test_df["date_string"]))
> test_df_older_todate = test_df_older.withColumn( "date_cast", to_date(test_df_older["date_string"]))
> test_df_strptime.show()
> test_df_todate.show()
> print test_df_strptime.collect()
> print test_df_todate.collect()
> print test_df_older_todate.collect()
> {code}
> {noformat}
> +-----------+------+----------+
> |date_string|number| date_cast|
> +-----------+------+----------+
> | 0002-01-01|     1|0002-01-03|
> | 1581-01-01|     2|1580-12-22|
> | 1582-01-01|     3|1581-12-22|
> | 1583-01-01|     4|1583-01-01|
> | 1584-01-01|     5|1584-01-01|
> | 2012-01-21|     6|2012-01-21|
> +-----------+------+----------+
> +-----------+------+----------+
> |date_string|number| date_cast|
> +-----------+------+----------+
> | 0002-01-01|     1|0002-01-01|
> | 1581-01-01|     2|1581-01-01|
> | 1582-01-01|     3|1582-01-01|
> | 1583-01-01|     4|1583-01-01|
> | 1584-01-01|     5|1584-01-01|
> | 2012-01-21|     6|2012-01-21|
> +-----------+------+----------+
> [Row(date_string=u'0002-01-01', number=1, date_cast=datetime.date(2, 1, 1)), Row(date_string=u'1581-01-01', number=2, date_cast=datetime.date(1581, 1, 1)), Row(date_string=u'1582-01-01', number=3, date_cast=datetime.date(1582, 1, 1)), Row(date_string=u'1583-01-01', number=4, date_cast=datetime.date(1583, 1, 1)), Row(date_string=u'1584-01-01', number=5, date_cast=datetime.date(1584, 1, 1)), Row(date_string=u'2012-01-21', number=6, date_cast=datetime.date(2012, 1, 21))]
> [Row(date_string=u'0002-01-01', number=1, date_cast=datetime.date(1, 12, 30)), Row(date_string=u'1581-01-01', number=2, date_cast=datetime.date(1581, 1, 11)), Row(date_string=u'1582-01-01', number=3, date_cast=datetime.date(1582, 1, 11)), Row(date_string=u'1583-01-01', number=4, date_cast=datetime.date(1583, 1, 1)), Row(date_string=u'1584-01-01', number=5, date_cast=datetime.date(1584, 1, 1)), Row(date_string=u'2012-01-21', number=6, date_cast=datetime.date(2012, 1, 21))]
> Traceback (most recent call last):
>   File "/tmp/zeppelin_pyspark-6043517212596195478.py", line 267, in <module>
>     raise Exception(traceback.format_exc())
> Exception: Traceback (most recent call last):
>   File "/tmp/zeppelin_pyspark-6043517212596195478.py", line 265, in <module>
>     exec(code)
>   File "<stdin>", line 15, in <module>
>   File "/usr/local/spark/python/pyspark/sql/dataframe.py", line 311, in collect
>     return list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
>   File "/usr/local/spark/python/pyspark/rdd.py", line 142, in _load_from_socket
>     for item in serializer.load_stream(rf):
>   File "/usr/local/spark/python/pyspark/serializers.py", line 139, in load_stream
>     yield self._read_with_length(stream)
>   File "/usr/local/spark/python/pyspark/serializers.py", line 164, in _read_with_length
>     return self.loads(obj)
>   File "/usr/local/spark/python/pyspark/serializers.py", line 422, in loads
>     return pickle.loads(obj)
>   File "/usr/local/spark/python/pyspark/sql/types.py", line 1354, in <lambda>
>     return lambda *a: dataType.fromInternal(a)
>   File "/usr/local/spark/python/pyspark/sql/types.py", line 600, in fromInternal
>     values = [f.fromInternal(v) for f, v in zip(self.fields, obj)]
>   File "/usr/local/spark/python/pyspark/sql/types.py", line 439, in fromInternal
>     return self.dataType.fromInternal(obj)
>   File "/usr/local/spark/python/pyspark/sql/types.py", line 176, in fromInternal
>     return datetime.date.fromordinal(v + self.EPOCH_ORDINAL)
> ValueError: ('ordinal must be >= 1', <function <lambda> at 0x7fa21bf7baa0>, (u'0001-01-01', 1, -719164))
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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