[SPARK-16078][SQL] from_utc_timestamp/to_utc_timestamp should not depends on local timezone

## What changes were proposed in this pull request?

Currently, we use local timezone to parse or format a timestamp (TimestampType), then use Long as the microseconds since epoch UTC.

In from_utc_timestamp() and to_utc_timestamp(), we did not consider the local timezone, they could return different results with different local timezone.

This PR will do the conversion based on human time (in local timezone), it should return same result in whatever timezone. But because the mapping from absolute timestamp to human time is not exactly one-to-one mapping, it will still return wrong result in some timezone (also in the begging or ending of DST).

This PR is kind of the best effort fix. In long term, we should make the TimestampType be timezone aware to fix this totally.

## How was this patch tested?

Tested these function in all timezone.

Author: Davies Liu <davies@databricks.com>

Closes #13784 from davies/convert_tz.
This commit is contained in:
Davies Liu 2016-06-22 13:40:24 -07:00 committed by Herman van Hovell
parent 43b04b7ecb
commit 20d411bc5d
3 changed files with 71 additions and 34 deletions

View file

@ -730,16 +730,17 @@ case class FromUTCTimestamp(left: Expression, right: Expression)
""".stripMargin)
} else {
val tzTerm = ctx.freshName("tz")
val utcTerm = ctx.freshName("utc")
val tzClass = classOf[TimeZone].getName
ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""")
ctx.addMutableState(tzClass, utcTerm, s"""$utcTerm = $tzClass.getTimeZone("UTC");""")
val eval = left.genCode(ctx)
ev.copy(code = s"""
|${eval.code}
|boolean ${ev.isNull} = ${eval.isNull};
|long ${ev.value} = 0;
|if (!${ev.isNull}) {
| ${ev.value} = ${eval.value} +
| ${tzTerm}.getOffset(${eval.value} / 1000) * 1000L;
| ${ev.value} = $dtu.convertTz(${eval.value}, $utcTerm, $tzTerm);
|}
""".stripMargin)
}
@ -869,16 +870,17 @@ case class ToUTCTimestamp(left: Expression, right: Expression)
""".stripMargin)
} else {
val tzTerm = ctx.freshName("tz")
val utcTerm = ctx.freshName("utc")
val tzClass = classOf[TimeZone].getName
ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""")
ctx.addMutableState(tzClass, utcTerm, s"""$utcTerm = $tzClass.getTimeZone("UTC");""")
val eval = left.genCode(ctx)
ev.copy(code = s"""
|${eval.code}
|boolean ${ev.isNull} = ${eval.isNull};
|long ${ev.value} = 0;
|if (!${ev.isNull}) {
| ${ev.value} = ${eval.value} -
| ${tzTerm}.getOffset(${eval.value} / 1000) * 1000L;
| ${ev.value} = $dtu.convertTz(${eval.value}, $tzTerm, $utcTerm);
|}
""".stripMargin)
}

View file

