spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Bruce Robbins (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (SPARK-23715) from_utc_timestamp returns incorrect results for some UTC date/time values
Date Sat, 17 Mar 2018 21:48:00 GMT

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

Bruce Robbins updated SPARK-23715:
----------------------------------
    Description: 
This produces the expected answer:
{noformat}
df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" ).as("dt")).show
+-------------------+
|                 dt|
+-------------------+
|2018-03-13 07:18:23|
+-------------------+
{noformat}
However, the equivalent UTC input (but with an explicit timezone) produces a wrong answer:
{noformat}
df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" ).as("dt")).show
+-------------------+
|                 dt|
+-------------------+
|2018-03-13 00:18:23|
+-------------------+
{noformat}
Additionally, the equivalent Unix time (1520921903, which is also "2018-03-13T06:18:23" in
the UTC time zone) produces the same wrong answer:
{noformat}
df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" ).as("dt")).show
+-------------------+
|                 dt|
+-------------------+
|2018-03-13 00:18:23|
+-------------------+
{noformat}
Digging a little into the code, I see the following:

There is sometimes a mismatch in expectations between the (string => timestamp) cast and
FromUTCTimestamp. Also, since the FromUTCTimestamp expression never sees the actual input
string (the cast "intercepts" the input and converts it to a long timestamp before FromUTCTimestamp
uses the value), FromUTCTimestamp cannot reject any input value that would exercise this mismatch
in expectations.

There is a similar mismatch in expectations in the (integer => timestamp) cast and FromUTCTimestamp.
As a result, Unix time input almost always produces incorrect output.
h3. When things work as expected for String input:

When from_utc_timestamp is passed a string time value with no time zone, DateTimeUtils.stringToTimestamp
(called from a Cast expression) treats the datetime string as though it's in the user's local
time zone. Because DateTimeUtils.stringToTimestamp is a general function, this is reasonable.

As a result, FromUTCTimestamp's input is a timestamp shifted by the local time zone's offset.
FromUTCTimestamp assumes this (or more accurately, a utility function called by FromUTCTimestamp
assumes this), so the first thing it does is reverse-shift to get it back the correct value.
Now that the long value has been shifted back to the correct timestamp value, it can now process
it (by shifting it again based on the specified time zone).
h3. When things go wrong with String input:

When from_utc_timestamp is passed a string time value with an explicit time zone, stringToTimestamp
honors that timezone and ignores the local time zone. stringToTimestamp does not shift the
timestamp by the local timezone, but by the timezone specified on the datetime string.

Unfortunately, FromUTCTimestamp, which has no insight into the actual input or the conversion,
still assumes the timestamp is shifted by the local time zone. So it reverse-shifts the long
value by the local time zone's offset, which produces a incorrect timestamp (except in the
case where the input datetime string just happened to have an explicit timezone that matches
the local timezone). FromUTCTimestamp then uses this incorrect value for further processing.
h3. When things go wrong for Unix time input:

The cast in this case simply multiplies the integer by 1000000. The cast does not shift the
resulting timestamp by the local time zone's offset.

Again, because FromUTCTimestamp's evaluation assumes a shifted timestamp, the result is wrong.

  was:
This produces the expected answer:
{noformat}
df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" ).as("dt")).show
+-------------------+
|                 dt|
+-------------------+
|2018-03-13 07:18:23|
+-------------------+
{noformat}
However, the equivalent UTC input (but with an explicit timezone) produces a wrong answer:
{noformat}
df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" ).as("dt")).show
+-------------------+
|                 dt|
+-------------------+
|2018-03-13 00:18:23|
+-------------------+
{noformat}
Additionally, the equivalent Unix time (1520921903, which is also "2018-03-13T06:18:23" in
the UTC time zone) produces the same wrong answer:
{noformat}
df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" ).as("dt")).show
+-------------------+
|                 dt|
+-------------------+
|2018-03-13 00:18:23|
+-------------------+
{noformat}
Digging a little into the code, I see the following:

There is sometimes a mismatch in expectations between the (string => timestamp) cast and
FromUTCTimestamp. Also, since the FromUTCTimestamp expression never sees the actual input
string (the cast "intercepts" the input and converts it to a long timestamp before FromUTCTimestamp
uses the value), FromUTCTimestamp cannot reject any input value that would exercise this mismatch
in expectations.

There is a similar mismatch in expectations in the (integer => timestamp) cast and FromUTCTimestamp.
As a result, Unix time input almost always produces incorrect output.
h3. When things work as expected for String input:

When from_utc_timestamp is passed a string time value with no time zone, DateTimeUtils.stringToTimestamp
(called from a Cast expression) treats the datetime string as though it's in the user's local
time zone. Because DateTimeUtils.stringToTimestamp is a general function, this is reasonable.

