You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2020/03/26 06:05:06 UTC
[spark] branch branch-3.0 updated: [SPARK-31237][SQL][TESTS]
Replace 3-letter time zones by zone offsets
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 6b568d7 [SPARK-31237][SQL][TESTS] Replace 3-letter time zones by zone offsets
6b568d7 is described below
commit 6b568d77fc0284b0b8a26b16fd105a7c5f54f874
Author: Maxim Gekk <ma...@gmail.com>
AuthorDate: Thu Mar 26 13:36:00 2020 +0800
[SPARK-31237][SQL][TESTS] Replace 3-letter time zones by zone offsets
In the PR, I propose to add a few `ZoneId` constant values to the `DateTimeTestUtils` object, and reuse the constants in tests. Proposed the following constants:
- PST = -08:00
- UTC = +00:00
- CEST = +02:00
- CET = +01:00
- JST = +09:00
- MIT = -09:30
- LA = America/Los_Angeles
All proposed constant values (except `LA`) are initialized by zone offsets according to their definitions. This will allow to avoid:
- Using of 3-letter time zones that have been already deprecated in JDK, see _Three-letter time zone IDs_ in https://docs.oracle.com/javase/8/docs/api/java/util/TimeZone.html
- Incorrect mapping of 3-letter time zones to zone offsets, see SPARK-31237. For example, `PST` is mapped to `America/Los_Angeles` instead of the `-08:00` zone offset.
Also this should improve stability and maintainability of test suites.
No
By running affected test suites.
Closes #28001 from MaxGekk/replace-pst.
Authored-by: Maxim Gekk <ma...@gmail.com>
Signed-off-by: Wenchen Fan <we...@databricks.com>
(cherry picked from commit cec9604eaec2e5ff17e705ed60565bd7506c6374)
Signed-off-by: Wenchen Fan <we...@databricks.com>
---
.../sql/catalyst/csv/CSVInferSchemaSuite.scala | 28 ++--
.../sql/catalyst/csv/UnivocityParserSuite.scala | 45 +++---
.../spark/sql/catalyst/expressions/CastSuite.scala | 27 ++--
.../catalyst/expressions/CodeGenerationSuite.scala | 5 +-
.../expressions/CollectionExpressionsSuite.scala | 3 +-
.../catalyst/expressions/CsvExpressionsSuite.scala | 47 +++---
.../expressions/DateExpressionsSuite.scala | 177 ++++++++++----------
.../expressions/JsonExpressionsSuite.scala | 77 +++++----
.../sql/catalyst/util/DateTimeTestUtils.scala | 17 +-
.../sql/catalyst/util/DateTimeUtilsSuite.scala | 178 ++++++++++-----------
.../apache/spark/sql/util/ArrowUtilsSuite.scala | 3 +-
.../spark/sql/util/TimestampFormatterSuite.scala | 11 +-
.../apache/spark/sql/DataFrameFunctionsSuite.scala | 3 +-
.../org/apache/spark/sql/DataFramePivotSuite.scala | 2 +-
.../org/apache/spark/sql/DataFrameSuite.scala | 4 +-
.../org/apache/spark/sql/DateFunctionsSuite.scala | 21 +--
.../sql/execution/datasources/csv/CSVSuite.scala | 4 +-
.../sql/execution/datasources/json/JsonSuite.scala | 10 +-
.../parquet/ParquetPartitionDiscoverySuite.scala | 10 +-
.../apache/spark/sql/internal/SQLConfSuite.scala | 5 +-
.../spark/sql/sources/PartitionedWriteSuite.scala | 6 +-
.../sql/streaming/EventTimeWatermarkSuite.scala | 3 +-
22 files changed, 346 insertions(+), 340 deletions(-)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala
index ee73da3..b014eb9 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.types._
class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
test("String fields types are inferred correctly from null types") {
- val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "GMT")
+ val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(NullType, "") == NullType)
@@ -48,7 +48,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("String fields types are inferred correctly from other types") {
- val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "GMT")
+ val options = new CSVOptions(Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"), false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(LongType, "1.0") == DoubleType)
@@ -69,18 +69,18 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("Timestamp field types are inferred correctly via custom data format") {
- var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "GMT")
+ var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "UTC")
var inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType)
- options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "GMT")
+ options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "UTC")
inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(TimestampType, "2015") == TimestampType)
}
test("Timestamp field types are inferred correctly from other types") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(IntegerType, "2015-08-20 14") == StringType)
@@ -89,7 +89,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("Boolean fields types are inferred correctly from other types") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(LongType, "Fale") == StringType)
@@ -97,7 +97,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("Type arrays are merged to highest common type") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(
@@ -112,14 +112,14 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("Null fields are handled properly when a nullValue is specified") {
- var options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT")
+ var options = new CSVOptions(Map("nullValue" -> "null"), false, "UTC")
var inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(NullType, "null") == NullType)
assert(inferSchema.inferField(StringType, "null") == StringType)
assert(inferSchema.inferField(LongType, "null") == LongType)
- options = new CSVOptions(Map("nullValue" -> "\\N"), false, "GMT")
+ options = new CSVOptions(Map("nullValue" -> "\\N"), false, "UTC")
inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(IntegerType, "\\N") == IntegerType)
@@ -130,7 +130,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("Merging Nulltypes should yield Nulltype.") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val inferSchema = new CSVInferSchema(options)
val mergedNullTypes = inferSchema.mergeRowTypes(Array(NullType), Array(NullType))
@@ -138,14 +138,14 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
}
test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") {
- val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "GMT")
+ val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType)
}
test("SPARK-18877: `inferField` on DecimalType should find a common type with `typeSoFar`") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val inferSchema = new CSVInferSchema(options)
withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") {
@@ -166,7 +166,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
test("DoubleType should be inferred when user defined nan/inf are provided") {
val options = new CSVOptions(Map("nanValue" -> "nan", "negativeInf" -> "-inf",
- "positiveInf" -> "inf"), false, "GMT")
+ "positiveInf" -> "inf"), false, "UTC")
val inferSchema = new CSVInferSchema(options)
assert(inferSchema.inferField(NullType, "nan") == DoubleType)
@@ -179,7 +179,7 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper {
val options = new CSVOptions(
parameters = Map("locale" -> langTag, "inferSchema" -> "true", "sep" -> "|"),
columnPruning = false,
- defaultTimeZoneId = "GMT")
+ defaultTimeZoneId = "UTC")
val inferSchema = new CSVInferSchema(options)
val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag)))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala
index c8c71bc..4853b4f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.csv
import java.math.BigDecimal
import java.text.{DecimalFormat, DecimalFormatSymbols}
-import java.time.ZoneOffset
import java.util.{Locale, TimeZone}
import org.apache.commons.lang3.time.FastDateFormat
@@ -44,7 +43,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) =>
val decimalValue = new BigDecimal(decimalVal.toString)
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
assert(parser.makeConverter("_1", decimalType).apply(strVal) ===
Decimal(decimalValue, decimalType.precision, decimalType.scale))
@@ -58,14 +57,14 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
// Nullable field with nullValue option.
types.foreach { t =>
// Tests that a custom nullValue.
- val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "GMT")
+ val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "UTC")
var parser = new UnivocityParser(StructType(Seq.empty), nullValueOptions)
val converter = parser.makeConverter("_1", t, nullable = true)
assertNull(converter.apply("-"))
assertNull(converter.apply(null))
// Tests that the default nullValue is empty string.
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
parser = new UnivocityParser(StructType(Seq.empty), options)
assertNull(parser.makeConverter("_1", t, nullable = true).apply(""))
}
@@ -73,7 +72,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
// Not nullable field with nullValue option.
types.foreach { t =>
// Casts a null to not nullable field should throw an exception.
- val options = new CSVOptions(Map("nullValue" -> "-"), false, "GMT")
+ val options = new CSVOptions(Map("nullValue" -> "-"), false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val converter = parser.makeConverter("_1", t, nullable = false)
var message = intercept[RuntimeException] {
@@ -89,7 +88,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
// If nullValue is different with empty string, then, empty string should not be casted into
// null.
Seq(true, false).foreach { b =>
- val options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT")
+ val options = new CSVOptions(Map("nullValue" -> "null"), false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val converter = parser.makeConverter("_1", StringType, nullable = b)
assert(converter.apply("") == UTF8String.fromString(""))
@@ -97,7 +96,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Throws exception for empty string with non null type") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val exception = intercept[RuntimeException]{
parser.makeConverter("_1", IntegerType, nullable = false).apply("")
@@ -106,7 +105,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Types are cast correctly") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
var parser = new UnivocityParser(StructType(Seq.empty), options)
assert(parser.makeConverter("_1", ByteType).apply("10") == 10)
assert(parser.makeConverter("_1", ShortType).apply("10") == 10)
@@ -117,7 +116,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
assert(parser.makeConverter("_1", BooleanType).apply("true") == true)
var timestampsOptions =
- new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy HH:mm"), false, "GMT")
+ new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy HH:mm"), false, "UTC")
parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions)
val customTimestamp = "31/01/2015 00:00"
var format = FastDateFormat.getInstance(
@@ -130,7 +129,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
assert(castedTimestamp == expectedTime * 1000L)
val customDate = "31/01/2015"
- val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "GMT")
+ val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "UTC")
parser = new UnivocityParser(StructType(Seq.empty), dateOptions)
format = FastDateFormat.getInstance(
dateOptions.dateFormat,
@@ -139,7 +138,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
val expectedDate = format.parse(customDate).getTime
val castedDate = parser.makeConverter("_1", DateType, nullable = true)
.apply(customDate)
- assert(castedDate == DateTimeUtils.millisToDays(expectedDate, ZoneOffset.UTC))
+ assert(castedDate == DateTimeUtils.millisToDays(expectedDate, UTC))
val timestamp = "2015-01-01 00:00:00"
timestampsOptions = new CSVOptions(Map(
@@ -154,7 +153,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Throws exception for casting an invalid string to Float and Double Types") {
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val types = Seq(DoubleType, FloatType)
val input = Seq("10u000", "abc", "1 2/3")
@@ -169,7 +168,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Float NaN values are parsed correctly") {
- val options = new CSVOptions(Map("nanValue" -> "nn"), false, "GMT")
+ val options = new CSVOptions(Map("nanValue" -> "nn"), false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val floatVal: Float = parser.makeConverter(
"_1", FloatType, nullable = true).apply("nn").asInstanceOf[Float]
@@ -180,7 +179,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Double NaN values are parsed correctly") {
- val options = new CSVOptions(Map("nanValue" -> "-"), false, "GMT")
+ val options = new CSVOptions(Map("nanValue" -> "-"), false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val doubleVal: Double = parser.makeConverter(
"_1", DoubleType, nullable = true).apply("-").asInstanceOf[Double]
@@ -189,14 +188,14 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Float infinite values can be parsed") {
- val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT")
+ val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "UTC")
var parser = new UnivocityParser(StructType(Seq.empty), negativeInfOptions)
val floatVal1 = parser.makeConverter(
"_1", FloatType, nullable = true).apply("max").asInstanceOf[Float]
assert(floatVal1 == Float.NegativeInfinity)
- val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT")
+ val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "UTC")
parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions)
val floatVal2 = parser.makeConverter(
"_1", FloatType, nullable = true).apply("max").asInstanceOf[Float]
@@ -205,14 +204,14 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
}
test("Double infinite values can be parsed") {
- val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT")
+ val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "UTC")
var parser = new UnivocityParser(StructType(Seq.empty), negativeInfOptions)
val doubleVal1 = parser.makeConverter(
"_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double]
assert(doubleVal1 == Double.NegativeInfinity)
- val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT")
+ val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "UTC")
parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions)
val doubleVal2 = parser.makeConverter(
"_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double]
@@ -228,7 +227,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag)))
val input = df.format(expected.toBigDecimal)
- val options = new CSVOptions(Map("locale" -> langTag), false, "GMT")
+ val options = new CSVOptions(Map("locale" -> langTag), false, "UTC")
val parser = new UnivocityParser(new StructType().add("d", decimalType), options)
assert(parser.makeConverter("_1", decimalType).apply(input) === expected)
@@ -263,7 +262,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
val input = "name\t42"
val expected = UTF8String.fromString(input)
- val options = new CSVOptions(Map.empty[String, String], false, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], false, "UTC")
val parser = new UnivocityParser(StructType(Seq.empty), options)
val convertedValue = parser.makeConverter("_1", StringBasedUDT, nullable = false).apply(input)
@@ -280,7 +279,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
filters: Seq[Filter],
expected: Option[InternalRow]): Unit = {
Seq(false, true).foreach { columnPruning =>
- val options = new CSVOptions(Map.empty[String, String], columnPruning, "GMT")
+ val options = new CSVOptions(Map.empty[String, String], columnPruning, "UTC")
val parser = new UnivocityParser(dataSchema, requiredSchema, options, filters)
val actual = parser.parse(input)
assert(actual === expected)
@@ -355,8 +354,8 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
val options = new CSVOptions(Map.empty[String, String], false, "UTC")
check(new UnivocityParser(StructType(Seq.empty), options))
- val optionsWithPattern =
- new CSVOptions(Map("timestampFormat" -> "invalid", "dateFormat" -> "invalid"), false, "UTC")
+ val optionsWithPattern = new CSVOptions(
+ Map("timestampFormat" -> "invalid", "dateFormat" -> "invalid"), false, "UTC")
check(new UnivocityParser(StructType(Seq.empty), optionsWithPattern))
}
}
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 e5e28b6..dd3f362 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
@@ -49,7 +49,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
}
protected def checkNullCast(from: DataType, to: DataType): Unit = {
- checkEvaluation(cast(Literal.create(null, from), to, Option("GMT")), null)
+ checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null)
}
test("null cast") {
@@ -141,7 +141,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
// If the string value includes timezone string, it represents the timestamp string
// in the timezone regardless of the timeZoneId parameter.
- c = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
+ c = Calendar.getInstance(TimeZone.getTimeZone(UTC))
c.set(2015, 2, 18, 12, 3, 17)
c.set(Calendar.MILLISECOND, 0)
checkCastStringToTimestamp("2015-03-18T12:03:17Z", new Timestamp(c.getTimeInMillis))
@@ -172,7 +172,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
// If the string value includes timezone string, it represents the timestamp string
// in the timezone regardless of the timeZoneId parameter.
- c = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
+ c = Calendar.getInstance(TimeZone.getTimeZone(UTC))
c.set(2015, 2, 18, 12, 3, 17)
c.set(Calendar.MILLISECOND, 456)
checkCastStringToTimestamp("2015-03-18T12:03:17.456Z", new Timestamp(c.getTimeInMillis))
@@ -280,23 +280,23 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
fromMillis(c.getTimeInMillis))
}
- val gmtId = Option("GMT")
-
checkEvaluation(cast("abdef", StringType), "abdef")
- checkEvaluation(cast("abdef", TimestampType, gmtId), null)
+ checkEvaluation(cast("abdef", TimestampType, UTC_OPT), null)
checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65))
checkEvaluation(cast(cast(sd, DateType), StringType), sd)
checkEvaluation(cast(cast(d, StringType), DateType), 0)
- checkEvaluation(cast(cast(nts, TimestampType, gmtId), StringType, gmtId), nts)
+ checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts)
checkEvaluation(
- cast(cast(ts, StringType, gmtId), TimestampType, gmtId),
+ cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT),
DateTimeUtils.fromJavaTimestamp(ts))
// all convert to string type to check
- checkEvaluation(cast(cast(cast(nts, TimestampType, gmtId), DateType, gmtId), StringType), sd)
checkEvaluation(
- cast(cast(cast(ts, DateType, gmtId), TimestampType, gmtId), StringType, gmtId),
+ cast(cast(cast(nts, TimestampType, UTC_OPT), DateType, UTC_OPT), StringType),
+ sd)
+ checkEvaluation(
+ cast(cast(cast(ts, DateType, UTC_OPT), TimestampType, UTC_OPT), StringType, UTC_OPT),
zts)
checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef")
@@ -309,7 +309,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
5.toShort)
checkEvaluation(
- cast(cast(cast(cast(cast(cast("5", TimestampType, gmtId), ByteType),
+ cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType),
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
null)
checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT),
@@ -360,8 +360,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(cast(d, DecimalType(10, 2)), null)
checkEvaluation(cast(d, StringType), "1970-01-01")
- val gmtId = Option("GMT")
- checkEvaluation(cast(cast(d, TimestampType, gmtId), StringType, gmtId), "1970-01-01 00:00:00")
+ checkEvaluation(
+ cast(cast(d, TimestampType, UTC_OPT), StringType, UTC_OPT),
+ "1970-01-01 00:00:00")
}
test("cast from timestamp") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
index b7628a3..db52229 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.expressions.objects._
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils}
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.LA
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.ThreadUtils
@@ -190,7 +191,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper {
val expressions = Seq.fill(length) {
ToUTCTimestamp(
Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType),
- Literal.create("PST", StringType))
+ Literal.create(LA.getId, StringType))
}
val plan = GenerateMutableProjection.generate(expressions)
val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType))
@@ -207,7 +208,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper {
val expressions = Seq.fill(length) {
ToUTCTimestamp(
Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType),
- Literal.create("PST", StringType))
+ Literal.create(LA.getId, StringType))
}
val plan = GenerateMutableProjection.generate(expressions)
val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType))
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 bb31a06..00cdc17 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
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils}
import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC
import org.apache.spark.sql.catalyst.util.IntervalUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
@@ -879,7 +880,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
}
test("Sequence of dates") {
- DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("UTC")) {
+ DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone(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 e623910..24cf933 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
@@ -27,19 +27,18 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.plans.PlanTestBase
import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{PST, UTC_OPT}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with PlanTestBase {
val badCsv = "\u0000\u0000\u0000A\u0001AAA"
- val gmtId = Option(DateTimeUtils.TimeZoneGMT.getID)
-
test("from_csv") {
val csvData = "1"
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal(csvData), gmtId),
+ CsvToStructs(schema, Map.empty, Literal(csvData), UTC_OPT),
InternalRow(1)
)
}
@@ -48,17 +47,17 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
val csvData = "---"
val schema = StructType(StructField("a", DoubleType) :: Nil)
checkEvaluation(
- CsvToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(csvData), gmtId),
+ CsvToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(csvData), UTC_OPT),
InternalRow(null))
// Default mode is Permissive
- checkEvaluation(CsvToStructs(schema, Map.empty, Literal(csvData), gmtId), InternalRow(null))
+ checkEvaluation(CsvToStructs(schema, Map.empty, Literal(csvData), UTC_OPT), InternalRow(null))
}
test("from_csv null input column") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal.create(null, StringType), gmtId),
+ CsvToStructs(schema, Map.empty, Literal.create(null, StringType), UTC_OPT),
null
)
}
@@ -66,7 +65,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
test("from_csv bad UTF-8") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal(badCsv), gmtId),
+ CsvToStructs(schema, Map.empty, Literal(badCsv), UTC_OPT),
InternalRow(null))
}
@@ -78,14 +77,14 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
c.set(2016, 0, 1, 0, 0, 0)
c.set(Calendar.MILLISECOND, 123)
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal(csvData1), gmtId),
+ CsvToStructs(schema, Map.empty, Literal(csvData1), UTC_OPT),
InternalRow(c.getTimeInMillis * 1000L)
)
// The result doesn't change because the CSV string includes timezone string ("Z" here),
// which means the string represents the timestamp string in the timezone regardless of
// the timeZoneId parameter.
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal(csvData1), Option("PST")),
+ CsvToStructs(schema, Map.empty, Literal(csvData1), Option(PST.getId)),
InternalRow(c.getTimeInMillis * 1000L)
)
@@ -108,7 +107,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
DateTimeUtils.TIMEZONE_OPTION -> tz.getID),
Literal(csvData2),
- gmtId),
+ UTC_OPT),
InternalRow(c.getTimeInMillis * 1000L)
)
}
@@ -117,7 +116,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
test("from_csv empty input column") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId),
+ CsvToStructs(schema, Map.empty, Literal.create(" ", StringType), UTC_OPT),
InternalRow(null)
)
}
@@ -129,7 +128,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
.add("b", StringType, nullable = false)
.add("c", StringType, nullable = false)
val output = InternalRow(1L, null, UTF8String.fromString("foo"))
- val expr = CsvToStructs(csvSchema, Map.empty, Literal.create(input, StringType), gmtId)
+ val expr = CsvToStructs(csvSchema, Map.empty, Literal.create(input, StringType), UTC_OPT)
checkEvaluation(expr, output)
val schema = expr.dataType
val schemaToCompare = csvSchema.asNullable
@@ -142,7 +141,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
.add("a", IntegerType)
.add("b", IntegerType)
checkEvaluation(
- CsvToStructs(schema, Map.empty, Literal.create("1"), gmtId),
+ CsvToStructs(schema, Map.empty, Literal.create("1"), UTC_OPT),
InternalRow(1, null)
)
}
@@ -152,7 +151,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
val schema = StructType(StructField("a", DoubleType) :: Nil)
val exception = intercept[TestFailedException] {
checkEvaluation(
- CsvToStructs(schema, Map("mode" -> DropMalformedMode.name), Literal(csvData), gmtId),
+ CsvToStructs(schema, Map("mode" -> DropMalformedMode.name), Literal(csvData), UTC_OPT),
InternalRow(null))
}.getCause
assert(exception.getMessage.contains("from_csv() doesn't support the DROPMALFORMED mode"))
@@ -171,14 +170,14 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
test("to_csv - struct") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
val struct = Literal.create(create_row(1), schema)
- checkEvaluation(StructsToCsv(Map.empty, struct, gmtId), "1")
+ checkEvaluation(StructsToCsv(Map.empty, struct, UTC_OPT), "1")
}
test("to_csv null input column") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
val struct = Literal.create(null, schema)
checkEvaluation(
- StructsToCsv(Map.empty, struct, gmtId),
+ StructsToCsv(Map.empty, struct, UTC_OPT),
null
)
}
@@ -190,24 +189,24 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
c.set(Calendar.MILLISECOND, 0)
val struct = Literal.create(create_row(c.getTimeInMillis * 1000L), schema)
- checkEvaluation(StructsToCsv(Map.empty, struct, gmtId), "2016-01-01T00:00:00.000Z")
+ checkEvaluation(StructsToCsv(Map.empty, struct, UTC_OPT), "2016-01-01T00:00:00.000Z")
checkEvaluation(
- StructsToCsv(Map.empty, struct, Option("PST")), "2015-12-31T16:00:00.000-08:00")
+ StructsToCsv(Map.empty, struct, Option(PST.getId)), "2015-12-31T16:00:00.000-08:00")
checkEvaluation(
StructsToCsv(
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
- DateTimeUtils.TIMEZONE_OPTION -> gmtId.get),
+ DateTimeUtils.TIMEZONE_OPTION -> UTC_OPT.get),
struct,
- gmtId),
+ UTC_OPT),
"2016-01-01T00:00:00"
)
checkEvaluation(
StructsToCsv(
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
- DateTimeUtils.TIMEZONE_OPTION -> "PST"),
+ DateTimeUtils.TIMEZONE_OPTION -> PST.getId),
struct,
- gmtId),
+ UTC_OPT),
"2015-12-31T16:00:00"
)
}
@@ -223,7 +222,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
val options = Map("dateFormat" -> dateFormat, "locale" -> langTag)
checkEvaluation(
- CsvToStructs(schema, options, Literal.create(dateStr), gmtId),
+ CsvToStructs(schema, options, Literal.create(dateStr), UTC_OPT),
InternalRow(17836)) // number of days from 1970-01-01
}
}
@@ -234,7 +233,7 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P
schema = StructType.fromDDL("i int, _unparsed boolean"),
options = Map("columnNameOfCorruptRecord" -> "_unparsed"),
child = Literal.create("a"),
- timeZoneId = gmtId),
+ timeZoneId = UTC_OPT),
expectedErrMsg = "The field for corrupt records must be string type and nullable")
}
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 4a04fbb..2cc6259 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
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
-import java.time.{Instant, LocalDate, LocalDateTime, ZoneId, ZoneOffset}
+import java.time.{Instant, LocalDate, LocalDateTime, ZoneId}
import java.util.{Calendar, Locale, TimeZone}
import java.util.concurrent.TimeUnit._
@@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
-import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
@@ -38,15 +37,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
import IntegralLiteralTestUtils._
- val TimeZonePST = TimeZone.getTimeZone("PST")
- val TimeZoneJST = TimeZone.getTimeZone("JST")
-
- val gmtId = Option(TimeZoneGMT.getID)
- val pstId = Option(TimeZonePST.getID)
- val jstId = Option(TimeZoneJST.getID)
+ private val PST_OPT = Option(PST.getId)
+ private val JST_OPT = Option(JST.getId)
def toMillis(timestamp: String): Long = {
- val tf = TimestampFormatter("yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC)
+ val tf = TimestampFormatter("yyyy-MM-dd HH:mm:ss", UTC)
DateTimeUtils.toMillis(tf.parse(timestamp))
}
val date = "2015-04-08 13:10:15"
@@ -55,13 +50,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val ts = new Timestamp(toMillis(time))
test("datetime function current_date") {
- val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC)
- val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int]
- val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC)
+ val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), UTC)
+ val cd = CurrentDate(UTC_OPT).eval(EmptyRow).asInstanceOf[Int]
+ val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), UTC)
assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1)
- val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int]
- val cdpst = CurrentDate(pstId).eval(EmptyRow).asInstanceOf[Int]
+ val cdjst = CurrentDate(JST_OPT).eval(EmptyRow).asInstanceOf[Int]
+ val cdpst = CurrentDate(PST_OPT).eval(EmptyRow).asInstanceOf[Int]
assert(cdpst <= cd && cd <= cdjst)
}
@@ -93,8 +88,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("Year") {
checkEvaluation(Year(Literal.create(null, DateType)), null)
checkEvaluation(Year(Literal(d)), 2015)
- checkEvaluation(Year(Cast(Literal(date), DateType, gmtId)), 2015)
- checkEvaluation(Year(Cast(Literal(ts), DateType, gmtId)), 2013)
+ checkEvaluation(Year(Cast(Literal(date), DateType, UTC_OPT)), 2015)
+ checkEvaluation(Year(Cast(Literal(ts), DateType, UTC_OPT)), 2013)
val c = Calendar.getInstance()
(2000 to 2002).foreach { y =>
@@ -115,8 +110,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("Quarter") {
checkEvaluation(Quarter(Literal.create(null, DateType)), null)
checkEvaluation(Quarter(Literal(d)), 2)
- checkEvaluation(Quarter(Cast(Literal(date), DateType, gmtId)), 2)
- checkEvaluation(Quarter(Cast(Literal(ts), DateType, gmtId)), 4)
+ checkEvaluation(Quarter(Cast(Literal(date), DateType, UTC_OPT)), 2)
+ checkEvaluation(Quarter(Cast(Literal(ts), DateType, UTC_OPT)), 4)
val c = Calendar.getInstance()
(2003 to 2004).foreach { y =>
@@ -138,8 +133,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("Month") {
checkEvaluation(Month(Literal.create(null, DateType)), null)
checkEvaluation(Month(Literal(d)), 4)
- checkEvaluation(Month(Cast(Literal(date), DateType, gmtId)), 4)
- checkEvaluation(Month(Cast(Literal(ts), DateType, gmtId)), 11)
+ checkEvaluation(Month(Cast(Literal(date), DateType, UTC_OPT)), 4)
+ checkEvaluation(Month(Cast(Literal(ts), DateType, UTC_OPT)), 11)
checkEvaluation(Month(Cast(Literal("1582-04-28 13:10:15"), DateType)), 4)
checkEvaluation(Month(Cast(Literal("1582-10-04 13:10:15"), DateType)), 10)
@@ -163,8 +158,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(DayOfMonth(Cast(Literal("2000-02-29"), DateType)), 29)
checkEvaluation(DayOfMonth(Literal.create(null, DateType)), null)
checkEvaluation(DayOfMonth(Literal(d)), 8)
- checkEvaluation(DayOfMonth(Cast(Literal(date), DateType, gmtId)), 8)
- checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, gmtId)), 8)
+ checkEvaluation(DayOfMonth(Cast(Literal(date), DateType, UTC_OPT)), 8)
+ checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, UTC_OPT)), 8)
checkEvaluation(DayOfMonth(Cast(Literal("1582-04-28 13:10:15"), DateType)), 28)
checkEvaluation(DayOfMonth(Cast(Literal("1582-10-15 13:10:15"), DateType)), 15)
@@ -183,14 +178,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("Seconds") {
- assert(Second(Literal.create(null, DateType), gmtId).resolved === false)
- assert(Second(Cast(Literal(d), TimestampType, gmtId), gmtId).resolved )
- checkEvaluation(Second(Cast(Literal(d), TimestampType, gmtId), gmtId), 0)
- checkEvaluation(Second(Cast(Literal(date), TimestampType, gmtId), gmtId), 15)
- checkEvaluation(Second(Literal(ts), gmtId), 15)
+ assert(Second(Literal.create(null, DateType), UTC_OPT).resolved === false)
+ assert(Second(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT).resolved )
+ checkEvaluation(Second(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT), 0)
+ checkEvaluation(Second(Cast(Literal(date), TimestampType, UTC_OPT), UTC_OPT), 15)
+ checkEvaluation(Second(Literal(ts), UTC_OPT), 15)
val c = Calendar.getInstance()
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
c.setTimeZone(tz)
(0 to 60 by 5).foreach { s =>
@@ -207,10 +202,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("DayOfWeek") {
checkEvaluation(DayOfWeek(Literal.create(null, DateType)), null)
checkEvaluation(DayOfWeek(Literal(d)), Calendar.WEDNESDAY)
- checkEvaluation(DayOfWeek(Cast(Literal(date), DateType, gmtId)),
+ checkEvaluation(DayOfWeek(Cast(Literal(date), DateType, UTC_OPT)),
Calendar.WEDNESDAY)
- checkEvaluation(DayOfWeek(Cast(Literal(ts), DateType, gmtId)), Calendar.FRIDAY)
- checkEvaluation(DayOfWeek(Cast(Literal("2011-05-06"), DateType, gmtId)), Calendar.FRIDAY)
+ checkEvaluation(DayOfWeek(Cast(Literal(ts), DateType, UTC_OPT)), Calendar.FRIDAY)
+ checkEvaluation(DayOfWeek(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), Calendar.FRIDAY)
checkEvaluation(DayOfWeek(Literal(new Date(toMillis("2017-05-27 13:10:15")))),
Calendar.SATURDAY)
checkEvaluation(DayOfWeek(Literal(new Date(toMillis("1582-10-15 13:10:15")))),
@@ -221,9 +216,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("WeekDay") {
checkEvaluation(WeekDay(Literal.create(null, DateType)), null)
checkEvaluation(WeekDay(Literal(d)), 2)
- checkEvaluation(WeekDay(Cast(Literal(date), DateType, gmtId)), 2)
- checkEvaluation(WeekDay(Cast(Literal(ts), DateType, gmtId)), 4)
- checkEvaluation(WeekDay(Cast(Literal("2011-05-06"), DateType, gmtId)), 4)
+ checkEvaluation(WeekDay(Cast(Literal(date), DateType, UTC_OPT)), 2)
+ checkEvaluation(WeekDay(Cast(Literal(ts), DateType, UTC_OPT)), 4)
+ checkEvaluation(WeekDay(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), 4)
checkEvaluation(WeekDay(Literal(new Date(toMillis("2017-05-27 13:10:15")))), 5)
checkEvaluation(WeekDay(Literal(new Date(toMillis("1582-10-15 13:10:15")))), 4)
checkConsistencyBetweenInterpretedAndCodegen(WeekDay, DateType)
@@ -232,11 +227,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("WeekOfYear") {
checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null)
checkEvaluation(WeekOfYear(Literal(d)), 15)
- checkEvaluation(WeekOfYear(Cast(Literal(date), DateType, gmtId)), 15)
- checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType, gmtId)), 45)
- checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType, gmtId)), 18)
- checkEvaluation(WeekOfYear(Cast(Literal("1582-10-15 13:10:15"), DateType, gmtId)), 41)
- checkEvaluation(WeekOfYear(Cast(Literal("1582-10-04 13:10:15"), DateType, gmtId)), 40)
+ checkEvaluation(WeekOfYear(Cast(Literal(date), DateType, UTC_OPT)), 15)
+ checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType, UTC_OPT)), 45)
+ checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType, UTC_OPT)), 18)
+ checkEvaluation(WeekOfYear(Cast(Literal("1582-10-15 13:10:15"), DateType, UTC_OPT)), 41)
+ checkEvaluation(WeekOfYear(Cast(Literal("1582-10-04 13:10:15"), DateType, UTC_OPT)), 40)
checkConsistencyBetweenInterpretedAndCodegen(WeekOfYear, DateType)
}
@@ -244,38 +239,38 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
Seq("legacy", "corrected").foreach { legacyParserPolicy =>
withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy) {
checkEvaluation(
- DateFormatClass(Literal.create(null, TimestampType), Literal("y"), gmtId),
+ DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT),
null)
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId),
- Literal.create(null, StringType), gmtId), null)
-
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId),
- Literal("y"), gmtId), "2015")
- checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), gmtId), "2013")
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId),
- Literal("H"), gmtId), "0")
- checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), gmtId), "13")
-
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId),
- Literal("y"), pstId), "2015")
- checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), pstId), "2013")
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId),
- Literal("H"), pstId), "0")
- checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), pstId), "5")
-
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId),
- Literal("y"), jstId), "2015")
- checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), jstId), "2013")
- checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId),
- Literal("H"), jstId), "0")
- checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), jstId), "22")
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT),
+ Literal.create(null, StringType), UTC_OPT), null)
+
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT),
+ Literal("y"), UTC_OPT), "2015")
+ checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013")
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT),
+ Literal("H"), UTC_OPT), "0")
+ checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13")
+
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT),
+ Literal("y"), PST_OPT), "2015")
+ checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013")
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT),
+ Literal("H"), PST_OPT), "0")
+ checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5")
+
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT),
+ Literal("y"), JST_OPT), "2015")
+ checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013")
+ checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT),
+ Literal("H"), JST_OPT), "0")
+ checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22")
// SPARK-28072 The codegen path should work
checkEvaluation(
expression = DateFormatClass(
BoundReference(ordinal = 0, dataType = TimestampType, nullable = true),
BoundReference(ordinal = 1, dataType = StringType, nullable = true),
- jstId),
+ JST_OPT),
expected = "22",
inputRow = InternalRow(DateTimeUtils.fromJavaTimestamp(ts), UTF8String.fromString("H")))
}
@@ -283,14 +278,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("Hour") {
- assert(Hour(Literal.create(null, DateType), gmtId).resolved === false)
- assert(Hour(Literal(ts), gmtId).resolved)
- checkEvaluation(Hour(Cast(Literal(d), TimestampType, gmtId), gmtId), 0)
- checkEvaluation(Hour(Cast(Literal(date), TimestampType, gmtId), gmtId), 13)
- checkEvaluation(Hour(Literal(ts), gmtId), 13)
+ assert(Hour(Literal.create(null, DateType), UTC_OPT).resolved === false)
+ assert(Hour(Literal(ts), UTC_OPT).resolved)
+ checkEvaluation(Hour(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT), 0)
+ checkEvaluation(Hour(Cast(Literal(date), TimestampType, UTC_OPT), UTC_OPT), 13)
+ checkEvaluation(Hour(Literal(ts), UTC_OPT), 13)
val c = Calendar.getInstance()
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
c.setTimeZone(tz)
(0 to 24 by 6).foreach { h =>
@@ -309,15 +304,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("Minute") {
- assert(Minute(Literal.create(null, DateType), gmtId).resolved === false)
- assert(Minute(Literal(ts), gmtId).resolved)
- checkEvaluation(Minute(Cast(Literal(d), TimestampType, gmtId), gmtId), 0)
+ assert(Minute(Literal.create(null, DateType), UTC_OPT).resolved === false)
+ assert(Minute(Literal(ts), UTC_OPT).resolved)
+ checkEvaluation(Minute(Cast(Literal(d), TimestampType, UTC_OPT), UTC_OPT), 0)
checkEvaluation(
- Minute(Cast(Literal(date), TimestampType, gmtId), gmtId), 10)
- checkEvaluation(Minute(Literal(ts), gmtId), 10)
+ Minute(Cast(Literal(date), TimestampType, UTC_OPT), UTC_OPT), 10)
+ checkEvaluation(Minute(Literal(ts), UTC_OPT), 10)
val c = Calendar.getInstance()
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
c.setTimeZone(tz)
(0 to 60 by 5).foreach { m =>
@@ -389,7 +384,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("time_add") {
val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
sdf.setTimeZone(tz)
@@ -427,7 +422,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("time_sub") {
val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS", Locale.US)
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
sdf.setTimeZone(tz)
@@ -513,7 +508,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("months_between") {
val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
sdf.setTimeZone(tz)
@@ -646,7 +641,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
expected)
}
- withDefaultTimeZone(TimeZoneGMT) {
+ withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
val inputDate = Timestamp.valueOf("2015-07-22 05:30:06")
Seq("yyyy", "YYYY", "year", "YEAR", "yy", "YY").foreach { fmt =>
@@ -716,7 +711,7 @@ 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 <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
sdf1.setTimeZone(tz)
sdf2.setTimeZone(tz)
@@ -765,10 +760,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val sdf2 = new SimpleDateFormat(fmt2, Locale.US)
val fmt3 = "yy-MM-dd"
val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
- sdf3.setTimeZone(TimeZoneGMT)
+ sdf3.setTimeZone(TimeZone.getTimeZone(UTC))
- withDefaultTimeZone(TimeZoneGMT) {
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
sdf1.setTimeZone(tz)
sdf2.setTimeZone(tz)
@@ -832,10 +827,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val sdf2 = new SimpleDateFormat(fmt2, Locale.US)
val fmt3 = "yy-MM-dd"
val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
- sdf3.setTimeZone(TimeZoneGMT)
+ sdf3.setTimeZone(TimeZone.getTimeZone(UTC))
- withDefaultTimeZone(TimeZoneGMT) {
- for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) {
+ withDefaultTimeZone(TimeZone.getTimeZone(UTC)) {
+ for (tz <- outstandingTimezones) {
val timeZoneId = Option(tz.getID)
sdf1.setTimeZone(tz)
sdf2.setTimeZone(tz)
@@ -924,8 +919,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
NonFoldableLiteral.create(tz, StringType)),
if (expected != null) Timestamp.valueOf(expected) else null)
}
- test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00")
- test("2015-01-24 00:00:00", "PST", "2015-01-24 08:00:00")
+ test("2015-07-24 00:00:00", LA.getId, "2015-07-24 07:00:00")
+ test("2015-01-24 00:00:00", LA.getId, "2015-01-24 08:00:00")
test(null, "UTC", null)
test("2015-07-24 00:00:00", null, null)
test(null, null, null)
@@ -955,8 +950,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
NonFoldableLiteral.create(tz, StringType)),
if (expected != null) Timestamp.valueOf(expected) else null)
}
- test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00")
- test("2015-01-24 00:00:00", "PST", "2015-01-23 16:00:00")
+ test("2015-07-24 00:00:00", LA.getId, "2015-07-23 17:00:00")
+ test("2015-01-24 00:00:00", LA.getId, "2015-01-23 16:00:00")
test(null, "UTC", null)
test("2015-07-24 00:00:00", null, null)
test(null, null, null)
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 3693531..dec2e46 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
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import java.text.{DecimalFormat, DecimalFormatSymbols, SimpleDateFormat}
-import java.util.{Calendar, Locale}
+import java.util.{Calendar, Locale, TimeZone}
import org.scalatest.exceptions.TestFailedException
@@ -27,6 +27,7 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.plans.PlanTestBase
import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{PST, UTC, UTC_OPT}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
@@ -395,13 +396,11 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
InternalRow(UTF8String.fromString("1"), null, UTF8String.fromString("1")))
}
- val gmtId = Option(DateTimeUtils.TimeZoneGMT.getID)
-
test("from_json") {
val jsonData = """{"a": 1}"""
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal(jsonData), gmtId),
+ JsonToStructs(schema, Map.empty, Literal(jsonData), UTC_OPT),
InternalRow(1)
)
}
@@ -410,13 +409,13 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val jsonData = """{"a" 1}"""
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal(jsonData), gmtId),
+ JsonToStructs(schema, Map.empty, Literal(jsonData), UTC_OPT),
InternalRow(null)
)
val exception = intercept[TestFailedException] {
checkEvaluation(
- JsonToStructs(schema, Map("mode" -> FailFastMode.name), Literal(jsonData), gmtId),
+ JsonToStructs(schema, Map("mode" -> FailFastMode.name), Literal(jsonData), UTC_OPT),
InternalRow(null)
)
}.getCause
@@ -429,35 +428,35 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val input = """[{"a": 1}, {"a": 2}]"""
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val output = InternalRow(1) :: InternalRow(2) :: Nil
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json - input=object, schema=array, output=array of single row") {
val input = """{"a": 1}"""
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val output = InternalRow(1) :: Nil
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json - input=empty array, schema=array, output=empty array") {
val input = "[ ]"
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val output = Nil
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json - input=empty object, schema=array, output=array of single row with null") {
val input = "{ }"
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val output = InternalRow(null) :: Nil
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json - input=array of single object, schema=struct, output=single row") {
val input = """[{"a": 1}]"""
val schema = StructType(StructField("a", IntegerType) :: Nil)
val output = InternalRow(null)
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json - input=array, schema=struct, output=single row") {
@@ -466,27 +465,27 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val schema = new StructType().add("a", IntegerType).add(corrupted, StringType)
val output = InternalRow(null, UTF8String.fromString(input))
val options = Map("columnNameOfCorruptRecord" -> corrupted)
- checkEvaluation(JsonToStructs(schema, options, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, options, Literal(input), UTC_OPT), output)
}
test("from_json - input=empty array, schema=struct, output=single row with null") {
val input = """[]"""
val schema = StructType(StructField("a", IntegerType) :: Nil)
val output = InternalRow(null)
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json - input=empty object, schema=struct, output=single row with null") {
val input = """{ }"""
val schema = StructType(StructField("a", IntegerType) :: Nil)
val output = InternalRow(null)
- checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output)
+ checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), UTC_OPT), output)
}
test("from_json null input column") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal.create(null, StringType), gmtId),
+ JsonToStructs(schema, Map.empty, Literal.create(null, StringType), UTC_OPT),
null
)
}
@@ -494,7 +493,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
test("SPARK-20549: from_json bad UTF-8") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal(badJson), gmtId),
+ JsonToStructs(schema, Map.empty, Literal(badJson), UTC_OPT),
InternalRow(null))
}
@@ -502,18 +501,18 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val schema = StructType(StructField("t", TimestampType) :: Nil)
val jsonData1 = """{"t": "2016-01-01T00:00:00.123Z"}"""
- var c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT)
+ var c = Calendar.getInstance(TimeZone.getTimeZone(UTC))
c.set(2016, 0, 1, 0, 0, 0)
c.set(Calendar.MILLISECOND, 123)
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal(jsonData1), gmtId),
+ JsonToStructs(schema, Map.empty, Literal(jsonData1), UTC_OPT),
InternalRow(c.getTimeInMillis * 1000L)
)
// The result doesn't change because the json string includes timezone string ("Z" here),
// which means the string represents the timestamp string in the timezone regardless of
// the timeZoneId parameter.
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal(jsonData1), Option("PST")),
+ JsonToStructs(schema, Map.empty, Literal(jsonData1), Option(PST.getId)),
InternalRow(c.getTimeInMillis * 1000L)
)
@@ -536,7 +535,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
DateTimeUtils.TIMEZONE_OPTION -> tz.getID),
Literal(jsonData2),
- gmtId),
+ UTC_OPT),
InternalRow(c.getTimeInMillis * 1000L)
)
}
@@ -545,7 +544,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
test("SPARK-19543: from_json empty input column") {
val schema = StructType(StructField("a", IntegerType) :: Nil)
checkEvaluation(
- JsonToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId),
+ JsonToStructs(schema, Map.empty, Literal.create(" ", StringType), UTC_OPT),
null
)
}
@@ -554,7 +553,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val schema = StructType(StructField("a", IntegerType) :: Nil)
val struct = Literal.create(create_row(1), schema)
checkEvaluation(
- StructsToJson(Map.empty, struct, gmtId),
+ StructsToJson(Map.empty, struct, UTC_OPT),
"""{"a":1}"""
)
}
@@ -564,7 +563,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val input = new GenericArrayData(InternalRow(1) :: InternalRow(2) :: Nil)
val output = """[{"a":1},{"a":2}]"""
checkEvaluation(
- StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId),
+ StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT),
output)
}
@@ -573,7 +572,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val input = new GenericArrayData(InternalRow(null) :: Nil)
val output = """[{}]"""
checkEvaluation(
- StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId),
+ StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT),
output)
}
@@ -582,7 +581,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val input = new GenericArrayData(Nil)
val output = """[]"""
checkEvaluation(
- StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId),
+ StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT),
output)
}
@@ -590,41 +589,41 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val schema = StructType(StructField("a", IntegerType) :: Nil)
val struct = Literal.create(null, schema)
checkEvaluation(
- StructsToJson(Map.empty, struct, gmtId),
+ StructsToJson(Map.empty, struct, UTC_OPT),
null
)
}
test("to_json with timestamp") {
val schema = StructType(StructField("t", TimestampType) :: Nil)
- val c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT)
+ val c = Calendar.getInstance(TimeZone.getTimeZone(UTC))
c.set(2016, 0, 1, 0, 0, 0)
c.set(Calendar.MILLISECOND, 0)
val struct = Literal.create(create_row(c.getTimeInMillis * 1000L), schema)
checkEvaluation(
- StructsToJson(Map.empty, struct, gmtId),
+ StructsToJson(Map.empty, struct, UTC_OPT),
"""{"t":"2016-01-01T00:00:00.000Z"}"""
)
checkEvaluation(
- StructsToJson(Map.empty, struct, Option("PST")),
+ StructsToJson(Map.empty, struct, Option(PST.getId)),
"""{"t":"2015-12-31T16:00:00.000-08:00"}"""
)
checkEvaluation(
StructsToJson(
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
- DateTimeUtils.TIMEZONE_OPTION -> gmtId.get),
+ DateTimeUtils.TIMEZONE_OPTION -> UTC_OPT.get),
struct,
- gmtId),
+ UTC_OPT),
"""{"t":"2016-01-01T00:00:00"}"""
)
checkEvaluation(
StructsToJson(
Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss",
- DateTimeUtils.TIMEZONE_OPTION -> "PST"),
+ DateTimeUtils.TIMEZONE_OPTION -> PST.getId),
struct,
- gmtId),
+ UTC_OPT),
"""{"t":"2015-12-31T16:00:00"}"""
)
}
@@ -665,7 +664,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
ArrayBasedMapData(Map(UTF8String.fromString("b") -> 2)) :: Nil)
val output = """[{"a":1},{"b":2}]"""
checkEvaluation(
- StructsToJson(Map.empty, Literal(input, inputSchema), gmtId),
+ StructsToJson(Map.empty, Literal(input, inputSchema), UTC_OPT),
output)
}
@@ -674,7 +673,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val input = new GenericArrayData(ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)) :: Nil)
val output = """[{"a":1}]"""
checkEvaluation(
- StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId),
+ StructsToJson(Map.empty, Literal.create(input, inputSchema), UTC_OPT),
output)
}
@@ -704,7 +703,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
.add("b", StringType, nullable = false)
.add("c", StringType, nullable = false)
val output = InternalRow(1L, null, UTF8String.fromString("foo"))
- val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId)
+ val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), UTC_OPT)
checkEvaluation(expr, output)
val schema = expr.dataType
val schemaToCompare = jsonSchema.asNullable
@@ -737,7 +736,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val options = Map("dateFormat" -> dateFormat, "locale" -> langTag)
checkEvaluation(
- JsonToStructs(schema, options, Literal.create(dateStr), gmtId),
+ JsonToStructs(schema, options, Literal.create(dateStr), UTC_OPT),
InternalRow(17836)) // number of days from 1970-01-01
}
}
@@ -748,7 +747,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
schema = StructType.fromDDL("i int, _unparsed boolean"),
options = Map("columnNameOfCorruptRecord" -> "_unparsed"),
child = Literal.create("""{"i":"a"}"""),
- timeZoneId = gmtId),
+ timeZoneId = UTC_OPT),
expectedErrMsg = "The field for corrupt records must be string type and nullable")
}
@@ -770,7 +769,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with
val (expected, input) = decimalInput(langTag)
checkEvaluation(
- JsonToStructs(schema, options, Literal.create(input), gmtId),
+ JsonToStructs(schema, options, Literal.create(input), UTC_OPT),
InternalRow(expected))
}
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 5f1428f..374400e 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
@@ -22,20 +22,31 @@ import java.util.TimeZone
import java.util.concurrent.TimeUnit
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
+import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId
/**
* Helper functions for testing date and time functionality.
*/
object DateTimeTestUtils {
+ val CEST = getZoneId("+02:00")
+ val CET = getZoneId("+01:00")
+ val JST = getZoneId("+09:00")
+ val LA = getZoneId("America/Los_Angeles")
+ val MIT = getZoneId("-09:30")
+ val PST = getZoneId("-08:00")
+ val UTC = getZoneId("+00:00")
+
+ val UTC_OPT = Option("UTC")
+
val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone)
val outstandingTimezonesIds: Seq[String] = Seq(
"UTC",
- "PST",
- "CET",
+ PST.getId,
+ CET.getId,
"Africa/Dakar",
- "America/Los_Angeles",
+ LA.getId,
"Antarctica/Vostok",
"Asia/Hong_Kong",
"Europe/Amsterdam")
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 10b2457..5eca864 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
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util
import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
-import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset}
+import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId}
import java.util.{Locale, TimeZone}
import java.util.concurrent.TimeUnit
@@ -34,9 +34,6 @@ import org.apache.spark.unsafe.types.UTF8String
class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
- val zonePST = getZoneId("PST")
- val zoneGMT = getZoneId("GMT")
-
private def defaultZoneId = ZoneId.systemDefault()
test("nanoseconds truncation") {
@@ -124,7 +121,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
checkFromToJavaDate(new Date(df2.parse("1776-07-04 18:30:00 UTC").getTime))
}
- private def toDate(s: String, zoneId: ZoneId = ZoneOffset.UTC): Option[SQLDate] = {
+ private def toDate(s: String, zoneId: ZoneId = UTC): Option[SQLDate] = {
stringToDate(UTF8String.fromString(s), zoneId)
}
@@ -176,29 +173,28 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
// If the string value includes timezone string, it represents the timestamp string
// in the timezone regardless of the tz parameter.
- var zoneId = getZoneId("GMT-13:53")
+ var zoneId = getZoneId("-13:53")
expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17-13:53", expected)
checkStringToTimestamp("2015-03-18T12:03:17GMT-13:53", expected)
- zoneId = getZoneId("UTC")
- expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId))
+ expected = Option(date(2015, 3, 18, 12, 3, 17, zid = UTC))
checkStringToTimestamp("2015-03-18T12:03:17Z", expected)
checkStringToTimestamp("2015-03-18 12:03:17Z", expected)
checkStringToTimestamp("2015-03-18 12:03:17UTC", expected)
- zoneId = getZoneId("GMT-01:00")
+ zoneId = getZoneId("-01:00")
expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17-1:0", expected)
checkStringToTimestamp("2015-03-18T12:03:17-01:00", expected)
checkStringToTimestamp("2015-03-18T12:03:17GMT-01:00", expected)
- zoneId = getZoneId("GMT+07:30")
+ zoneId = getZoneId("+07:30")
expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17+07:30", expected)
checkStringToTimestamp("2015-03-18T12:03:17 GMT+07:30", expected)
- zoneId = getZoneId("GMT+07:03")
+ zoneId = getZoneId("+07:03")
expected = Option(date(2015, 3, 18, 12, 3, 17, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17+07:03", expected)
checkStringToTimestamp("2015-03-18T12:03:17GMT+07:03", expected)
@@ -210,24 +206,23 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
// If the string value includes timezone string, it represents the timestamp string
// in the timezone regardless of the tz parameter.
- zoneId = getZoneId("UTC")
- expected = Option(date(2015, 3, 18, 12, 3, 17, 456000, zid = zoneId))
+ expected = Option(date(2015, 3, 18, 12, 3, 17, 456000, zid = UTC))
checkStringToTimestamp("2015-03-18T12:03:17.456Z", expected)
checkStringToTimestamp("2015-03-18 12:03:17.456Z", expected)
checkStringToTimestamp("2015-03-18 12:03:17.456 UTC", expected)
- zoneId = getZoneId("GMT-01:00")
+ zoneId = getZoneId("-01:00")
expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17.123-1:0", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123-01:00", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123 GMT-01:00", expected)
- zoneId = getZoneId("GMT+07:30")
+ zoneId = getZoneId("+07:30")
expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123 GMT+07:30", expected)
- zoneId = getZoneId("GMT+07:30")
+ zoneId = getZoneId("+07:30")
expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123GMT+07:30", expected)
@@ -236,7 +231,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
checkStringToTimestamp("2015-03-18T12:03:17.123121+7:30", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123121 GMT+0730", expected)
- zoneId = getZoneId("GMT+07:30")
+ zoneId = getZoneId("+07:30")
expected = Option(date(2015, 3, 18, 12, 3, 17, 123120, zid = zoneId))
checkStringToTimestamp("2015-03-18T12:03:17.12312+7:30", expected)
checkStringToTimestamp("2015-03-18T12:03:17.12312 UT+07:30", expected)
@@ -244,12 +239,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
expected = Option(time(18, 12, 15, zid = zid))
checkStringToTimestamp("18:12:15", expected)
- zoneId = getZoneId("GMT+07:30")
+ zoneId = getZoneId("+07:30")
expected = Option(time(18, 12, 15, 123120, zid = zoneId))
checkStringToTimestamp("T18:12:15.12312+7:30", expected)
checkStringToTimestamp("T18:12:15.12312 UTC+07:30", expected)
- zoneId = getZoneId("GMT+07:30")
+ zoneId = getZoneId("+07:30")
expected = Option(time(18, 12, 15, 123120, zid = zoneId))
checkStringToTimestamp("18:12:15.12312+7:30", expected)
checkStringToTimestamp("18:12:15.12312 GMT+07:30", expected)
@@ -276,8 +271,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
checkStringToTimestamp("1999 08", None)
// Truncating the fractional seconds
- zoneId = getZoneId("GMT+00:00")
- expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, zid = zoneId))
+ expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, zid = UTC))
checkStringToTimestamp("2015-03-18T12:03:17.123456789+0:00", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123456789 UTC+0", expected)
checkStringToTimestamp("2015-03-18T12:03:17.123456789GMT+00:00", expected)
@@ -306,38 +300,38 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
}
test("hours") {
- var input = date(2015, 3, 18, 13, 2, 11, 0, zonePST)
- assert(getHours(input, zonePST) === 13)
- assert(getHours(input, zoneGMT) === 20)
- input = date(2015, 12, 8, 2, 7, 9, 0, zonePST)
- assert(getHours(input, zonePST) === 2)
- assert(getHours(input, zoneGMT) === 10)
- input = date(10, 1, 1, 0, 0, 0, 0, zonePST)
- assert(getHours(input, zonePST) === 0)
+ var input = date(2015, 3, 18, 13, 2, 11, 0, LA)
+ assert(getHours(input, LA) === 13)
+ assert(getHours(input, UTC) === 20)
+ input = date(2015, 12, 8, 2, 7, 9, 0, LA)
+ assert(getHours(input, LA) === 2)
+ assert(getHours(input, UTC) === 10)
+ input = date(10, 1, 1, 0, 0, 0, 0, LA)
+ assert(getHours(input, LA) === 0)
}
test("minutes") {
- var input = date(2015, 3, 18, 13, 2, 11, 0, zonePST)
- assert(getMinutes(input, zonePST) === 2)
- assert(getMinutes(input, zoneGMT) === 2)
+ var input = date(2015, 3, 18, 13, 2, 11, 0, LA)
+ assert(getMinutes(input, LA) === 2)
+ assert(getMinutes(input, UTC) === 2)
assert(getMinutes(input, getZoneId("Australia/North")) === 32)
- input = date(2015, 3, 8, 2, 7, 9, 0, zonePST)
- assert(getMinutes(input, zonePST) === 7)
- assert(getMinutes(input, zoneGMT) === 7)
+ input = date(2015, 3, 8, 2, 7, 9, 0, LA)
+ assert(getMinutes(input, LA) === 7)
+ assert(getMinutes(input, UTC) === 7)
assert(getMinutes(input, getZoneId("Australia/North")) === 37)
- input = date(10, 1, 1, 0, 0, 0, 0, zonePST)
- assert(getMinutes(input, zonePST) === 0)
+ input = date(10, 1, 1, 0, 0, 0, 0, LA)
+ assert(getMinutes(input, LA) === 0)
}
test("seconds") {
- var input = date(2015, 3, 18, 13, 2, 11, 0, zonePST)
- assert(getSeconds(input, zonePST) === 11)
- assert(getSeconds(input, zoneGMT) === 11)
- input = date(2015, 3, 8, 2, 7, 9, 0, zonePST)
- assert(getSeconds(input, zonePST) === 9)
- assert(getSeconds(input, zoneGMT) === 9)
- input = date(10, 1, 1, 0, 0, 0, 0, zonePST)
- assert(getSeconds(input, zonePST) === 0)
+ var input = date(2015, 3, 18, 13, 2, 11, 0, LA)
+ assert(getSeconds(input, LA) === 11)
+ assert(getSeconds(input, UTC) === 11)
+ input = date(2015, 3, 8, 2, 7, 9, 0, LA)
+ assert(getSeconds(input, LA) === 9)
+ assert(getSeconds(input, UTC) === 9)
+ input = date(10, 1, 1, 0, 0, 0, 0, LA)
+ assert(getSeconds(input, LA) === 0)
}
test("hours / minutes / seconds") {
@@ -403,58 +397,58 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
val ts2 = date(2000, 2, 28, 10, 30, 0, 123000)
assert(timestampAddInterval(ts1, 36, 0, 123000, defaultZoneId) === ts2)
- val ts3 = date(1997, 2, 27, 16, 0, 0, 0, zonePST)
- val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, zonePST)
- val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, zoneGMT)
- assert(timestampAddInterval(ts3, 36, 0, 123000, zonePST) === ts4)
- assert(timestampAddInterval(ts3, 36, 0, 123000, zoneGMT) === ts5)
+ val ts3 = date(1997, 2, 27, 16, 0, 0, 0, LA)
+ val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, LA)
+ val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, UTC)
+ assert(timestampAddInterval(ts3, 36, 0, 123000, LA) === ts4)
+ assert(timestampAddInterval(ts3, 36, 0, 123000, UTC) === ts5)
}
test("timestamp add days") {
// 2019-3-9 is the end of Pacific Standard Time
- val ts1 = date(2019, 3, 9, 12, 0, 0, 123000, zonePST)
+ val ts1 = date(2019, 3, 9, 12, 0, 0, 123000, LA)
// 2019-3-10 is the start of Pacific Daylight Time
- val ts2 = date(2019, 3, 10, 12, 0, 0, 123000, zonePST)
- val ts3 = date(2019, 5, 9, 12, 0, 0, 123000, zonePST)
- val ts4 = date(2019, 5, 10, 12, 0, 0, 123000, zonePST)
+ val ts2 = date(2019, 3, 10, 12, 0, 0, 123000, LA)
+ val ts3 = date(2019, 5, 9, 12, 0, 0, 123000, LA)
+ val ts4 = date(2019, 5, 10, 12, 0, 0, 123000, LA)
// 2019-11-2 is the end of Pacific Daylight Time
- val ts5 = date(2019, 11, 2, 12, 0, 0, 123000, zonePST)
+ val ts5 = date(2019, 11, 2, 12, 0, 0, 123000, LA)
// 2019-11-3 is the start of Pacific Standard Time
- val ts6 = date(2019, 11, 3, 12, 0, 0, 123000, zonePST)
+ val ts6 = date(2019, 11, 3, 12, 0, 0, 123000, LA)
// transit from Pacific Standard Time to Pacific Daylight Time
assert(timestampAddInterval(
- ts1, 0, 0, 23 * MICROS_PER_HOUR, zonePST) === ts2)
- assert(timestampAddInterval(ts1, 0, 1, 0, zonePST) === ts2)
+ ts1, 0, 0, 23 * MICROS_PER_HOUR, LA) === ts2)
+ assert(timestampAddInterval(ts1, 0, 1, 0, LA) === ts2)
// just a normal day
assert(timestampAddInterval(
- ts3, 0, 0, 24 * MICROS_PER_HOUR, zonePST) === ts4)
- assert(timestampAddInterval(ts3, 0, 1, 0, zonePST) === ts4)
+ ts3, 0, 0, 24 * MICROS_PER_HOUR, LA) === ts4)
+ assert(timestampAddInterval(ts3, 0, 1, 0, LA) === ts4)
// transit from Pacific Daylight Time to Pacific Standard Time
assert(timestampAddInterval(
- ts5, 0, 0, 25 * MICROS_PER_HOUR, zonePST) === ts6)
- assert(timestampAddInterval(ts5, 0, 1, 0, zonePST) === ts6)
+ ts5, 0, 0, 25 * MICROS_PER_HOUR, LA) === ts6)
+ assert(timestampAddInterval(ts5, 0, 1, 0, LA) === ts6)
}
test("monthsBetween") {
val date1 = date(1997, 2, 28, 10, 30, 0)
var date2 = date(1996, 10, 30)
- assert(monthsBetween(date1, date2, true, ZoneOffset.UTC) === 3.94959677)
- assert(monthsBetween(date1, date2, false, ZoneOffset.UTC) === 3.9495967741935485)
+ assert(monthsBetween(date1, date2, true, UTC) === 3.94959677)
+ assert(monthsBetween(date1, date2, false, UTC) === 3.9495967741935485)
Seq(true, false).foreach { roundOff =>
date2 = date(2000, 2, 28)
- assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36)
+ assert(monthsBetween(date1, date2, roundOff, UTC) === -36)
date2 = date(2000, 2, 29)
- assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36)
+ assert(monthsBetween(date1, date2, roundOff, UTC) === -36)
date2 = date(1996, 3, 31)
- assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === 11)
+ assert(monthsBetween(date1, date2, roundOff, UTC) === 11)
}
- val date3 = date(2000, 2, 28, 16, zid = zonePST)
- val date4 = date(1997, 2, 28, 16, zid = zonePST)
- assert(monthsBetween(date3, date4, true, zonePST) === 36.0)
- assert(monthsBetween(date3, date4, true, ZoneOffset.UTC) === 35.90322581)
- assert(monthsBetween(date3, date4, false, ZoneOffset.UTC) === 35.903225806451616)
+ val date3 = date(2000, 2, 28, 16, zid = LA)
+ val date4 = date(1997, 2, 28, 16, zid = LA)
+ assert(monthsBetween(date3, date4, true, LA) === 36.0)
+ assert(monthsBetween(date3, date4, true, UTC) === 35.90322581)
+ assert(monthsBetween(date3, date4, false, UTC) === 35.903225806451616)
}
test("from UTC timestamp") {
@@ -465,19 +459,19 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
for (tz <- ALL_TIMEZONES) {
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", JST.getId, "2011-12-25 18:00:00.123456")
+ test("2011-12-25 09:00:00.123456", LA.getId, "2011-12-25 01:00:00.123456")
test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456")
}
}
- withDefaultTimeZone(TimeZone.getTimeZone("PST")) {
+ withDefaultTimeZone(TimeZone.getTimeZone(LA.getId)) {
// 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("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")
+ test("2016-11-06 08:59:59.0", LA.getId, "2016-11-06 01:59:59.0")
+ test("2016-11-06 09:00:00.0", LA.getId, "2016-11-06 01:00:00.0")
+ test("2016-11-06 10:00:00.0", LA.getId, "2016-11-06 02:00:00.0")
}
}
@@ -490,13 +484,13 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
for (tz <- ALL_TIMEZONES) {
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 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")
test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456")
}
}
- val tz = "America/Los_Angeles"
+ val tz = LA.getId
withDefaultTimeZone(TimeZone.getTimeZone(tz)) {
// Daylight Saving Time
test("2016-03-13 01:59:59", tz, "2016-03-13 09:59:59.0")
@@ -596,16 +590,16 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
}
test("daysToMillis and millisToDays") {
- val input = toMillis(date(2015, 12, 31, 16, zid = zonePST))
- assert(millisToDays(input, zonePST) === 16800)
- assert(millisToDays(input, ZoneOffset.UTC) === 16801)
- assert(millisToDays(-1 * MILLIS_PER_DAY + 1, ZoneOffset.UTC) == -1)
+ val input = toMillis(date(2015, 12, 31, 16, zid = LA))
+ assert(millisToDays(input, LA) === 16800)
+ assert(millisToDays(input, UTC) === 16801)
+ assert(millisToDays(-1 * MILLIS_PER_DAY + 1, UTC) == -1)
- var expected = toMillis(date(2015, 12, 31, zid = zonePST))
- assert(daysToMillis(16800, zonePST) === expected)
+ var expected = toMillis(date(2015, 12, 31, zid = LA))
+ assert(daysToMillis(16800, LA) === expected)
- expected = toMillis(date(2015, 12, 31, zid = zoneGMT))
- assert(daysToMillis(16800, ZoneOffset.UTC) === expected)
+ expected = toMillis(date(2015, 12, 31, zid = UTC))
+ assert(daysToMillis(16800, UTC) === expected)
// There are some days are skipped entirely in some timezone, skip them here.
val skipped_days = Map[String, Set[Int]](
@@ -615,7 +609,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
"Pacific/Fakaofo" -> Set(15338),
"Pacific/Kiritimati" -> Set(9130, 9131),
"Pacific/Kwajalein" -> Set(8632, 8633, 8634),
- "MIT" -> Set(15338))
+ MIT.getId -> Set(15338))
for (tz <- ALL_TIMEZONES) {
val skipped = skipped_days.getOrElse(tz.getID, Set.empty)
val testingData = Seq(-20000, 20000) ++
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
index 4439a7b..0ffb492 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.util
import org.apache.arrow.vector.types.pojo.ArrowType
import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.LA
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
@@ -65,7 +66,7 @@ class ArrowUtilsSuite extends SparkFunSuite {
roundtripWithTz(DateTimeUtils.defaultTimeZone().getID)
roundtripWithTz("Asia/Tokyo")
roundtripWithTz("UTC")
- roundtripWithTz("America/Los_Angeles")
+ roundtripWithTz(LA.getId)
}
test("array") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala
index 3cf1437..b2c3924 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala
@@ -25,6 +25,7 @@ import org.scalatest.Matchers
import org.apache.spark.{SparkFunSuite, SparkUpgradeException}
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CET, PST, UTC}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros
import org.apache.spark.unsafe.types.UTF8String
@@ -34,8 +35,8 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
val localDate = "2018-12-02T10:11:12.001234"
val expectedMicros = Map(
"UTC" -> 1543745472001234L,
- "PST" -> 1543774272001234L,
- "CET" -> 1543741872001234L,
+ PST.getId -> 1543774272001234L,
+ CET.getId -> 1543741872001234L,
"Africa/Dakar" -> 1543745472001234L,
"America/Los_Angeles" -> 1543774272001234L,
"Antarctica/Vostok" -> 1543723872001234L,
@@ -55,8 +56,8 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
val microsSinceEpoch = 1543745472001234L
val expectedTimestamp = Map(
"UTC" -> "2018-12-02T10:11:12.001234",
- "PST" -> "2018-12-02T02:11:12.001234",
- "CET" -> "2018-12-02T11:11:12.001234",
+ PST.getId -> "2018-12-02T02:11:12.001234",
+ CET.getId -> "2018-12-02T11:11:12.001234",
"Africa/Dakar" -> "2018-12-02T10:11:12.001234",
"America/Los_Angeles" -> "2018-12-02T02:11:12.001234",
"Antarctica/Vostok" -> "2018-12-02T16:11:12.001234",
@@ -251,7 +252,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
}
// "yyyy" with "G" can't parse negative year or year 0000.
- val formatter2 = TimestampFormatter("G yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC, true)
+ val formatter2 = TimestampFormatter("G yyyy-MM-dd HH:mm:ss", UTC, true)
assertParsingError(formatter2.parse("BC -1234-02-22 02:22:22"))
assertParsingError(formatter2.parse("AC 0000-02-22 02:22:22"))
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 c41eb98..875a671 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
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
@@ -1028,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(TimeZone.getTimeZone(UTC)) {
checkAnswer(
spark.sql("select sequence(" +
" cast('2018-01-01' as date)" +
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
index 51c6a83..51d861e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
@@ -258,7 +258,7 @@ class DataFramePivotSuite extends QueryTest with SharedSparkSession {
val ts = "2012-12-31 16:00:10.011"
val tsWithZone = "2013-01-01 00:00:10.011"
- withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") {
+ withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
val df = Seq(java.sql.Timestamp.valueOf(ts)).toDF("a").groupBy("a").pivot("a").count()
val expected = StructType(
StructField("a", TimestampType) ::
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index e74d553..72aa7bf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -1167,7 +1167,7 @@ class DataFrameSuite extends QueryTest
|""".stripMargin
assert(df.showString(1, truncate = 0) === expectedAnswer)
- withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") {
+ withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
val expectedAnswer = """+----------+-------------------+
||d |ts |
@@ -1188,7 +1188,7 @@ class DataFrameSuite extends QueryTest
" ts | 2016-12-01 00:00:00 \n"
assert(df.showString(1, truncate = 0, vertical = true) === expectedAnswer)
- withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") {
+ withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
val expectedAnswer = "-RECORD 0------------------\n" +
" d | 2016-12-01 \n" +
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
index da7cc1c..6ed7483 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
@@ -24,6 +24,7 @@ import java.util.{Locale, TimeZone}
import java.util.concurrent.TimeUnit
import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{CEST, LA}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
@@ -761,12 +762,12 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
(Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00")
).toDF("a", "b")
checkAnswer(
- df.select(from_utc_timestamp(col("a"), "PST")),
+ df.select(from_utc_timestamp(col("a"), LA.getId)),
Seq(
Row(Timestamp.valueOf("2015-07-23 17:00:00")),
Row(Timestamp.valueOf("2015-07-24 17:00:00"))))
checkAnswer(
- df.select(from_utc_timestamp(col("b"), "PST")),
+ df.select(from_utc_timestamp(col("b"), LA.getId)),
Seq(
Row(Timestamp.valueOf("2015-07-23 17:00:00")),
Row(Timestamp.valueOf("2015-07-24 17:00:00"))))
@@ -774,8 +775,8 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
test("from_utc_timestamp with column zone") {
val df = Seq(
- (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "CET"),
- (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "PST")
+ (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", CEST.getId),
+ (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", LA.getId)
).toDF("a", "b", "c")
checkAnswer(
df.select(from_utc_timestamp(col("a"), col("c"))),
@@ -804,12 +805,12 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
(Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00")
).toDF("a", "b")
checkAnswer(
- df.select(to_utc_timestamp(col("a"), "PST")),
+ df.select(to_utc_timestamp(col("a"), LA.getId)),
Seq(
Row(Timestamp.valueOf("2015-07-24 07:00:00")),
Row(Timestamp.valueOf("2015-07-25 07:00:00"))))
checkAnswer(
- df.select(to_utc_timestamp(col("b"), "PST")),
+ df.select(to_utc_timestamp(col("b"), LA.getId)),
Seq(
Row(Timestamp.valueOf("2015-07-24 07:00:00")),
Row(Timestamp.valueOf("2015-07-25 07:00:00"))))
@@ -817,8 +818,8 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
test("to_utc_timestamp with column zone") {
val df = Seq(
- (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "PST"),
- (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "CET")
+ (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", LA.getId),
+ (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", CEST.getId)
).toDF("a", "b", "c")
checkAnswer(
df.select(to_utc_timestamp(col("a"), col("c"))),
@@ -849,10 +850,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
}
test("SPARK-30752: convert time zones on a daylight saving day") {
- val systemTz = "PST"
+ val systemTz = LA.getId
val sessionTz = "UTC"
val fromTz = "Asia/Hong_Kong"
- val fromTs = "2019-11-03T12:00:00" // daylight saving date in PST
+ val fromTs = "2019-11-03T12:00:00" // daylight saving date in America/Los_Angeles
val utsTs = "2019-11-03T04:00:00"
val defaultTz = TimeZone.getDefault
try {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index 30ae9dc..f9a510d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -1080,7 +1080,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa
.format("csv")
.option("header", "true")
.option("timestampFormat", "yyyy/MM/dd HH:mm")
- .option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ .option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.save(timestampsWithFormatPath)
// This will load back the timestamps as string.
@@ -1102,7 +1102,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa
.option("header", "true")
.option("inferSchema", "true")
.option("timestampFormat", "yyyy/MM/dd HH:mm")
- .option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ .option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.load(timestampsWithFormatPath)
checkAnswer(readBack, timestampsWithFormat)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
index fb3328c..5466800 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
@@ -21,7 +21,7 @@ import java.io._
import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException}
import java.nio.file.Files
import java.sql.{Date, Timestamp}
-import java.time.{LocalDate, LocalDateTime, ZoneId}
+import java.time.LocalDate
import java.util.Locale
import com.fasterxml.jackson.core.JsonFactory
@@ -1329,7 +1329,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson
test("SPARK-6245 JsonInferSchema.infer on empty RDD") {
// This is really a test that it doesn't throw an exception
- val options = new JSONOptions(Map.empty[String, String], "GMT")
+ val options = new JSONOptions(Map.empty[String, String], "UTC")
val emptySchema = new JsonInferSchema(options).infer(
empty.rdd,
CreateJacksonParser.string)
@@ -1356,7 +1356,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson
}
test("SPARK-8093 Erase empty structs") {
- val options = new JSONOptions(Map.empty[String, String], "GMT")
+ val options = new JSONOptions(Map.empty[String, String], "UTC")
val emptySchema = new JsonInferSchema(options).infer(
emptyRecords.rdd,
CreateJacksonParser.string)
@@ -1730,7 +1730,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson
timestampsWithFormat.write
.format("json")
.option("timestampFormat", "yyyy/MM/dd HH:mm")
- .option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ .option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.save(timestampsWithFormatPath)
// This will load back the timestamps as string.
@@ -1748,7 +1748,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson
val readBack = spark.read
.schema(customSchema)
.option("timestampFormat", "yyyy/MM/dd HH:mm")
- .option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ .option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.json(timestampsWithFormatPath)
checkAnswer(readBack, timestampsWithFormat)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
index f4499fe..d3301ce 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
@@ -88,7 +88,7 @@ abstract class ParquetPartitionDiscoverySuite
check("1990-02-24 12:00:30",
Literal.create(Timestamp.valueOf("1990-02-24 12:00:30"), TimestampType))
- val c = Calendar.getInstance(TimeZone.getTimeZone("GMT"))
+ val c = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
c.set(1990, 1, 24, 12, 0, 30)
c.set(Calendar.MILLISECOND, 0)
check("1990-02-24 12:00:30",
@@ -692,10 +692,10 @@ abstract class ParquetPartitionDiscoverySuite
}
withTempPath { dir =>
- df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ df.write.option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString)
val fields = schema.map(f => Column(f.name).cast(f.dataType))
- checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.load(dir.toString).select(fields: _*), row)
}
}
@@ -734,10 +734,10 @@ abstract class ParquetPartitionDiscoverySuite
}
withTempPath { dir =>
- df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ df.write.option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString)
val fields = schema.map(f => Column(f.name))
- checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.load(dir.toString).select(fields: _*), row)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
index 10ea948..cde2aa7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path
import org.apache.log4j.Level
import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.MIT
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext}
import org.apache.spark.util.Utils
@@ -350,8 +351,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
}
test("spark.sql.session.timeZone should only accept valid zone id") {
- spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "MIT")
- assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === "MIT")
+ spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, MIT.getId)
+ assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === MIT.getId)
spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Chicago")
assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) ===
"America/Chicago")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
index ab1d1f8..1c4e2a9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
@@ -139,15 +139,15 @@ class PartitionedWriteSuite extends QueryTest with SharedSparkSession {
checkPartitionValues(files.head, "2016-12-01 00:00:00")
}
withTempPath { f =>
- df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT")
+ df.write.option(DateTimeUtils.TIMEZONE_OPTION, "UTC")
.partitionBy("ts").parquet(f.getAbsolutePath)
val files = TestUtils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet"))
assert(files.length == 1)
- // use timeZone option "GMT" to format partition value.
+ // use timeZone option utcTz.getId to format partition value.
checkPartitionValues(files.head, "2016-12-01 08:00:00")
}
withTempPath { f =>
- withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") {
+ withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") {
df.write.partitionBy("ts").parquet(f.getAbsolutePath)
val files = TestUtils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet"))
assert(files.length == 1)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
index 92ec2a0..f29a6c7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala
@@ -29,6 +29,7 @@ import org.scalatest.{BeforeAndAfter, Matchers}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, Dataset}
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark
+import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.sources.MemorySink
import org.apache.spark.sql.functions.{count, window}
@@ -773,7 +774,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche
}
private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
- timestampFormat.setTimeZone(ju.TimeZone.getTimeZone("UTC"))
+ timestampFormat.setTimeZone(ju.TimeZone.getTimeZone(UTC))
private def formatTimestamp(sec: Long): String = {
timestampFormat.format(new ju.Date(sec * 1000))
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org