Skip to content

[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp #24195

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 19 commits into from

Conversation

MaxGekk
Copy link
Member

@MaxGekk MaxGekk commented Mar 24, 2019

What changes were proposed in this pull request?

In the PR, I propose to deprecate the from_utc_timestamp() and to_utc_timestamp, and disable them by default. The functions can be enabled back via the SQL config spark.sql.legacy.utcTimestampFunc.enabled. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of TimestampType which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as TimestampType.

How was this patch tested?

The changes were tested by DateExpressionsSuite and DateFunctionsSuite.

@MaxGekk
Copy link
Member Author

MaxGekk commented Mar 24, 2019

@cloud-fan Please, take a look at the PR.

@@ -1025,6 +1027,11 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
case class FromUTCTimestamp(left: Expression, right: Expression)
extends BinaryExpression with ImplicitCastInputTypes {

if (!SQLConf.get.utcTimestampFuncEnabled) {
throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0." +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ur, @MaxGekk . Usually, deprecation means showing warnings instead of Exception. This looks like a ban to me.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea, deprecation is deprecation. I think we can simply deprecate them in functions.scala, and that's all.

@srowen
Copy link
Member

srowen commented Mar 25, 2019

Wait. Aren't these functions from Hive? We can't drop them if so. I know the semantics are weird but they aren't incoherent, just poorly named

@SparkQA
Copy link

SparkQA commented Mar 25, 2019

Test build #103876 has finished for PR 24195 at commit daa1eb7.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

AFAIK hive's timestamp has different semantic from Spark's timestamp, I think it's more than a naming issue. It's arguable if we should deprecate these two functions, but we definitely should not ban them.

@cloud-fan
Copy link
Contributor

retest this please

@cloud-fan
Copy link
Contributor

some more thoughts: I think there is really no way to deprecate functions in SQL, because users don't see warning messages. The current approach looks fine to me as a way to deprecate SQL functions, and a flag is needed for sure.

LGTM

@dilipbiswal
Copy link
Contributor

dilipbiswal commented Mar 26, 2019

@cloud-fan @MaxGekk Sorry if my question is dumb as i don't know the inner workings of the timestamp type (just want to learn). I know that these functions are used frequently. I just looked at this page link. For all the three examples in that page, our result seems to match the expected result ?

spark-sql> values from_utc_timestamp('2011-12-25 09:00:00.123456', 'Asia/Tokyo');
2011-12-25 18:00:00.123456
Time taken: 2.289 seconds, Fetched 1 row(s)
spark-sql> values from_utc_timestamp('2014-11-02 06:55:00', 'America/Toronto');
2014-11-02 01:55:00
Time taken: 0.031 seconds, Fetched 1 row(s)
spark-sql> values from_utc_timestamp('2015-03-02 06:05:00', 'America/Toronto');
2015-03-02 01:05:00
Time taken: 0.029 seconds, Fetched 1 row(s)

@srowen
Copy link
Member

srowen commented Mar 26, 2019

@cloud-fan this does more than deprecate them, it makes them fail (though with an undocumented flag to reenable). This still feels a bit strong to me. Do we have evidence users are a) not using this at all or b) using it incorrectly? The function seems to work as intended w.r.t. the Hive function it's copying. Why do we really have to remove this? You can say, leave them enabled by default, but then why bother with a flag now?

@MaxGekk
Copy link
Member Author

MaxGekk commented Mar 26, 2019

@dilipbiswal Let's look at the first example, and assume that we are in the UTC-11:00 time zone:

scala> spark.conf.set("spark.sql.session.timeZone", "UTC-11:00")

and apply a function which assumes its input of TimestampType like HOUR:

scala> spark.sql("values HOUR(FROM_UTC_TIMESTAMP(TIMESTAMP '2011-12-25 09:00:00.123456', 'Asia/Tokyo'))").show(false)
+----+
|col1|
+----+
|5   |
+----+

The name of the function tells us we converted from a timestamp from UTC, so, in UTC the hour is 09 but we got 5 which does match to the time zone offset of 11 hours.

In general, problems occur when you apply functions or expressions to output of FROM_UTC_TIMESTAMP/TO_UTC_TIMESTAMP because such functions expect microseconds since epoch in UTC time zone but get as an input something different due FROM_UTC_TIMESTAMP/TO_UTC_TIMESTAMP shift UTC time points.

@srowen
Copy link
Member

srowen commented Mar 26, 2019

I think this is what is happening:

  1. At '2011-12-25 09:00:00' UTC, the time was '2011-12-25 18:00:00' in Asia/Tokyo (GMT+9)
  2. '2011-12-25 18:00:00' results in a timestamp w.r.t. the session timezone of GMT-11, a point in time which was also '2011-12-26 05:00:00' UTC
  3. HOUR returns '5'

Not 100% sure but that explains the result. It's weird and confusing, but is it the from_utc_timestamp part? Seems like step 2 is the problem.

I 'give in' on disabling the confusing function in step 1; its semantics are something even I have to think about 3 times and I even tried to fix the docs once.

But is my analysis right and are your other changes helping improve or fix the rest of it?

@SparkQA
Copy link

SparkQA commented Mar 26, 2019

Test build #103986 has finished for PR 24195 at commit daa1eb7.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dilipbiswal
Copy link
Contributor

@MaxGekk @srowen Thank you very much. I understand it now.

@MaxGekk
Copy link
Member Author

MaxGekk commented Mar 26, 2019

I have traced what is happening in the example:

  1. The TIMESTAMP typed literal takes 2011-12-25 09:00:00.123456, and calls stringToTimestamp with UTC-11:00 :
    toLiteral(stringToTimestamp(_, zoneId), TimestampType)
    which produces 1324843200123456. This is Sunday December 25, 2011 20:00:00 (pm) in time zone UTC.
  2. FROM_UTC_TIMESTAMP takes 1324843200123456 and fromZone = GMT, toZone = Asia/Tokyo (offset=32400000), and produces 1324875600123456 which is Monday December 26, 2011 05:00:00 (am) in time zone UTC (UTC)
  3. HOUR invokes DateTimeUtils.getHours with 1324875600123456 and timeZone = GMT (???), and produces just 5. The getHours is called from

It seems we have another issue in folding with wrong time zone here besides of FROM_UTC_TIMESTAMP.

@cloud-fan
Copy link
Contributor

long time ago I tried to improve these 2 functions in #21169 but now I can't remember all the details. These 2 functions are confusing as Spark's timestamp is UTC timestamp and users can't associate a timezone to it.

IIUC these two functions look correctly if the input is string and we cast the result to string. Otherwise these two functions do not make any sense.

@srowen
Copy link
Member

srowen commented Mar 26, 2019

Heh OK so the steps I guessed were 'right' but in the wrong order, sort of. Yeah might be something else confusing here.

I'm not against removing these methods as they are also confusing at best, but probably more going on here indeed.

I'm more inclined to support changes now that remove these ambiguities for Spark 3. It's really hard to understand what these do.

Note to self for a very future version of Spark: disallow all timezones but UTC internally. Force all timezones issues to input and output as much as possible. Timezones are such a source of errors.

if (!SQLConf.get.utcTimestampFuncEnabled) {
throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0. " +
s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.")
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan, I don't think this is a right way to deprecate. We should then at least rather throw a warning, and or we should better have a better mechanism to deprecate it in SQL side. Do we really want to add every configuration for every deprecation?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As I said, I don't think warning log is the right way to deprecate a SQL function, as users won't see it. This is not good either but this is the best I can think of.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then we should have the right way first before we start to do it differently from what I have done so far.

@adaitche
Copy link

adaitche commented Aug 7, 2019

Once these functions are deprecated/removed, will there be a way to shift a timestamp into another timezone? I do understand that internally timestamps are in UTC and one can not really shift it to another timezone. However, what if I want to correct a timestamp column?

For example:

  1. imagine a table with multiple timestamp columns
  2. some of these columns are not in UTC in the source system
  3. when loaded into Spark these are assumed to be in UTC, which is wrong
  4. to correct this, I would use to_utc_timestamp(column, source_timezone)

This use case arises often when dealing with ETL from a system where timestamps are not handled cleanly. How should it be addressed once these functions are gone?

@MaxGekk
Copy link
Member Author

MaxGekk commented Aug 7, 2019

However, what if I want to correct a timestamp column?

@adaitche You can shift timestamps by an interval:

spark-sql> select timestamp '2019-08-07 10:11:12' + INTERVAL '15' MINUTE;
2019-08-07 10:26:12

@adaitche
Copy link

adaitche commented Aug 7, 2019

@MaxGekk Thanks for the quick response.

If the source timezone is one with daylight saving (i.e. the offset changes twice a year), for example Europe/Berlin, simple manual shifting doesn't work. Of course, I could try to compute the time-dependent offset and shift by it. But then I would start implementing timezone-logic, which is tricky to get right.

@MaxGekk
Copy link
Member Author

MaxGekk commented Aug 7, 2019

Of course, I could try to compute the time-dependent offset and shift by it. But then I would start implementing timezone-logic, which is tricky to get right.

It seems you need a function which takes timestamps and zone id columns and returns zone offset as day-second interval. Like the code:

zoneId.getRules().getOffset(instant)

You can turn on spark.sql.datetime.java8API.enabled and write an UDF which takes an instance of java.time.Instant and a string with zone id.

@MaxGekk
Copy link
Member Author

MaxGekk commented Aug 7, 2019

@adaitche Please, open JIRA ticket for the feature.

@MaxGekk MaxGekk deleted the conv-utc-timestamp-deprecate branch September 18, 2019 15:57
@marmbrus
Copy link
Contributor

Hey everyone, I'm actually building a data pipeline right now and I just hit this. I think the solution we went with here is pretty actively hostile to users of Spark.

I'm worried that failing this function behind a flag (that has been available since Spark 1.5) just because it has confusing semantics is really going to hinder the adoption of Spark 3.0.

I would consider two different types of users here:

  • Someone building something new - Here, you give them no advice. We drop the ability work with timezones with no pointer on what they should do instead. It was also surprising to me that the function was both deprecated and threw and error from Scala. Everything I googled points back to this function. I'm now very frustrated.
  • Someone with an existing pipelines - Here, it probably doesn't matter that the function is "confusing", its doing what they want. Again, you don't give them a pointer to something better, you just annoy them by forcing them to change their code to specify this flag (and I assure you the vast majority of them will just set the flag without finding and reading this whole discussion).

Given the above, I think this PR should be reverted until you can at a minimum point the users to something better.

@gatorsmile
Copy link
Member

Yes, let us revert this PR. cc @MaxGekk

@MaxGekk
Copy link
Member Author

MaxGekk commented Jan 23, 2020

@marmbrus What is the use case when you need to shift UTC timestamps by zone offsets? At the moment, I see only one when you somehow placed non-UTC timestamps in to TIMESTAMP column which is by definition contains UTC timestamps. If it is not the case, you should be careful because any other Spark SQL functions/operators assume UTC timestamps, and you will get wrong results if you apply them.

@marmbrus
Copy link
Contributor

marmbrus commented Jan 23, 2020

From the JIRA REST I get a string that is a timestamp encoded in PST. My local clock is set to UTC. I would like to store the data in Delta as a UTC timestamp. This function seems to do what I want. Is there a better way to do this in Spark? I do not want to hard code the offset and do interval subtraction.

Regardless of the answer to the above, I think we are being hostile to our users with the current approach. I think we should back this out, and any other similar deprecation that are happening in Spark 3.0

  • DO build better ways to work with timestamps safely in Spark.
  • DO update the documentation to point out the caveats of bad APIs.
  • DO build mechanisms to let SQL users know about bad patterns.
  • DO NOT annoy users by breaking code that used to work unless you have to.
  • DO NOT force them to blindly set flags without giving them some better path forward.
  • DO NOT both deprecate and flag fail scala code paths.

The current error messages:

This function is deprecated and will be removed in future versions.

or

The to_utc_timestamp function has been disabled since Spark 3.0. Set spark.sql.legacy.utcTimestampFunc.enabled to true to enable this function.

These messages tell the user nothing, other than the fact that Spark developers would like to make it hard for them to upgrade their cluster. It doesn't even point them to the JIRA discussion (which IMO is still putting too much cognitive burden on them).

Do you think any of our users are going to do anything other than blindly set the flag when they see that message (That is what I did)? I worry that this accomplishes nothing other than frustration.

@srowen
Copy link
Member

srowen commented Jan 23, 2020

TL;DR: these functions are super confusing and so I am sad we had to emulate them from Hive. I think I also was not in favor of actually failing them though, but deprecation sounds OK. that is, I don’t think these functions are particularly good; they’re error prone; but I wouldn’t object to not failing them of course

@rxin
Copy link
Contributor

rxin commented Jan 23, 2020

I think a good suggestion (Michael already hinted) would be to either not fail, or fail with a helpful message telling user what to do.

@gatorsmile
Copy link
Member

The code freeze of 3.0 is close. So far, we do not have a user-friendly way to replace these deprecated functions. Let us first revert it and then discuss what is the next step #27351

@MaxGekk
Copy link
Member Author

MaxGekk commented Jan 24, 2020

From the JIRA REST I get a string that is a timestamp encoded in PST.

I guess if timestamp strings in PST, the strings contain timezone info. Looking at https://developer.atlassian.com/cloud/jira/platform/jira-expressions/#date-and-time, they are.

  • If you parse the strings via JSON datasource, you should set JSON option timestampFormat with time zone sub-pattern correctly.
  • If you want to parse it just as string, use def to_timestamp(s: Column, fmt: String).

Spark's TimestampFormatter takes into account time zone info

val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId
val zonedDateTime = toZonedDateTime(parsed, timeZoneId)
and shifts local timestamp respectively while converting to UTC timestamps:
val epochSeconds = zonedDateTime.toEpochSecond
val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND)

If input strings don't contain time zone info, you can either:

  • append time zone id string manually, or
  • temporary set SQL config spark.sql.session.timeZone while parsing timestamp strings.

For me, still not clear why did you decide to even look at the functions.

@adaitche
Copy link

adaitche commented Jan 24, 2020

If input strings don't contain time zone info, you can either:

  • append time zone id string manually, or
  • temporary set SQL config spark.sql.session.timeZone while parsing timestamp strings.

Regarding the first approach. With zone id string, you have something like +01:00 in mind, right? In many cases the offset is variable due to daylight saving time. I didn't find a way to specify a variable-offset-timezone in the datetime string. Is there any way?

Regarding the second approach. In my scenario this is not possible unfortunately. The timezone of the source datetime varies per row. The only working option for this scenario I am aware of is using the to_utc_timestamp or from_utc_timestamp functions.

I definitely agree that the semantics of these functions are difficult to understand which can lead to mistakes. It took me sometime to understand what they are actually doing. But unfortunately, I am not aware of an alternative.

@MaxGekk
Copy link
Member Author

MaxGekk commented Jan 24, 2020

With zone id string, you have something like +01:00 in mind, right?

You can say to TimestampFormatter that you set time zone name, see https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html:

scala> import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions._

scala> val df = Seq("2019-01-24 11:30:00.123 PST").toDF("ts_str")
df: org.apache.spark.sql.DataFrame = [ts_str: string]

scala> df.select(to_timestamp($"ts_str", "uuuu-MM-dd HH:mm:ss.SSS z")).show(false)
+---------------------------------------------------+
|to_timestamp(`ts_str`, 'uuuu-MM-dd HH:mm:ss.SSS z')|
+---------------------------------------------------+
|2019-01-24 22:30:00.123                            |
+---------------------------------------------------+

@MaxGekk
Copy link
Member Author

MaxGekk commented Jan 24, 2020

You can append time zone names per each row:

scala> val df = Seq(
  ("2019-01-24 11:30:00.123", "America/Los_Angeles"), 
  ("2020-01-01 01:30:00.123", "PST")).toDF("ts_str", "tz_name")
df: org.apache.spark.sql.DataFrame = [ts_str: string, tz_name: string]

scala> df.select(to_timestamp(concat_ws(" ", $"ts_str", $"tz_name"), "uuuu-MM-dd HH:mm:ss.SSS z").as("timestamp")).show(false)
+-----------------------+
|timestamp              |
+-----------------------+
|2019-01-24 22:30:00.123|
|2020-01-01 12:30:00.123|
+-----------------------+

@HyukjinKwon
Copy link
Member

@MaxGekk, so are all cases of from_utc_timestamp and to_utc_timestamp covered by existing APIs? If that's the case, what about we just remove the flag here and make the documentation better added in #24259?

I think that satisfies what @marmbrus pointed out at #24195 (comment) if I didn't misread.

@adaitche
Copy link

adaitche commented Jan 24, 2020

@MaxGekk

I just tried it in PySpark (Spark version 2.4.4). Weirdly it doesn't work for America/Los_Angeles.

Here is my code:

from pyspark.sql import SparkSession
from pyspark.sql.functions import to_timestamp, concat_ws

spark = (SparkSession
         .builder
         .config("spark.sql.session.timeZone", "UTC")
         .getOrCreate()
        )

df = spark.createDataFrame(
    [
        ("2019-01-24 11:30:00.123", "America/Los_Angeles"),
        ("2020-01-01 01:30:00.123", "PST"),
    ], 
    ["ts_str", "tz_name"]
)

df = df.withColumn("ts_str_tz", concat_ws(" ", "ts_str", "tz_name"))
df = df.withColumn("dt", to_timestamp("ts_str_tz", "yyyy-MM-dd HH:mm:ss.SSS z"))

df.show(10, False)

This prints

+-----------------------+-------------------+-------------------------------------------+-------------------+
|ts_str                 |tz_name            |ts_str_tz                                  |dt                 |
+-----------------------+-------------------+-------------------------------------------+-------------------+
|2019-01-24 11:30:00.123|America/Los_Angeles|2019-01-24 11:30:00.123 America/Los_Angeles|null               |
|2020-01-01 01:30:00.123|PST                |2020-01-01 01:30:00.123 PST                |2020-01-01 09:30:00|
+-----------------------+-------------------+-------------------------------------------+-------------------+

@MaxGekk
Copy link
Member Author

MaxGekk commented Jan 24, 2020

I just tried it in PySpark (Spark version 2.4.4)

Maybe 2.4 has a bug. I run the example on recent master. Please, fill in a JIRA ticket for 2.4. @adaitche I run your code on master in pyspark. It works fine:

>>> df = df.withColumn("dt", to_timestamp("ts_str_tz", "yyyy-MM-dd HH:mm:ss.SSS z"))
>>> df.show(10, False)
+-----------------------+-------------------+-------------------------------------------+-----------------------+
|ts_str                 |tz_name            |ts_str_tz                                  |dt                     |
+-----------------------+-------------------+-------------------------------------------+-----------------------+
|2019-01-24 11:30:00.123|America/Los_Angeles|2019-01-24 11:30:00.123 America/Los_Angeles|2019-01-24 19:30:00.123|
|2020-01-01 01:30:00.123|PST                |2020-01-01 01:30:00.123 PST                |2020-01-01 09:30:00.123|
+-----------------------+-------------------+-------------------------------------------+-----------------------+

@adaitche
Copy link

Created a Jira ticket here.

@MaxGekk I have also reproduced the same issue in pure Spark (vs PySpark) on versions 2.3.0 and 2.4.4. So, it has nothing to do with PySpark, as you also found.

@adaitche
Copy link

adaitche commented Jan 24, 2020

@MaxGekk Thanks for pointing out how to parse the datetime strings properly. I have tried it before by appending the timezone with a '+', which worked for offsets but not for timezone-names.

Once this bug is fixed, this approach will be an alternative to from_utc_timestamp and to_utc_timestamp, in my scenario.

@marmbrus
Copy link
Contributor

@MaxGekk thanks for showing the examples on parsing timestamps with timezones! I think I will change my code to use this mechanism instead.

That said, I remain strongly -1 on the idea that we should require users to set a flag to allow their code that was working to continue to work. I do not think that this is not how the Spark project operates. Look at the RDD API. There is a lot of cruft in there. That, however, is better than the alternative: breaking users programs or forcing them to stay on old versions.

I'm less strongly -1 on the deprecation, but I do think that it must include pointers to the correct way to accomplish what the user is trying to do. IMO, this string concatenation suggestion is more of a hack than from_utc_timestamp.

Given this, and the closeness to Spark 3.0. I still think the correct course of action is to back out this change and come back with a holistic plan for better timestamp handling in Spark.

Separate but relatedly, I'm not sure how you figured that magic incantation for to_timestamp. Looking at the docs for to_timestamp I cannot find anything that tells me what is allowed in the format field. I think fixing that would be a very high impact activity.

@dongjoon-hyun
Copy link
Member

Hi, All. According to the above discussion, I'll merge the reverting PR (#27351)

@HyukjinKwon
Copy link
Member

+1 for reverting for now.

@bart-samwel
Copy link

I think the to_timestamp conversion function should really have a per-function override on the default timezone. Appending strings is a hack, and it doesn't work if the input format changes or sometimes has a time zone. Setting a config doesn't work if you combine multiple input sources that have different default time zones (e.g. ETL data from multiple silos across the world), or if later parts of your pipeline rely on other functions that are also affected by this config.

@Megaprog
Copy link

Megaprog commented Jun 8, 2020

I suppose make some functions deprecated without clear alternative is not good. I hope the option "spark.sql.legacy.utcTimestampFunc.enabled" will work forever

@bart-samwel
Copy link

@MaxGekk How are we on the rebase function ("get me the same date + hands of the clock in a different timezone") that we talked about at some point?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.