@ -885,14 +885,38 @@ object DateTimeUtils {
guess
}
/**
* Convert the timestamp `ts` from one timezone to another.
*
* TODO: Because of DST, the conversion between UTC and human time is not exactly one-to-one
* mapping, the conversion here may return wrong result, we should make the timestamp
* timezone-aware.
*/
def convertTz(ts: SQLTimestamp, fromZone: TimeZone, toZone: TimeZone): SQLTimestamp = {
// We always use local timezone to parse or format a timestamp
val localZone = threadLocalLocalTimeZone.get()
val utcTs = if (fromZone.getID == localZone.getID) {
ts
} else {
// get the human time using local time zone, that actually is in fromZone.
val localTs = ts + localZone.getOffset(ts / 1000L) * 1000L // in fromZone
localTs - getOffsetFromLocalMillis(localTs / 1000L, fromZone) * 1000L
}
if (toZone.getID == localZone.getID) {
utcTs
} else {
val localTs2 = utcTs + toZone.getOffset(utcTs / 1000L) * 1000L // in toZone
// treat it as local timezone, convert to UTC (we could get the expected human time back)
localTs2 - getOffsetFromLocalMillis(localTs2 / 1000L, localZone) * 1000L
}
}
/**
* Returns a timestamp of given timezone from utc timestamp, with the same string
* representation in their timezone.
*/
def fromUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = {
val tz = TimeZone.getTimeZone(timeZone)
val offset = tz.getOffset(time / 1000L)
time + offset * 1000L
convertTz(time, TimeZoneGMT, TimeZone.getTimeZone(timeZone))
}
/**
@ -900,9 +924,7 @@ object DateTimeUtils {
* string representation in their timezone.
*/
def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = {
val tz = TimeZone.getTimeZone(timeZone)
val offset = getOffsetFromLocalMillis(time / 1000L, tz)
time - offset * 1000L
convertTz(time, TimeZone.getTimeZone(timeZone), TimeZoneGMT)
}
/**

View file

@ -488,17 +488,23 @@ class DateTimeUtilsSuite extends SparkFunSuite {
assert(toJavaTimestamp(fromUTCTime(fromJavaTimestamp(Timestamp.valueOf(utc)), tz)).toString
=== expected)
}
test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456")
test("2011-12-25 09:00:00.123456", "JST", "2011-12-25 18:00:00.123456")
test("2011-12-25 09:00:00.123456", "PST", "2011-12-25 01:00:00.123456")
test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456")
for (tz <- DateTimeTestUtils.ALL_TIMEZONES) {
DateTimeTestUtils.withDefaultTimeZone(tz) {
test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456")
test("2011-12-25 09:00:00.123456", "JST", "2011-12-25 18:00:00.123456")
test("2011-12-25 09:00:00.123456", "PST", "2011-12-25 01:00:00.123456")
test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456")
}
}
// Daylight Saving Time
test("2016-03-13 09:59:59.0", "PST", "2016-03-13 01:59:59.0")
test("2016-03-13 10:00:00.0", "PST", "2016-03-13 03:00:00.0")
test("2016-11-06 08:59:59.0", "PST", "2016-11-06 01:59:59.0")
test("2016-11-06 09:00:00.0", "PST", "2016-11-06 01:00:00.0")
test("2016-11-06 10:00:00.0", "PST", "2016-11-06 02:00:00.0")
DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("PST")) {
// Daylight Saving Time
test("2016-03-13 09:59:59.0", "PST", "2016-03-13 01:59:59.0")
test("2016-03-13 10:00:00.0", "PST", "2016-03-13 03:00:00.0")
test("2016-11-06 08:59:59.0", "PST", "2016-11-06 01:59:59.0")
test("2016-11-06 09:00:00.0", "PST", "2016-11-06 01:00:00.0")
test("2016-11-06 10:00:00.0", "PST", "2016-11-06 02:00:00.0")
}
}
test("to UTC timestamp") {
@ -506,21 +512,28 @@ class DateTimeUtilsSuite extends SparkFunSuite {
assert(toJavaTimestamp(toUTCTime(fromJavaTimestamp(Timestamp.valueOf(utc)), tz)).toString
=== expected)
}
test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456")
test("2011-12-25 18:00:00.123456", "JST", "2011-12-25 09:00:00.123456")
test("2011-12-25 01:00:00.123456", "PST", "2011-12-25 09:00:00.123456")
test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456")
// Daylight Saving Time
test("2016-03-13 01:59:59", "PST", "2016-03-13 09:59:59.0")
// 2016-03-13 02:00:00 PST does not exists
test("2016-03-13 02:00:00", "PST", "2016-03-13 10:00:00.0")
test("2016-03-13 03:00:00", "PST", "2016-03-13 10:00:00.0")
test("2016-11-06 00:59:59", "PST", "2016-11-06 07:59:59.0")
// 2016-11-06 01:00:00 PST could be 2016-11-06 08:00:00 UTC or 2016-11-06 09:00:00 UTC
test("2016-11-06 01:00:00", "PST", "2016-11-06 09:00:00.0")
test("2016-11-06 01:59:59", "PST", "2016-11-06 09:59:59.0")
test("2016-11-06 02:00:00", "PST", "2016-11-06 10:00:00.0")
for (tz <- DateTimeTestUtils.ALL_TIMEZONES) {
DateTimeTestUtils.withDefaultTimeZone(tz) {
test("2011-12-25 09:00:00.123456", "UTC", "2011-12-25 09:00:00.123456")
test("2011-12-25 18:00:00.123456", "JST", "2011-12-25 09:00:00.123456")
test("2011-12-25 01:00:00.123456", "PST", "2011-12-25 09:00:00.123456")
test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456")
}
}
DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("PST")) {
// Daylight Saving Time
test("2016-03-13 01:59:59", "PST", "2016-03-13 09:59:59.0")
// 2016-03-13 02:00:00 PST does not exists
test("2016-03-13 02:00:00", "PST", "2016-03-13 10:00:00.0")
test("2016-03-13 03:00:00", "PST", "2016-03-13 10:00:00.0")
test("2016-11-06 00:59:59", "PST", "2016-11-06 07:59:59.0")
// 2016-11-06 01:00:00 PST could be 2016-11-06 08:00:00 UTC or 2016-11-06 09:00:00 UTC
test("2016-11-06 01:00:00", "PST", "2016-11-06 09:00:00.0")
test("2016-11-06 01:59:59", "PST", "2016-11-06 09:59:59.0")
test("2016-11-06 02:00:00", "PST", "2016-11-06 10:00:00.0")
}
}
test("daysToMillis and millisToDays") {