As a result, FromUTCTimestamp's input is a timestamp shifted by the local time zone's offset.
FromUTCTimestamp assumes this (or more accurately, a utility function called by FromUTCTimestamp
assumes this), so the first thing it does is reverse-shift to get it back the correct value.
Now that the long value has been shifted back to the correct timestamp value, it can now process
it (by shifting it again based on the specified time zone).
h3. When things go wrong with String input:

When from_utc_timestamp is passed a string time value with an explicit time zone, stringToTimestamp
honors that timezone and ignores the local time zone. stringToTimestamp does not shift the
timestamp.

Unfortunately, FromUTCTimestamp, which has no insight into the actual input or the conversion,
still assumes the timestamp is shifted by the local time zone. So it reverse-shifts the long
value by the local time zone's offset, which produces a incorrect timestamp. FromUTCTimestamp
then uses this incorrect value for further processing.
h3. When things go wrong for Unix time input:

The cast in this case simply multiplies the integer by 1000000. The cast does not shift the
resulting timestamp by the local time zone's offset.

Again, because FromUTCTimestamp's evaluation assumes a shifted timestamp, the result is wrong.


> from_utc_timestamp returns incorrect results for some UTC date/time values
> --------------------------------------------------------------------------
>
>                 Key: SPARK-23715
>                 URL: https://issues.apache.org/jira/browse/SPARK-23715
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.3.0
>            Reporter: Bruce Robbins
>            Priority: Major
>
> This produces the expected answer:
> {noformat}
> df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" ).as("dt")).show
> +-------------------+
> |                 dt|
> +-------------------+
> |2018-03-13 07:18:23|
> +-------------------+
> {noformat}
> However, the equivalent UTC input (but with an explicit timezone) produces a wrong answer:
> {noformat}
> df.select(from_utc_timestamp(lit("2018-03-13T06:18:23+00:00"), "GMT+1" ).as("dt")).show
> +-------------------+
> |                 dt|
> +-------------------+
> |2018-03-13 00:18:23|
> +-------------------+
> {noformat}
> Additionally, the equivalent Unix time (1520921903, which is also "2018-03-13T06:18:23"
in the UTC time zone) produces the same wrong answer:
> {noformat}
> df.select(from_utc_timestamp(to_timestamp(lit(1520921903)), "GMT+1" ).as("dt")).show
> +-------------------+
> |                 dt|
> +-------------------+
> |2018-03-13 00:18:23|
> +-------------------+
> {noformat}
> Digging a little into the code, I see the following:
> There is sometimes a mismatch in expectations between the (string => timestamp) cast
and FromUTCTimestamp. Also, since the FromUTCTimestamp expression never sees the actual input
string (the cast "intercepts" the input and converts it to a long timestamp before FromUTCTimestamp
uses the value), FromUTCTimestamp cannot reject any input value that would exercise this mismatch
in expectations.
> There is a similar mismatch in expectations in the (integer => timestamp) cast and
FromUTCTimestamp. As a result, Unix time input almost always produces incorrect output.
> h3. When things work as expected for String input:
> When from_utc_timestamp is passed a string time value with no time zone, DateTimeUtils.stringToTimestamp
(called from a Cast expression) treats the datetime string as though it's in the user's local
time zone. Because DateTimeUtils.stringToTimestamp is a general function, this is reasonable.
> As a result, FromUTCTimestamp's input is a timestamp shifted by the local time zone's
offset. FromUTCTimestamp assumes this (or more accurately, a utility function called by FromUTCTimestamp
assumes this), so the first thing it does is reverse-shift to get it back the correct value.
Now that the long value has been shifted back to the correct timestamp value, it can now process
it (by shifting it again based on the specified time zone).
> h3. When things go wrong with String input:
> When from_utc_timestamp is passed a string time value with an explicit time zone, stringToTimestamp
honors that timezone and ignores the local time zone. stringToTimestamp does not shift the
timestamp by the local timezone, but by the timezone specified on the datetime string.
> Unfortunately, FromUTCTimestamp, which has no insight into the actual input or the conversion,
still assumes the timestamp is shifted by the local time zone. So it reverse-shifts the long
value by the local time zone's offset, which produces a incorrect timestamp (except in the
case where the input datetime string just happened to have an explicit timezone that matches
the local timezone). FromUTCTimestamp then uses this incorrect value for further processing.
> h3. When things go wrong for Unix time input:
> The cast in this case simply multiplies the integer by 1000000. The cast does not shift
the resulting timestamp by the local time zone's offset.
> Again, because FromUTCTimestamp's evaluation assumes a shifted timestamp, the result
is wrong.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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


Mime
View raw message