[ https://issues.apache.org/jira/browse/SPARK-23715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16454221#comment-16454221 ]
Wenchen Fan commented on SPARK-23715: ------------------------------------- It seems the `from_utc_timestamp` doesn't make a lot of sense in Spark SQL. The timestamp in Spark SQL is TIMESTAMP WITH LOCAL TIME ZONE according to the SQL standard. Physically it stores microseconds from unix epoch, and when it's involved in timezone aware operations, like convert to string format, get the hour component, etc., Spark SQL uses session local timezone to interpret it. That said, the timestamp in Spark does not carry the timezone information, so `from_utc_timestamp` doesn't make sense in Spark to change the timezone of a timestamp. `from_utc_timestamp` was added in Spark 1.5, I think we can't remove it now, we should think of a reasonable definition for it to make it work as users expect. What do users expect? {code} scala> sql("select from_utc_timestamp('2018-01-01 00:00:00', 'GMT+8')").show +-----------------------------------------------------------------+ |from_utc_timestamp(CAST(2018-01-01 00:00:00 AS TIMESTAMP), GMT+8)| +-----------------------------------------------------------------+ | 2018-01-01 08:00:00| +-----------------------------------------------------------------+ scala> spark.conf.set("spark.sql.session.timeZone", "GMT+4") scala> sql("select from_utc_timestamp('2018-01-01 00:00:00', 'GMT+8')").show +-----------------------------------------------------------------+ |from_utc_timestamp(CAST(2018-01-01 00:00:00 AS TIMESTAMP), GMT+8)| +-----------------------------------------------------------------+ | 2018-01-01 08:00:00| +-----------------------------------------------------------------+ {code} The expected behavior is to shift the timestamp from UTC to the specified timezone, assuming the timestamp is timezone-agnostic. So the session local timezone should not affect the result. This also means the timestamp string can't carry the timezone and this is a bug in Spark. For integer input, {code} scala> java.util.TimeZone.getDefault.getID res27: String = Asia/Shanghai // This is GMT+8 scala> sql("select from_utc_timestamp(cast(0 as timestamp), 'GMT+8')").show +-----------------------------------------------+ |from_utc_timestamp(CAST(0 AS TIMESTAMP), GMT+8)| +-----------------------------------------------+ | 1970-01-01 16:00:00| +-----------------------------------------------+ scala> spark.conf.set("spark.sql.session.timeZone", "GMT+4") scala> sql("select from_utc_timestamp(cast(0 as timestamp), 'GMT+8')").show +-----------------------------------------------+ |from_utc_timestamp(CAST(0 AS TIMESTAMP), GMT+8)| +-----------------------------------------------+ | 1970-01-01 12:00:00| +-----------------------------------------------+ {code} so what happened is, `cast(0 as timestamp)` assumes the input integer is seconds from unix epoch, and it converts the seconds to microseconds and done, no timezone information is needed. According to the semantic of Spark timestamp, `cast(0 as timestamp)` is effectively `1970-01-01 08:00:00`(local timezone is GMT+8) as an input to `from_utc_timestamp`, and that's why the result is `1970-01-01 16:00:00`. And the result changes if the local timezone changes. Since this behavior is consistent with Hive, let's stick with it. Now the only thing we need to change is, if input is string, return null if it contains timezone. > 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} > These issues stem from the fact that the FromUTCTimestamp expression, despite > its name, expects the input to be in the user's local timezone. There is some > magic under the covers to make things work (mostly) as the user expects. > As an example, let's say a user in Los Angeles issues the following: > {noformat} > df.select(from_utc_timestamp(lit("2018-03-13T06:18:23"), "GMT+1" > ).as("dt")).show > {noformat} > FromUTCTimestamp gets as input a Timestamp (long) value representing > {noformat} > 2018-03-13T06:18:23-07:00 (long value 1520947103000000) > {noformat} > What FromUTCTimestamp needs instead is > {noformat} > 2018-03-13T06:18:23+00:00 (long value 1520921903000000) > {noformat} > So, it applies the local timezone's offset to the input timestamp to get the > correct value (1520947103000000 minus 7 hours is 1520921903000000). Then it > can process the value and produce the expected output. > When the user explicitly specifies a time zone, FromUTCTimestamp's > assumptions break down. The input is no longer in the local time zone. > Because of the way input data is implicitly casted, FromUTCTimestamp never > knows whether the input data had an explicit timezone. > Here are some gory details: > 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 datetime 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's > offset, 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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org