Skip to content

Commit aa0d136

Browse files
MaxGekkcloud-fan
authored andcommitted
[SPARK-30760][SQL] Port millisToDays and daysToMillis on Java 8 time API
### What changes were proposed in this pull request? In the PR, I propose to rewrite the `millisToDays` and `daysToMillis` of `DateTimeUtils` using Java 8 time API. I removed `getOffsetFromLocalMillis` from `DateTimeUtils` because it is a private methods, and is not used anymore in Spark SQL. ### Why are the changes needed? New implementation is based on Proleptic Gregorian calendar which has been already used by other date-time functions. This changes make `millisToDays` and `daysToMillis` consistent to rest Spark SQL API related to date & time operations. ### Does this PR introduce any user-facing change? Yes, this might effect behavior for old dates before 1582 year. ### How was this patch tested? By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, DateExpressionsSuite`, `SQLQuerySuite` and `HiveResultSuite`. Closes #27494 from MaxGekk/millis-2-days-java8-api. Authored-by: Maxim Gekk <max.gekk@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent 5919bd3 commit aa0d136

File tree

8 files changed

+62
-78
lines changed

8 files changed

+62
-78
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -135,7 +135,7 @@ case class CurrentBatchTimestamp(
135135
def toLiteral: Literal = dataType match {
136136
case _: TimestampType =>
137137
Literal(DateTimeUtils.fromJavaTimestamp(new Timestamp(timestampMs)), TimestampType)
138-
case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, timeZone), DateType)
138+
case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, zoneId), DateType)
139139
}
140140
}
141141

@@ -1332,14 +1332,14 @@ case class MonthsBetween(
13321332

13331333
override def nullSafeEval(t1: Any, t2: Any, roundOff: Any): Any = {
13341334
DateTimeUtils.monthsBetween(
1335-
t1.asInstanceOf[Long], t2.asInstanceOf[Long], roundOff.asInstanceOf[Boolean], timeZone)
1335+
t1.asInstanceOf[Long], t2.asInstanceOf[Long], roundOff.asInstanceOf[Boolean], zoneId)
13361336
}
13371337

13381338
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
1339-
val tz = ctx.addReferenceObj("timeZone", timeZone)
1339+
val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
13401340
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
13411341
defineCodeGen(ctx, ev, (d1, d2, roundOff) => {
1342-
s"""$dtu.monthsBetween($d1, $d2, $roundOff, $tz)"""
1342+
s"""$dtu.monthsBetween($d1, $d2, $roundOff, $zid)"""
13431343
})
13441344
}
13451345

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala

Lines changed: 15 additions & 43 deletions
Original file line numberDiff line numberDiff line change
@@ -67,24 +67,22 @@ object DateTimeUtils {
6767

6868
// we should use the exact day as Int, for example, (year, month, day) -> day
6969
def millisToDays(millisUtc: Long): SQLDate = {
70-
millisToDays(millisUtc, defaultTimeZone())
70+
millisToDays(millisUtc, defaultTimeZone().toZoneId)
7171
}
7272

73-
def millisToDays(millisUtc: Long, timeZone: TimeZone): SQLDate = {
74-
// SPARK-6785: use Math.floorDiv so negative number of days (dates before 1970)
75-
// will correctly work as input for function toJavaDate(Int)
76-
val millisLocal = millisUtc + timeZone.getOffset(millisUtc)
77-
Math.floorDiv(millisLocal, MILLIS_PER_DAY).toInt
73+
def millisToDays(millisUtc: Long, zoneId: ZoneId): SQLDate = {
74+
val instant = microsToInstant(Math.multiplyExact(millisUtc, MICROS_PER_MILLIS))
75+
localDateToDays(LocalDateTime.ofInstant(instant, zoneId).toLocalDate)
7876
}
7977

8078
// reverse of millisToDays
8179
def daysToMillis(days: SQLDate): Long = {
82-
daysToMillis(days, defaultTimeZone())
80+
daysToMillis(days, defaultTimeZone().toZoneId)
8381
}
8482

85-
def daysToMillis(days: SQLDate, timeZone: TimeZone): Long = {
86-
val millisLocal = days.toLong * MILLIS_PER_DAY
87-
millisLocal - getOffsetFromLocalMillis(millisLocal, timeZone)
83+
def daysToMillis(days: SQLDate, zoneId: ZoneId): Long = {
84+
val instant = daysToLocalDate(days).atStartOfDay(zoneId).toInstant
85+
instantToMicros(instant) / MICROS_PER_MILLIS
8886
}
8987

9088
// Converts Timestamp to string according to Hive TimestampWritable convention.
@@ -589,11 +587,11 @@ object DateTimeUtils {
589587
time1: SQLTimestamp,
590588
time2: SQLTimestamp,
591589
roundOff: Boolean,
592-
timeZone: TimeZone): Double = {
590+
zoneId: ZoneId): Double = {
593591
val millis1 = MICROSECONDS.toMillis(time1)
594592
val millis2 = MICROSECONDS.toMillis(time2)
595-
val date1 = millisToDays(millis1, timeZone)
596-
val date2 = millisToDays(millis2, timeZone)
593+
val date1 = millisToDays(millis1, zoneId)
594+
val date2 = millisToDays(millis2, zoneId)
597595
val (year1, monthInYear1, dayInMonth1, daysToMonthEnd1) = splitDate(date1)
598596
val (year2, monthInYear2, dayInMonth2, daysToMonthEnd2) = splitDate(date2)
599597

@@ -607,8 +605,8 @@ object DateTimeUtils {
607605
}
608606
// using milliseconds can cause precision loss with more than 8 digits
609607
// we follow Hive's implementation which uses seconds
610-
val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, timeZone))
611-
val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, timeZone))
608+
val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, zoneId))
609+
val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, zoneId))
612610
val secondsDiff = (dayInMonth1 - dayInMonth2) * SECONDS_PER_DAY + secondsInDay1 - secondsInDay2
613611
val secondsInMonth = DAYS.toSeconds(31)
614612
val diff = monthDiff + secondsDiff / secondsInMonth.toDouble
@@ -737,8 +735,8 @@ object DateTimeUtils {
737735
millis += offset
738736
millis - millis % MILLIS_PER_DAY - offset
739737
case _ => // Try to truncate date levels
740-
val dDays = millisToDays(millis, timeZone)
741-
daysToMillis(truncDate(dDays, level), timeZone)
738+
val dDays = millisToDays(millis, timeZone.toZoneId)
739+
daysToMillis(truncDate(dDays, level), timeZone.toZoneId)
742740
}
743741
truncated * MICROS_PER_MILLIS
744742
}
@@ -770,32 +768,6 @@ object DateTimeUtils {
770768
}
771769
}
772770

