This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new c2f79a6 [SPARK-31277][SQL][TESTS] Migrate `DateTimeTestUtils` from
`TimeZone` to `ZoneId`
c2f79a6 is described below
commit c2f79a61de2af529999ff9ef8321094f2ea7ea9b
Author: Maxim Gekk <[email protected]>
AuthorDate: Fri Mar 27 21:14:25 2020 +0800
[SPARK-31277][SQL][TESTS] Migrate `DateTimeTestUtils` from `TimeZone` to
`ZoneId`
In the PR, I propose to change types of `DateTimeTestUtils` values and
functions by replacing `java.util.TimeZone` to `java.time.ZoneId`. In
particular:
1. Type of `ALL_TIMEZONES` is changed to `Seq[ZoneId]`.
2. Remove `val outstandingTimezones: Seq[TimeZone]`.
3. Change the type of the time zone parameter in `withDefaultTimeZone` to
`ZoneId`.
4. Modify affected test suites.
Currently, Spark SQL's date-time expressions and functions have been
already ported on Java 8 time API but tests still use old time APIs. In
particular, `DateTimeTestUtils` exposes functions that accept only TimeZone
instances. This is inconvenient, and CPU consuming because need to convert
TimeZone instances to ZoneId instances via strings (zone ids).
No
By affected test suites executed by jenkins builds.
Closes #28033 from MaxGekk/with-default-time-zone.
Authored-by: Maxim Gekk <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
(cherry picked from commit 9f0c010a5c9b5c10401f8e16fa7a151714b6dbb0)
Signed-off-by: Wenchen Fan <[email protected]>
---
.../org/apache/spark/sql/avro/AvroSuite.scala | 4 +-
.../spark/sql/catalyst/expressions/CastSuite.scala | 9 ++--
.../expressions/CollectionExpressionsSuite.scala | 4 +-
.../catalyst/expressions/CsvExpressionsSuite.scala | 10 ++--
.../expressions/DateExpressionsSuite.scala | 63 +++++++++++-----------
.../expressions/JsonExpressionsSuite.scala | 8 +--
.../catalyst/parser/ExpressionParserSuite.scala | 6 +--
.../sql/catalyst/util/DateTimeTestUtils.scala | 15 +++---
.../sql/catalyst/util/DateTimeUtilsSuite.scala | 54 +++++++++----------
.../apache/spark/sql/DataFrameFunctionsSuite.scala | 2 +-
.../spark/sql/StatisticsCollectionSuite.scala | 3 +-
11 files changed, 91 insertions(+), 87 deletions(-)
diff --git
a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
index 34a0e2b..a04037c 100644
--- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
+++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.TestingUDT.IntervalData
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.Filter
-import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils}
+import
org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, UTC}
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.datasources.{DataSource, FilePartition}
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
@@ -408,7 +408,7 @@ abstract class AvroSuite extends QueryTest with
SharedSparkSession {
StructField("float", FloatType, true),
StructField("date", DateType, true)
))
- DateTimeTestUtils.withDefaultTimeZone(DateTimeUtils.TimeZoneUTC) {
+ withDefaultTimeZone(UTC) {
val rdd = spark.sparkContext.parallelize(Seq(
Row(1f, null),
Row(2f, new Date(1451948400000L)),
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
index e25d805..5c57843 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
@@ -112,13 +112,14 @@ abstract class CastSuiteBase extends SparkFunSuite with
ExpressionEvalHelper {
}
test("cast string to timestamp") {
- new ParVector(ALL_TIMEZONES.toVector).foreach { tz =>
+ new ParVector(ALL_TIMEZONES.toVector).foreach { zid =>
def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit =
{
- checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)),
expected)
+ checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)),
expected)
}
checkCastStringToTimestamp("123", null)
+ val tz = TimeZone.getTimeZone(zid)
var c = Calendar.getInstance(tz)
c.set(2015, 0, 1, 0, 0, 0)
c.set(Calendar.MILLISECOND, 0)
@@ -263,10 +264,10 @@ abstract class CastSuiteBase extends SparkFunSuite with
ExpressionEvalHelper {
val zts = sd + " 00:00:00"
val sts = sd + " 00:00:02"
val nts = sts + ".1"
- val ts = withDefaultTimeZone(TimeZoneGMT)(Timestamp.valueOf(nts))
+ val ts = withDefaultTimeZone(UTC)(Timestamp.valueOf(nts))
for (tz <- ALL_TIMEZONES) {
- val timeZoneId = Option(tz.getID)
+ val timeZoneId = Option(tz.getId)
var c = Calendar.getInstance(TimeZoneGMT)
c.set(2015, 2, 8, 2, 30, 0)
checkEvaluation(
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
index 00cdc17..9177382 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
@@ -852,7 +852,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper
DateTimeUtils.fromMillis(Timestamp.valueOf(s).getTime - offset)
}
- DateTimeTestUtils.withDefaultTimeZone(timeZone) {
+ DateTimeTestUtils.withDefaultTimeZone(timeZone.toZoneId) {
// Spring time change
checkEvaluation(new Sequence(
Literal(Timestamp.valueOf("2018-03-25 01:30:00")),
@@ -880,7 +880,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper
}
test("Sequence of dates") {
- DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
+ DateTimeTestUtils.withDefaultTimeZone(UTC) {
checkEvaluation(new Sequence(
Literal(Date.valueOf("2018-01-01")),
Literal(Date.valueOf("2018-01-05")),
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala
index 24cf933..16d78b0 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import java.text.SimpleDateFormat
-import java.util.{Calendar, Locale}
+import java.util.{Calendar, Locale, TimeZone}
import org.scalatest.exceptions.TestFailedException
@@ -89,8 +89,8 @@ class CsvExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper with P
)
val csvData2 = "2016-01-01T00:00:00"
- for (tz <- DateTimeTestUtils.outstandingTimezones) {
- c = Calendar.getInstance(tz)
+ for (zid <- DateTimeTestUtils.outstandingZoneIds) {
+ c = Calendar.getInstance(TimeZone.getTimeZone(zid))
c.set(2016, 0, 1, 0, 0, 0)
c.set(Calendar.MILLISECOND, 0)
checkEvaluation(
@@ -98,14 +98,14 @@ class CsvExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper with P
schema,
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"),
Literal(csvData2),
- Option(tz.getID)),
+ Option(zid.getId)),
InternalRow(c.getTimeInMillis * 1000L)
)
checkEvaluation(
CsvToStructs(
schema,
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
- DateTimeUtils.TIMEZONE_OPTION -> tz.getID),
+ DateTimeUtils.TIMEZONE_OPTION -> zid.getId),
Literal(csvData2),
UTC_OPT),
InternalRow(c.getTimeInMillis * 1000L)
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 2cc6259..350b3a4 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -185,9 +185,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
checkEvaluation(Second(Literal(ts), UTC_OPT), 15)
val c = Calendar.getInstance()
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
- c.setTimeZone(tz)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ c.setTimeZone(TimeZone.getTimeZone(zid))
(0 to 60 by 5).foreach { s =>
c.set(2015, 18, 3, 3, 5, s)
checkEvaluation(
@@ -285,9 +285,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
checkEvaluation(Hour(Literal(ts), UTC_OPT), 13)
val c = Calendar.getInstance()
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
- c.setTimeZone(tz)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ c.setTimeZone(TimeZone.getTimeZone(zid))
(0 to 24 by 6).foreach { h =>
(0 to 60 by 30).foreach { m =>
(0 to 60 by 30).foreach { s =>
@@ -312,9 +312,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
checkEvaluation(Minute(Literal(ts), UTC_OPT), 10)
val c = Calendar.getInstance()
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
- c.setTimeZone(tz)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ c.setTimeZone(TimeZone.getTimeZone(zid))
(0 to 60 by 5).foreach { m =>
(0 to 60 by 15).foreach { s =>
c.set(2015, 18, 3, 3, m, s)
@@ -384,9 +384,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
test("time_add") {
val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
- sdf.setTimeZone(tz)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ sdf.setTimeZone(TimeZone.getTimeZone(zid))
checkEvaluation(
TimeAdd(
@@ -422,9 +422,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
test("time_sub") {
val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
- sdf.setTimeZone(tz)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ sdf.setTimeZone(TimeZone.getTimeZone(zid))
checkEvaluation(
TimeSub(
@@ -508,9 +508,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
test("months_between") {
val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
- sdf.setTimeZone(tz)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ sdf.setTimeZone(TimeZone.getTimeZone(zid))
checkEvaluation(
MonthsBetween(
@@ -641,7 +641,7 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
expected)
}
- withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
+ withDefaultTimeZone(UTC) {
val inputDate = Timestamp.valueOf("2015-07-22 05:30:06")
Seq("yyyy", "YYYY", "year", "YEAR", "yy", "YY").foreach { fmt =>
@@ -711,8 +711,9 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
val sdf1 = new SimpleDateFormat(fmt1, Locale.US)
val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS"
val sdf2 = new SimpleDateFormat(fmt2, Locale.US)
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ val tz = TimeZone.getTimeZone(zid)
sdf1.setTimeZone(tz)
sdf2.setTimeZone(tz)
@@ -762,9 +763,10 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
sdf3.setTimeZone(TimeZone.getTimeZone(UTC))
- withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
+ withDefaultTimeZone(UTC) {
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ val tz = TimeZone.getTimeZone(zid)
sdf1.setTimeZone(tz)
sdf2.setTimeZone(tz)
@@ -829,9 +831,10 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
sdf3.setTimeZone(TimeZone.getTimeZone(UTC))
- withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
- for (tz <- outstandingTimezones) {
- val timeZoneId = Option(tz.getID)
+ withDefaultTimeZone(UTC) {
+ for (zid <- outstandingZoneIds) {
+ val timeZoneId = Option(zid.getId)
+ val tz = TimeZone.getTimeZone(zid)
sdf1.setTimeZone(tz)
sdf2.setTimeZone(tz)
@@ -847,7 +850,7 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
checkEvaluation(
ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId),
MILLISECONDS.toSeconds(
- DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1),
tz.toZoneId)))
+ DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1),
zid)))
checkEvaluation(
ToUnixTimestamp(
Literal(sdf2.format(new Timestamp(-1000000))),
@@ -856,7 +859,7 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
checkEvaluation(ToUnixTimestamp(
Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3),
timeZoneId),
MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(
- DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")),
tz.toZoneId)))
+ DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), zid)))
val t1 = ToUnixTimestamp(
CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long]
val t2 = ToUnixTimestamp(
@@ -871,7 +874,7 @@ class DateExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper {
checkEvaluation(ToUnixTimestamp(
Literal(date1), Literal.create(null, StringType), timeZoneId),
MILLISECONDS.toSeconds(
- DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1),
tz.toZoneId)))
+ DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1),
zid)))
checkEvaluation(
ToUnixTimestamp(
Literal("2015-07-24"),
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
index dec2e46..032e0ac 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala
@@ -517,8 +517,8 @@ class JsonExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper with
)
val jsonData2 = """{"t": "2016-01-01T00:00:00"}"""
- for (tz <- DateTimeTestUtils.outstandingTimezones) {
- c = Calendar.getInstance(tz)
+ for (zid <- DateTimeTestUtils.outstandingZoneIds) {
+ c = Calendar.getInstance(TimeZone.getTimeZone(zid))
c.set(2016, 0, 1, 0, 0, 0)
c.set(Calendar.MILLISECOND, 0)
checkEvaluation(
@@ -526,14 +526,14 @@ class JsonExpressionsSuite extends SparkFunSuite with
ExpressionEvalHelper with
schema,
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"),
Literal(jsonData2),
- Option(tz.getID)),
+ Option(zid.getId)),
InternalRow(c.getTimeInMillis * 1000L)
)
checkEvaluation(
JsonToStructs(
schema,
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
- DateTimeUtils.TIMEZONE_OPTION -> tz.getID),
+ DateTimeUtils.TIMEZONE_OPTION -> zid.getId),
Literal(jsonData2),
UTC_OPT),
InternalRow(c.getTimeInMillis * 1000L)
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
index 74fd48d..522d49e 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
@@ -772,10 +772,10 @@ class ExpressionParserSuite extends AnalysisTest {
}
test("timestamp literals") {
- DateTimeTestUtils.outstandingTimezones.foreach { timeZone =>
- withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone.getID) {
+ DateTimeTestUtils.outstandingZoneIds.foreach { zid =>
+ withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId) {
def toMicros(time: LocalDateTime): Long = {
- val seconds = time.atZone(timeZone.toZoneId).toInstant.getEpochSecond
+ val seconds = time.atZone(zid).toInstant.getEpochSecond
TimeUnit.SECONDS.toMicros(seconds)
}
assertEval(
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala
index 374400e..bf9e8f7 100644
---
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala
@@ -21,6 +21,8 @@ import java.time.{LocalDate, LocalDateTime, LocalTime,
ZoneId, ZoneOffset}
import java.util.TimeZone
import java.util.concurrent.TimeUnit
+import scala.collection.JavaConverters._
+
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId
@@ -39,7 +41,7 @@ object DateTimeTestUtils {
val UTC_OPT = Option("UTC")
- val ALL_TIMEZONES: Seq[TimeZone] =
TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone)
+ val ALL_TIMEZONES: Seq[ZoneId] =
ZoneId.getAvailableZoneIds.asScala.map(getZoneId).toSeq
val outstandingTimezonesIds: Seq[String] = Seq(
"UTC",
@@ -50,16 +52,15 @@ object DateTimeTestUtils {
"Antarctica/Vostok",
"Asia/Hong_Kong",
"Europe/Amsterdam")
- val outstandingTimezones: Seq[TimeZone] =
outstandingTimezonesIds.map(TimeZone.getTimeZone)
- val outstandingZoneIds: Seq[ZoneId] =
outstandingTimezonesIds.map(DateTimeUtils.getZoneId)
+ val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId)
- def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = {
- val originalDefaultTimeZone = TimeZone.getDefault
+ def withDefaultTimeZone[T](newDefaultTimeZone: ZoneId)(block: => T): T = {
+ val originalDefaultTimeZone = ZoneId.systemDefault()
try {
- TimeZone.setDefault(newDefaultTimeZone)
+ TimeZone.setDefault(TimeZone.getTimeZone(newDefaultTimeZone))
block
} finally {
- TimeZone.setDefault(originalDefaultTimeZone)
+ TimeZone.setDefault(TimeZone.getTimeZone(originalDefaultTimeZone))
}
}
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 5eca864..922e09d 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
@@ -154,11 +154,10 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
test("string to timestamp") {
- for (tz <- ALL_TIMEZONES) {
+ for (zid <- ALL_TIMEZONES) {
def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = {
- assert(toTimestamp(str, tz.toZoneId) === expected)
+ assert(toTimestamp(str, zid) === expected)
}
- val zid = tz.toZoneId
checkStringToTimestamp("1969-12-31 16:00:00", Option(date(1969, 12, 31,
16, zid = zid)))
checkStringToTimestamp("0001", Option(date(1, 1, 1, 0, zid = zid)))
@@ -465,7 +464,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
}
- withDefaultTimeZone(TimeZone.getTimeZone(LA.getId)) {
+ withDefaultTimeZone(LA) {
// Daylight Saving Time
test("2016-03-13 09:59:59.0", LA.getId, "2016-03-13 01:59:59.0")
test("2016-03-13 10:00:00.0", LA.getId, "2016-03-13 03:00:00.0")
@@ -481,8 +480,8 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
=== expected)
}
- for (tz <- ALL_TIMEZONES) {
- withDefaultTimeZone(tz) {
+ for (zid <- ALL_TIMEZONES) {
+ withDefaultTimeZone(zid) {
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.getId, "2011-12-25
09:00:00.123456")
test("2011-12-25 01:00:00.123456", LA.getId, "2011-12-25
09:00:00.123456")
@@ -490,8 +489,8 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
}
- val tz = LA.getId
- withDefaultTimeZone(TimeZone.getTimeZone(tz)) {
+ withDefaultTimeZone(LA) {
+ val tz = LA.getId
// Daylight Saving Time
test("2016-03-13 01:59:59", tz, "2016-03-13 09:59:59.0")
test("2016-03-13 02:00:00", tz, "2016-03-13 10:00:00.0")
@@ -552,9 +551,8 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
testTrunc(DateTimeUtils.TRUNC_TO_CENTURY, "2001-01-01", defaultInputTS.get)
testTrunc(DateTimeUtils.TRUNC_TO_MILLENNIUM, "2001-01-01",
defaultInputTS.get)
- for (tz <- ALL_TIMEZONES) {
- withDefaultTimeZone(tz) {
- val zid = tz.toZoneId
+ for (zid <- ALL_TIMEZONES) {
+ withDefaultTimeZone(zid) {
val inputTS = DateTimeUtils.stringToTimestamp(
UTF8String.fromString("2015-03-05T09:32:05.359"), defaultZoneId)
val inputTS1 = DateTimeUtils.stringToTimestamp(
@@ -610,14 +608,14 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
"Pacific/Kiritimati" -> Set(9130, 9131),
"Pacific/Kwajalein" -> Set(8632, 8633, 8634),
MIT.getId -> Set(15338))
- for (tz <- ALL_TIMEZONES) {
- val skipped = skipped_days.getOrElse(tz.getID, Set.empty)
+ for (zid <- ALL_TIMEZONES) {
+ val skipped = skipped_days.getOrElse(zid.getId, Set.empty)
val testingData = Seq(-20000, 20000) ++
(1 to 1000).map(_ => (math.random() * 40000 - 20000).toInt)
testingData.foreach { d =>
if (!skipped.contains(d)) {
- assert(millisToDays(daysToMillis(d, tz.toZoneId), tz.toZoneId) === d,
- s"Round trip of $d did not work in tz $tz")
+ assert(millisToDays(daysToMillis(d, zid), zid) === d,
+ s"Round trip of $d did not work in tz ${zid.getId}")
}
}
}
@@ -672,8 +670,8 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
test("rebase julian to/from gregorian micros") {
- outstandingTimezones.foreach { timeZone =>
- withDefaultTimeZone(timeZone) {
+ outstandingZoneIds.foreach { zid =>
+ withDefaultTimeZone(zid) {
Seq(
"0001-01-01 01:02:03.654321",
"1000-01-01 03:02:01.123456",
@@ -684,9 +682,9 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
"1969-12-31 11:22:33.000100",
"1970-01-01 00:00:00.000001", // The epoch day
"2020-03-14 09:33:01.500000").foreach { ts =>
- withClue(s"time zone = ${timeZone.getID} ts = $ts") {
+ withClue(s"time zone = ${zid.getId} ts = $ts") {
val julianMicros = parseToJulianMicros(ts)
- val gregMicros = parseToGregMicros(ts.replace(' ', 'T'),
timeZone.toZoneId)
+ val gregMicros = parseToGregMicros(ts.replace(' ', 'T'), zid)
assert(rebaseJulianToGregorianMicros(julianMicros) === gregMicros)
assert(rebaseGregorianToJulianMicros(gregMicros) === julianMicros)
@@ -706,8 +704,8 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
test("rebase gregorian to/from julian days") {
- outstandingTimezones.foreach { timeZone =>
- withDefaultTimeZone(timeZone) {
+ outstandingZoneIds.foreach { zid =>
+ withDefaultTimeZone(zid) {
Seq(
"0001-01-01",
"1000-01-01",
@@ -729,14 +727,14 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
test("rebase julian to gregorian date for leap years") {
- outstandingTimezones.foreach { timeZone =>
- withDefaultTimeZone(timeZone) {
+ outstandingZoneIds.foreach { zid =>
+ withDefaultTimeZone(zid) {
Seq(
"1000-02-29" -> "1000-03-01",
"1600-02-29" -> "1600-02-29",
"1700-02-29" -> "1700-03-01",
"2000-02-29" -> "2000-02-29").foreach { case (julianDate, gregDate)
=>
- withClue(s"tz = ${timeZone.getID} julian date = $julianDate greg
date = $gregDate") {
+ withClue(s"tz = ${zid.getId} julian date = $julianDate greg date =
$gregDate") {
val date = Date.valueOf(julianDate)
val julianDays = fromJavaDateLegacy(date)
val gregorianDays = localDateToDays(LocalDate.parse(gregDate))
@@ -749,17 +747,17 @@ class DateTimeUtilsSuite extends SparkFunSuite with
Matchers with SQLHelper {
}
test("rebase julian to gregorian timestamp for leap years") {
- outstandingTimezones.foreach { timeZone =>
- withDefaultTimeZone(timeZone) {
+ outstandingZoneIds.foreach { zid =>
+ withDefaultTimeZone(zid) {
Seq(
"1000-02-29 01:02:03.123456" -> "1000-03-01T01:02:03.123456",
"1600-02-29 11:12:13.654321" -> "1600-02-29T11:12:13.654321",
"1700-02-29 21:22:23.000001" -> "1700-03-01T21:22:23.000001",
"2000-02-29 00:00:00.999999" -> "2000-02-29T00:00:00.999999"
).foreach { case (julianTs, gregTs) =>
- withClue(s"tz = ${timeZone.getID} julian ts = $julianTs greg ts =
$gregTs") {
+ withClue(s"tz = ${zid.getId} julian ts = $julianTs greg ts =
$gregTs") {
val julianMicros = parseToJulianMicros(julianTs)
- val gregorianMicros = parseToGregMicros(gregTs, timeZone.toZoneId)
+ val gregorianMicros = parseToGregMicros(gregTs, zid)
assert(rebaseJulianToGregorianMicros(julianMicros) ===
gregorianMicros)
}
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index 9100ee3..ac98d3f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -1029,7 +1029,7 @@ class DataFrameFunctionsSuite extends QueryTest with
SharedSparkSession {
Timestamp.valueOf("2018-01-01 12:00:00"),
Timestamp.valueOf("2018-01-02 00:00:00")))))
- DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
+ DateTimeTestUtils.withDefaultTimeZone(UTC) {
checkAnswer(
spark.sql("select sequence(" +
" cast('2018-01-01' as date)" +
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
index 30b15a8..708b98e 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
@@ -481,7 +481,8 @@ class StatisticsCollectionSuite extends
StatisticsCollectionTestBase with Shared
}
}
- DateTimeTestUtils.outstandingTimezones.foreach { timeZone =>
+ DateTimeTestUtils.outstandingZoneIds.foreach { zid =>
+ val timeZone = TimeZone.getTimeZone(zid)
checkTimestampStats(DateType, DateTimeUtils.TimeZoneUTC, timeZone) {
stats =>
assert(stats.min.get.asInstanceOf[Int] ==
TimeUnit.SECONDS.toDays(start))
assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end
- 1))
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]