From 1211f328565c3a1206f537e59e27c621b680e6f2 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 7 Feb 2020 23:22:37 +0300 Subject: [PATCH 1/7] Port daysToMillis() --- .../sql/catalyst/util/DateTimeUtils.scala | 30 ++----------------- 1 file changed, 2 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 8eb560944d4cb..43b02effb9f38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -83,8 +83,8 @@ object DateTimeUtils { } def daysToMillis(days: SQLDate, timeZone: TimeZone): Long = { - val millisLocal = days.toLong * MILLIS_PER_DAY - millisLocal - getOffsetFromLocalMillis(millisLocal, timeZone) + val instant = daysToLocalDate(days).atStartOfDay(timeZone.toZoneId).toInstant + instantToMicros(instant) / MICROS_PER_MILLIS } // Converts Timestamp to string according to Hive TimestampWritable convention. @@ -768,32 +768,6 @@ object DateTimeUtils { } } - /** - * Lookup the offset for given millis seconds since 1970-01-01 00:00:00 in given timezone. - * TODO: Improve handling of normalization differences. - * TODO: Replace with JSR-310 or similar system - see SPARK-16788 - */ - private[sql] def getOffsetFromLocalMillis(millisLocal: Long, tz: TimeZone): Long = { - var guess = tz.getRawOffset - // the actual offset should be calculated based on milliseconds in UTC - val offset = tz.getOffset(millisLocal - guess) - if (offset != guess) { - guess = tz.getOffset(millisLocal - offset) - if (guess != offset) { - // fallback to do the reverse lookup using java.time.LocalDateTime - // this should only happen near the start or end of DST - val localDate = LocalDate.ofEpochDay(MILLISECONDS.toDays(millisLocal)) - val localTime = LocalTime.ofNanoOfDay(MILLISECONDS.toNanos( - Math.floorMod(millisLocal, MILLIS_PER_DAY))) - val localDateTime = LocalDateTime.of(localDate, localTime) - val millisEpoch = localDateTime.atZone(tz.toZoneId).toInstant.toEpochMilli - - guess = (millisLocal - millisEpoch).toInt - } - } - guess - } - /** * Convert the timestamp `ts` from one timezone to another. * From 9a2b7becc37c647f63d85c519d62c729b5498939 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 8 Feb 2020 00:05:45 +0300 Subject: [PATCH 2/7] Port millisToDays() --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 6 ++---- .../sql/catalyst/expressions/DateExpressionsSuite.scala | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 43b02effb9f38..f0b459667ee42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -71,10 +71,8 @@ object DateTimeUtils { } def millisToDays(millisUtc: Long, timeZone: TimeZone): SQLDate = { - // SPARK-6785: use Math.floorDiv so negative number of days (dates before 1970) - // will correctly work as input for function toJavaDate(Int) - val millisLocal = millisUtc + timeZone.getOffset(millisUtc) - Math.floorDiv(millisLocal, MILLIS_PER_DAY).toInt + val instant = microsToInstant(Math.multiplyExact(millisUtc, MICROS_PER_MILLIS)) + localDateToDays(LocalDateTime.ofInstant(instant, timeZone.toZoneId).toLocalDate) } // reverse of millisToDays 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 274d0beebd300..a46ede5cfcd39 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 @@ -494,7 +494,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // Valid range of DateType is [0001-01-01, 9999-12-31] val maxMonthInterval = 10000 * 12 checkEvaluation( - AddMonths(Literal(Date.valueOf("0001-01-01")), Literal(maxMonthInterval)), 2933261) + AddMonths(Literal(LocalDate.parse("0001-01-01")), Literal(maxMonthInterval)), 2933263) checkEvaluation( AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529) // Test evaluation results between Interpreted mode and Codegen mode From cc4c07dc6dfa0e729ef3123069616bce04e7e9ac Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 8 Feb 2020 00:53:52 +0300 Subject: [PATCH 3/7] Pass ZoneId to millisToDays() and daysToMillis() --- .../expressions/datetimeExpressions.scala | 8 +++--- .../sql/catalyst/util/DateTimeUtils.scala | 26 ++++++++--------- .../catalyst/csv/UnivocityParserSuite.scala | 3 +- .../expressions/DateExpressionsSuite.scala | 20 +++++++------ .../catalyst/util/DateTimeUtilsSuite.scala | 28 +++++++++---------- 5 files changed, 45 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index aa2bd5a1273e0..ddf9ceed94dac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -134,7 +134,7 @@ case class CurrentBatchTimestamp( def toLiteral: Literal = dataType match { case _: TimestampType => Literal(DateTimeUtils.fromJavaTimestamp(new Timestamp(timestampMs)), TimestampType) - case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, timeZone), DateType) + case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, zoneId), DateType) } } @@ -1320,14 +1320,14 @@ case class MonthsBetween( override def nullSafeEval(t1: Any, t2: Any, roundOff: Any): Any = { DateTimeUtils.monthsBetween( - t1.asInstanceOf[Long], t2.asInstanceOf[Long], roundOff.asInstanceOf[Boolean], timeZone) + t1.asInstanceOf[Long], t2.asInstanceOf[Long], roundOff.asInstanceOf[Boolean], zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (d1, d2, roundOff) => { - s"""$dtu.monthsBetween($d1, $d2, $roundOff, $tz)""" + s"""$dtu.monthsBetween($d1, $d2, $roundOff, $zid)""" }) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index f0b459667ee42..01d36f19fc06f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -67,21 +67,21 @@ object DateTimeUtils { // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisUtc: Long): SQLDate = { - millisToDays(millisUtc, defaultTimeZone()) + millisToDays(millisUtc, defaultTimeZone().toZoneId) } - def millisToDays(millisUtc: Long, timeZone: TimeZone): SQLDate = { + def millisToDays(millisUtc: Long, zoneId: ZoneId): SQLDate = { val instant = microsToInstant(Math.multiplyExact(millisUtc, MICROS_PER_MILLIS)) - localDateToDays(LocalDateTime.ofInstant(instant, timeZone.toZoneId).toLocalDate) + localDateToDays(LocalDateTime.ofInstant(instant, zoneId).toLocalDate) } // reverse of millisToDays def daysToMillis(days: SQLDate): Long = { - daysToMillis(days, defaultTimeZone()) + daysToMillis(days, defaultTimeZone().toZoneId) } - def daysToMillis(days: SQLDate, timeZone: TimeZone): Long = { - val instant = daysToLocalDate(days).atStartOfDay(timeZone.toZoneId).toInstant + def daysToMillis(days: SQLDate, zoneId: ZoneId): Long = { + val instant = daysToLocalDate(days).atStartOfDay(zoneId).toInstant instantToMicros(instant) / MICROS_PER_MILLIS } @@ -585,11 +585,11 @@ object DateTimeUtils { time1: SQLTimestamp, time2: SQLTimestamp, roundOff: Boolean, - timeZone: TimeZone): Double = { + zoneId: ZoneId): Double = { val millis1 = MICROSECONDS.toMillis(time1) val millis2 = MICROSECONDS.toMillis(time2) - val date1 = millisToDays(millis1, timeZone) - val date2 = millisToDays(millis2, timeZone) + val date1 = millisToDays(millis1, zoneId) + val date2 = millisToDays(millis2, zoneId) val (year1, monthInYear1, dayInMonth1, daysToMonthEnd1) = splitDate(date1) val (year2, monthInYear2, dayInMonth2, daysToMonthEnd2) = splitDate(date2) @@ -603,8 +603,8 @@ object DateTimeUtils { } // using milliseconds can cause precision loss with more than 8 digits // we follow Hive's implementation which uses seconds - val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, timeZone)) - val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, timeZone)) + val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, zoneId)) + val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, zoneId)) val secondsDiff = (dayInMonth1 - dayInMonth2) * SECONDS_PER_DAY + secondsInDay1 - secondsInDay2 val secondsInMonth = DAYS.toSeconds(31) val diff = monthDiff + secondsDiff / secondsInMonth.toDouble @@ -733,8 +733,8 @@ object DateTimeUtils { millis += offset millis - millis % MILLIS_PER_DAY - offset case _ => // Try to truncate date levels - val dDays = millisToDays(millis, timeZone) - daysToMillis(truncDate(dDays, level), timeZone) + val dDays = millisToDays(millis, timeZone.toZoneId) + daysToMillis(truncDate(dDays, level), timeZone.toZoneId) } truncated * MICROS_PER_MILLIS } 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 77a2ca7e4a828..536c76f042d23 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,6 +19,7 @@ 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 @@ -137,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, TimeZone.getTimeZone("GMT"))) + assert(castedDate == DateTimeUtils.millisToDays(expectedDate, ZoneOffset.UTC)) val timestamp = "2015-01-01 00:00:00" timestampsOptions = new CSVOptions(Map( 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 a46ede5cfcd39..852ab87e24101 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 @@ -55,9 +55,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val ts = new Timestamp(toMillis(time)) test("datetime function current_date") { - val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) + val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC) val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int] - val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) + val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC) assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1) val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int] @@ -772,14 +772,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { 1000L) checkEvaluation( UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), -1000L) checkEvaluation(UnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) val t1 = UnixTimestamp( CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] val t2 = UnixTimestamp( @@ -794,7 +795,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { null) checkEvaluation( UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) } @@ -827,14 +829,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { 1000L) checkEvaluation( ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( ToUnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), -1000L) checkEvaluation(ToUnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) val t1 = ToUnixTimestamp( CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] val t2 = ToUnixTimestamp( @@ -848,7 +851,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { null) checkEvaluation(ToUnixTimestamp( Literal(date1), Literal.create(null, StringType), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( ToUnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) 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 cabcd3007d1c0..7f914b05f1a24 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 @@ -413,22 +413,22 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { test("monthsBetween") { val date1 = date(1997, 2, 28, 10, 30, 0) var date2 = date(1996, 10, 30) - assert(monthsBetween(date1, date2, true, TimeZoneUTC) === 3.94959677) - assert(monthsBetween(date1, date2, false, TimeZoneUTC) === 3.9495967741935485) + assert(monthsBetween(date1, date2, true, ZoneOffset.UTC) === 3.94959677) + assert(monthsBetween(date1, date2, false, ZoneOffset.UTC) === 3.9495967741935485) Seq(true, false).foreach { roundOff => date2 = date(2000, 2, 28) - assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36) + assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36) date2 = date(2000, 2, 29) - assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36) + assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36) date2 = date(1996, 3, 31) - assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === 11) + assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === 11) } val date3 = date(2000, 2, 28, 16, tz = TimeZonePST) val date4 = date(1997, 2, 28, 16, tz = TimeZonePST) - assert(monthsBetween(date3, date4, true, TimeZonePST) === 36.0) - assert(monthsBetween(date3, date4, true, TimeZoneGMT) === 35.90322581) - assert(monthsBetween(date3, date4, false, TimeZoneGMT) === 35.903225806451616) + assert(monthsBetween(date3, date4, true, TimeZonePST.toZoneId) === 36.0) + assert(monthsBetween(date3, date4, true, ZoneOffset.UTC) === 35.90322581) + assert(monthsBetween(date3, date4, false, ZoneOffset.UTC) === 35.903225806451616) } test("from UTC timestamp") { @@ -571,15 +571,15 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { test("daysToMillis and millisToDays") { val input = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, 16, tz = TimeZonePST)) - assert(millisToDays(input, TimeZonePST) === 16800) - assert(millisToDays(input, TimeZoneGMT) === 16801) - assert(millisToDays(-1 * MILLIS_PER_DAY + 1, TimeZoneGMT) == -1) + assert(millisToDays(input, TimeZonePST.toZoneId) === 16800) + assert(millisToDays(input, ZoneOffset.UTC) === 16801) + assert(millisToDays(-1 * MILLIS_PER_DAY + 1, ZoneOffset.UTC) == -1) var expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZonePST)) - assert(daysToMillis(16800, TimeZonePST) === expected) + assert(daysToMillis(16800, TimeZonePST.toZoneId) === expected) expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZoneGMT)) - assert(daysToMillis(16800, TimeZoneGMT) === expected) + assert(daysToMillis(16800, ZoneOffset.UTC) === expected) // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( @@ -594,7 +594,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { val skipped = skipped_days.getOrElse(tz.getID, Set.empty) (-20000 to 20000).foreach { d => if (!skipped.contains(d)) { - assert(millisToDays(daysToMillis(d, tz), tz) === d, + assert(millisToDays(daysToMillis(d, tz.toZoneId), tz.toZoneId) === d, s"Round trip of ${d} did not work in tz ${tz}") } } From 1890589021d6b925ba9cce9d7b7053010e15a628 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 8 Feb 2020 11:06:25 +0300 Subject: [PATCH 4/7] Use Java 8 time API classes in SQLQueryTestSuite --- .../org/apache/spark/sql/execution/HiveResult.scala | 5 +++++ .../sql-tests/results/postgreSQL/date.sql.out | 12 ++++++------ .../org/apache/spark/sql/SQLQueryTestSuite.scala | 1 + 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index bbe47a63f4d61..5a2f16d8e1526 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} @@ -67,8 +68,12 @@ object HiveResult { case (null, _) => if (nested) "null" else "NULL" case (b, BooleanType) => b.toString case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d)) + case (ld: LocalDate, DateType) => + dateFormatter.format(DateTimeUtils.localDateToDays(ld)) case (t: Timestamp, TimestampType) => timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t)) + case (i: Instant, TimestampType) => + timestampFormatter.format(DateTimeUtils.instantToMicros(i)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (n, _: NumericType) => n.toString diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index fd5dc42632176..ed27317121623 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -800,7 +800,7 @@ SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query schema struct -- !query output -1001-01-01 00:07:02 +1001-01-01 00:00:00 -- !query @@ -808,7 +808,7 @@ SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') -- !query schema struct -- !query output -1001-01-01 00:07:02 +1001-01-01 00:00:00 -- !query @@ -840,7 +840,7 @@ SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') -- !query schema struct -- !query output -0001-01-01 00:07:02 +0001-01-01 00:00:00 -- !query @@ -848,7 +848,7 @@ SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) -- !query schema struct -- !query output --0099-01-01 00:07:02 +-0099-01-01 00:00:00 -- !query @@ -864,7 +864,7 @@ SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') -- !query schema struct -- !query output -0000-01-01 00:07:02 +0000-01-01 00:00:00 -- !query @@ -872,7 +872,7 @@ SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query schema struct -- !query output --0010-01-01 00:07:02 +-0010-01-01 00:00:00 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 2e5a9e0b4d45d..6b9e5bbd3c961 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -337,6 +337,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } + localSparkSession.conf.set(SQLConf.DATETIME_JAVA8API_ENABLED.key, true) if (configSet.nonEmpty) { // Execute the list of set operation in order to add the desired configs From 6a64946661de797b35629c4f99242e8052c0fda6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 8 Feb 2020 19:15:54 +0300 Subject: [PATCH 5/7] Fix DateTimeUtilsSuite --- .../apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 7f914b05f1a24..cd0594c775a47 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 @@ -86,9 +86,13 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("SPARK-6785: java date conversion before and after epoch") { + def format(d: Date): String = { + TimestampFormatter("uuuu-MM-dd", defaultTimeZone().toZoneId) + .format(d.getTime * MICROS_PER_MILLIS) + } def checkFromToJavaDate(d1: Date): Unit = { val d2 = toJavaDate(fromJavaDate(d1)) - assert(d2.toString === d1.toString) + assert(format(d2) === format(d1)) } val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) From cb37fe3e94ec62f86c884083737ffeb2bb803baa Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 10 Feb 2020 13:33:10 +0300 Subject: [PATCH 6/7] Replace 2933263 by LocalDate.of(10001, 1, 1).toEpochDay --- .../spark/sql/catalyst/expressions/DateExpressionsSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 852ab87e24101..856e699f80485 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 @@ -494,7 +494,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // Valid range of DateType is [0001-01-01, 9999-12-31] val maxMonthInterval = 10000 * 12 checkEvaluation( - AddMonths(Literal(LocalDate.parse("0001-01-01")), Literal(maxMonthInterval)), 2933263) + AddMonths(Literal(LocalDate.parse("0001-01-01")), Literal(maxMonthInterval)), + LocalDate.of(10001, 1, 1).toEpochDay.toInt) checkEvaluation( AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529) // Test evaluation results between Interpreted mode and Codegen mode From e5d463f67d68efcff046851479b4789d1e4f50a0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 12 Feb 2020 15:44:15 +0300 Subject: [PATCH 7/7] Fix merge conflicts --- .../catalyst/expressions/DateExpressionsSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 d11ecf070ea0b..39b859af47ca9 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 @@ -789,7 +789,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), @@ -797,7 +797,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(UnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) val t1 = UnixTimestamp( CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] val t2 = UnixTimestamp( @@ -815,7 +815,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) } @@ -853,7 +853,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( ToUnixTimestamp( Literal(sdf2.format(new Timestamp(-1000000))), @@ -862,7 +862,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ToUnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) val t1 = ToUnixTimestamp( CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] val t2 = ToUnixTimestamp( @@ -877,7 +877,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ToUnixTimestamp( Literal(date1), Literal.create(null, StringType), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( ToUnixTimestamp( Literal("2015-07-24"),