773-
/**
774-
* Lookup the offset for given millis seconds since 1970-01-01 00:00:00 in given timezone.
775-
* TODO: Improve handling of normalization differences.
776-
* TODO: Replace with JSR-310 or similar system - see SPARK-16788
777-
*/
778-
private[sql] def getOffsetFromLocalMillis(millisLocal: Long, tz: TimeZone): Long = {
779-
var guess = tz.getRawOffset
780-
// the actual offset should be calculated based on milliseconds in UTC
781-
val offset = tz.getOffset(millisLocal - guess)
782-
if (offset != guess) {
783-
guess = tz.getOffset(millisLocal - offset)
784-
if (guess != offset) {
785-
// fallback to do the reverse lookup using java.time.LocalDateTime
786-
// this should only happen near the start or end of DST
787-
val localDate = LocalDate.ofEpochDay(MILLISECONDS.toDays(millisLocal))
788-
val localTime = LocalTime.ofNanoOfDay(MILLISECONDS.toNanos(
789-
Math.floorMod(millisLocal, MILLIS_PER_DAY)))
790-
val localDateTime = LocalDateTime.of(localDate, localTime)
791-
val millisEpoch = localDateTime.atZone(tz.toZoneId).toInstant.toEpochMilli
792-
793-
guess = (millisLocal - millisEpoch).toInt
794-
}
795-
}
796-
guess
797-
}
798-
799771
/**
800772
* Convert the timestamp `ts` from one timezone to another.
801773
*

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.csv
1919

2020
import java.math.BigDecimal
2121
import java.text.{DecimalFormat, DecimalFormatSymbols}
22+
import java.time.ZoneOffset
2223
import java.util.{Locale, TimeZone}
2324

2425
import org.apache.commons.lang3.time.FastDateFormat
@@ -137,7 +138,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper {
137138
val expectedDate = format.parse(customDate).getTime
138139
val castedDate = parser.makeConverter("_1", DateType, nullable = true)
139140
.apply(customDate)
140-
assert(castedDate == DateTimeUtils.millisToDays(expectedDate, TimeZone.getTimeZone("GMT")))
141+
assert(castedDate == DateTimeUtils.millisToDays(expectedDate, ZoneOffset.UTC))
141142

142143
val timestamp = "2015-01-01 00:00:00"
143144
timestampsOptions = new CSVOptions(Map(

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -56,9 +56,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
5656
val ts = new Timestamp(toMillis(time))
5757

5858
test("datetime function current_date") {
59-
val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT)
59+
val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC)
6060
val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int]
61-
val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT)
61+
val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC)
6262
assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1)
6363

6464
val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int]
@@ -499,7 +499,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
499499
// Valid range of DateType is [0001-01-01, 9999-12-31]
500500
val maxMonthInterval = 10000 * 12
501501
checkEvaluation(
502-
AddMonths(Literal(Date.valueOf("0001-01-01")), Literal(maxMonthInterval)), 2933261)
502+
AddMonths(Literal(LocalDate.parse("0001-01-01")), Literal(maxMonthInterval)),
503+
LocalDate.of(10001, 1, 1).toEpochDay.toInt)
503504
checkEvaluation(
504505
AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529)
505506
// Test evaluation results between Interpreted mode and Codegen mode
@@ -788,15 +789,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
788789
checkEvaluation(
789790
UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId),
790791
MILLISECONDS.toSeconds(
791-
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz)))
792+
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)))
792793
checkEvaluation(
793794
UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))),
794795
Literal(fmt2), timeZoneId),
795796
-1000L)
796797
checkEvaluation(UnixTimestamp(
797798
Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId),
798799
MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(
799-
DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz)))
800+
DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId)))
800801
val t1 = UnixTimestamp(
801802
CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long]
802803
val t2 = UnixTimestamp(
@@ -814,7 +815,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
814815
checkEvaluation(
815816
UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId),
816817
MILLISECONDS.toSeconds(
817-
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz)))
818+
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)))
818819
checkEvaluation(
819820
UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null)
820821
}
@@ -852,7 +853,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
852853
checkEvaluation(
853854
ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId),
854855
MILLISECONDS.toSeconds(
855-
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz)))
856+
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)))
856857
checkEvaluation(
857858
ToUnixTimestamp(
858859
Literal(sdf2.format(new Timestamp(-1000000))),
@@ -861,7 +862,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
861862
checkEvaluation(ToUnixTimestamp(
862863
Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId),
863864
MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(
864-
DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz)))
865+
DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId)))
865866
val t1 = ToUnixTimestamp(
866867
CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long]
867868
val t2 = ToUnixTimestamp(
@@ -876,7 +877,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
876877
checkEvaluation(ToUnixTimestamp(
877878
Literal(date1), Literal.create(null, StringType), timeZoneId),
878879
MILLISECONDS.toSeconds(
879-
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz)))
880+
DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)))
880881
checkEvaluation(
881882
ToUnixTimestamp(
882883
Literal("2015-07-24"),

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala

Lines changed: 19 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -86,9 +86,13 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
8686
}
8787

8888
test("SPARK-6785: java date conversion before and after epoch") {
89+
def format(d: Date): String = {
90+
TimestampFormatter("uuuu-MM-dd", defaultTimeZone().toZoneId)
91+
.format(d.getTime * MICROS_PER_MILLIS)
92+
}
8993
def checkFromToJavaDate(d1: Date): Unit = {
9094
val d2 = toJavaDate(fromJavaDate(d1))
91-
assert(d2.toString === d1.toString)
95+
assert(format(d2) === format(d1))
9296
}
9397

9498
val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
@@ -413,22 +417,22 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
413417
test("monthsBetween") {
414418
val date1 = date(1997, 2, 28, 10, 30, 0)
415419
var date2 = date(1996, 10, 30)
416-
assert(monthsBetween(date1, date2, true, TimeZoneUTC) === 3.94959677)
417-
assert(monthsBetween(date1, date2, false, TimeZoneUTC) === 3.9495967741935485)
420+
assert(monthsBetween(date1, date2, true, ZoneOffset.UTC) === 3.94959677)
421+
assert(monthsBetween(date1, date2, false, ZoneOffset.UTC) === 3.9495967741935485)
418422
Seq(true, false).foreach { roundOff =>
419423
date2 = date(2000, 2, 28)
420-
assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36)
424+
assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36)
421425
date2 = date(2000, 2, 29)
422-
assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36)
426+
assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36)
423427
date2 = date(1996, 3, 31)
424-
assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === 11)
428+
assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === 11)
425429
}
426430

427431
val date3 = date(2000, 2, 28, 16, tz = TimeZonePST)
428432
val date4 = date(1997, 2, 28, 16, tz = TimeZonePST)
429-
assert(monthsBetween(date3, date4, true, TimeZonePST) === 36.0)
430-
assert(monthsBetween(date3, date4, true, TimeZoneGMT) === 35.90322581)
431-
assert(monthsBetween(date3, date4, false, TimeZoneGMT) === 35.903225806451616)
433+
assert(monthsBetween(date3, date4, true, TimeZonePST.toZoneId) === 36.0)
434+
assert(monthsBetween(date3, date4, true, ZoneOffset.UTC) === 35.90322581)
435+
assert(monthsBetween(date3, date4, false, ZoneOffset.UTC) === 35.903225806451616)
432436
}
433437

434438
test("from UTC timestamp") {
@@ -571,15 +575,15 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
571575

572576
test("daysToMillis and millisToDays") {
573577
val input = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, 16, tz = TimeZonePST))
574-
assert(millisToDays(input, TimeZonePST) === 16800)
575-
assert(millisToDays(input, TimeZoneGMT) === 16801)
576-
assert(millisToDays(-1 * MILLIS_PER_DAY + 1, TimeZoneGMT) == -1)
578+
assert(millisToDays(input, TimeZonePST.toZoneId) === 16800)
579+
assert(millisToDays(input, ZoneOffset.UTC) === 16801)
580+
assert(millisToDays(-1 * MILLIS_PER_DAY + 1, ZoneOffset.UTC) == -1)
577581

578582
var expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZonePST))
579-
assert(daysToMillis(16800, TimeZonePST) === expected)
583+
assert(daysToMillis(16800, TimeZonePST.toZoneId) === expected)
580584

581585
expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZoneGMT))
582-
assert(daysToMillis(16800, TimeZoneGMT) === expected)
586+
assert(daysToMillis(16800, ZoneOffset.UTC) === expected)
583587

584588
// There are some days are skipped entirely in some timezone, skip them here.
585589
val skipped_days = Map[String, Set[Int]](
@@ -594,7 +598,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
594598
val skipped = skipped_days.getOrElse(tz.getID, Set.empty)
595599
(-20000 to 20000).foreach { d =>
596600
if (!skipped.contains(d)) {
597-
assert(millisToDays(daysToMillis(d, tz), tz) === d,
601+
assert(millisToDays(daysToMillis(d, tz.toZoneId), tz.toZoneId) === d,
598602
s"Round trip of ${d} did not work in tz ${tz}")
599603
}
600604
}

sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution
1919

2020
import java.nio.charset.StandardCharsets
2121
import java.sql.{Date, Timestamp}
22+
import java.time.{Instant, LocalDate}
2223

2324
import org.apache.spark.sql.Row
2425
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
@@ -67,8 +68,12 @@ object HiveResult {
6768
case (null, _) => if (nested) "null" else "NULL"
6869
case (b, BooleanType) => b.toString
6970
case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d))
71+
case (ld: LocalDate, DateType) =>
72+
dateFormatter.format(DateTimeUtils.localDateToDays(ld))
7073
case (t: Timestamp, TimestampType) =>
7174
timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t))
75+
case (i: Instant, TimestampType) =>
76+
timestampFormatter.format(DateTimeUtils.instantToMicros(i))
7277
case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8)
7378
case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString
7479
case (n, _: NumericType) => n.toString

sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -800,15 +800,15 @@ SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000')
800800
-- !query schema
801801
struct<date_trunc(MILLENNIUM, TIMESTAMP '1970-03-20 04:30:00'):timestamp>
802802
-- !query output
803-
1001-01-01 00:07:02
803+
1001-01-01 00:00:00
804804

805805

806806
-- !query
807807
SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20')
808808
-- !query schema
809809
struct<date_trunc(MILLENNIUM, CAST(DATE '1970-03-20' AS TIMESTAMP)):timestamp>
810810
-- !query output
811-
1001-01-01 00:07:02
811+
1001-01-01 00:00:00
812812

813813

814814
-- !query
@@ -840,15 +840,15 @@ SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04')
840840
-- !query schema
841841
struct<date_trunc(CENTURY, CAST(DATE '0002-02-04' AS TIMESTAMP)):timestamp>
842842
-- !query output
843-
0001-01-01 00:07:02
843+
0001-01-01 00:00:00
844844

845845

846846
-- !query
847847
SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G'))
848848
-- !query schema
849849
struct<date_trunc(CENTURY, CAST(to_date('0055-08-10 BC', 'yyyy-MM-dd G') AS TIMESTAMP)):timestamp>
850850
-- !query output
851-
-0099-01-01 00:07:02
851+
-0099-01-01 00:00:00
852852

853853

854854
-- !query
@@ -864,15 +864,15 @@ SELECT DATE_TRUNC('DECADE', DATE '0004-12-25')
864864
-- !query schema
865865
struct<date_trunc(DECADE, CAST(DATE '0004-12-25' AS TIMESTAMP)):timestamp>
866866
-- !query output
867-
0000-01-01 00:07:02
867+
0000-01-01 00:00:00
868868

869869

870870
-- !query
871871
SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G'))
872872
-- !query schema
873873
struct<date_trunc(DECADE, CAST(to_date('0002-12-31 BC', 'yyyy-MM-dd G') AS TIMESTAMP)):timestamp>
874874
-- !query output
875-
-0010-01-01 00:07:02
875+
-0010-01-01 00:00:00
876876

877877

878878
-- !query

sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -337,6 +337,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession {
337337
localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true)
338338
case _ =>
339339
}
340+
localSparkSession.conf.set(SQLConf.DATETIME_JAVA8API_ENABLED.key, true)
340341

341342
if (configSet.nonEmpty) {
342343
// Execute the list of set operation in order to add the desired configs

0 commit comments

Comments
 (0)