aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala10
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala34
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala65
3 files changed, 73 insertions, 36 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 773431dd33..04c17bdaf2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -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)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 56bf9a7863..df480a1d65 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -886,13 +886,37 @@ object DateTimeUtils {
}
/**
+ * 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)
}
/**
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
index f9cb97629f..059a5b7d07 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
@@ -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")
-
- // 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")
+ 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")
+ }
+ }
+
+ 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") {