diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 76ae3e5e8469a..08c704bf375ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.text.SimpleDateFormat import java.time.{LocalDate, ZoneId} -import java.util.{Date, Locale} +import java.util.{Date, Locale, TimeZone} import org.apache.commons.lang3.time.FastDateFormat @@ -85,14 +85,16 @@ class Iso8601DateFormatter( } trait LegacyDateFormatter extends DateFormatter { + val zoneId: ZoneId + val timeZone: TimeZone = TimeZone.getTimeZone(zoneId) def parseToDate(s: String): Date override def parse(s: String): Int = { - fromJavaDate(new java.sql.Date(parseToDate(s).getTime)) + fromJavaDate(new java.sql.Date(parseToDate(s).getTime), timeZone) } override def format(days: Int): String = { - format(DateTimeUtils.toJavaDate(days)) + format(DateTimeUtils.toJavaDate(days, timeZone)) } override def format(localDate: LocalDate): String = { @@ -100,21 +102,30 @@ trait LegacyDateFormatter extends DateFormatter { } } -class LegacyFastDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { +class LegacyFastDateFormatter( + pattern: String, + override val zoneId: ZoneId, + locale: Locale) extends LegacyDateFormatter { @transient - private lazy val fdf = FastDateFormat.getInstance(pattern, locale) + private lazy val fdf = FastDateFormat.getInstance(pattern, timeZone, locale) override def parseToDate(s: String): Date = fdf.parse(s) override def format(d: Date): String = fdf.format(d) override def validatePatternString(): Unit = fdf } -class LegacySimpleDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { +class LegacySimpleDateFormatter( + pattern: String, + override val zoneId: ZoneId, + locale: Locale) extends LegacyDateFormatter { @transient - private lazy val sdf = new SimpleDateFormat(pattern, locale) + private lazy val sdf = { + val formatter = new SimpleDateFormat(pattern, locale) + formatter.setTimeZone(timeZone) + formatter + } override def parseToDate(s: String): Date = sdf.parse(s) override def format(d: Date): String = sdf.format(d) override def validatePatternString(): Unit = sdf - } object DateFormatter { @@ -147,9 +158,9 @@ object DateFormatter { legacyFormat: LegacyDateFormat): DateFormatter = { legacyFormat match { case FAST_DATE_FORMAT => - new LegacyFastDateFormatter(pattern, locale) + new LegacyFastDateFormatter(pattern, zoneId, locale) case SIMPLE_DATE_FORMAT | LENIENT_SIMPLE_DATE_FORMAT => - new LegacySimpleDateFormatter(pattern, locale) + new LegacySimpleDateFormatter(pattern, zoneId, locale) } } 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 21a478aaf06a6..d89e9f6da2a54 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 @@ -88,50 +88,55 @@ object DateTimeUtils { } /** - * Converts an instance of `java.sql.Date` to a number of days since the epoch - * 1970-01-01 via extracting date fields `year`, `month`, `days` from the input, - * creating a local date in Proleptic Gregorian calendar from the fields, and - * getting the number of days from the resulted local date. - * - * This approach was taken to have the same local date as the triple of `year`, - * `month`, `day` in the original hybrid calendar used by `java.sql.Date` and - * Proleptic Gregorian calendar used by Spark since version 3.0.0, see SPARK-26651. + * Converts a local date at the give time zone to the number of days since 1970-01-01 in the + * hybrid calendar (Julian + Gregorian) by discarding the time part. The resulted days are + * rebased from the hybrid to Proleptic Gregorian calendar. The days rebasing is performed via + * UTC time zone for simplicity because the difference between two calendars is the same in + * any given time zone and UTC time zone. * * @param date It represents a specific instant in time based on * the hybrid calendar which combines Julian and * Gregorian calendars. - * @return The number of days since epoch from java.sql.Date. + * @param timeZone The time zone of the local `date`. + * @return The number of days since the epoch in Proleptic Gregorian calendar. */ - def fromJavaDate(date: Date): SQLDate = { + def fromJavaDate(date: Date, timeZone: TimeZone): SQLDate = { val millisUtc = date.getTime - val millisLocal = millisUtc + TimeZone.getDefault.getOffset(millisUtc) + val millisLocal = millisUtc + timeZone.getOffset(millisUtc) val julianDays = Math.toIntExact(Math.floorDiv(millisLocal, MILLIS_PER_DAY)) rebaseJulianToGregorianDays(julianDays) } + def fromJavaDate(date: Date): SQLDate = { + fromJavaDate(date, TimeZone.getDefault) + } + /** - * The opposite to `fromJavaDate` method which converts a number of days to an - * instance of `java.sql.Date`. It builds a local date in Proleptic Gregorian - * calendar, extracts date fields `year`, `month`, `day`, and creates a local - * date in the hybrid calendar (Julian + Gregorian calendars) from the fields. + * Converts days since the epoch 1970-01-01 in Proleptic Gregorian calendar to a local date + * at the given time zone in the hybrid calendar (Julian + Gregorian). It rebases the given + * days from Proleptic Gregorian to the hybrid calendar at UTC time zone for simplicity because + * the difference between two calendars doesn't depend on any time zone. The result is shifted + * by time zone offset in wall clock to have the same date fields (year, month, day) + * at the given `timeZone` as the input `daysSinceEpoch` in Proleptic Gregorian calendar. * - * The purpose of the conversion is to have the same local date as the triple - * of `year`, `month`, `day` in the original Proleptic Gregorian calendar and - * in the target calender. - * - * @param daysSinceEpoch The number of days since 1970-01-01. - * @return A `java.sql.Date` from number of days since epoch. + * @param daysSinceEpoch The number of days since 1970-01-01 in Proleptic Gregorian calendar. + * @param timeZone The time zone of the desired local date. + * @return A local date in the hybrid calendar as `java.sql.Date` from number of days since epoch. */ - def toJavaDate(daysSinceEpoch: SQLDate): Date = { + def toJavaDate(daysSinceEpoch: SQLDate, timeZone: TimeZone): Date = { val rebasedDays = rebaseGregorianToJulianDays(daysSinceEpoch) val localMillis = Math.multiplyExact(rebasedDays, MILLIS_PER_DAY) - val timeZoneOffset = TimeZone.getDefault match { + val timeZoneOffset = timeZone match { case zoneInfo: ZoneInfo => zoneInfo.getOffsetsByWall(localMillis, null) case timeZone: TimeZone => timeZone.getOffset(localMillis - timeZone.getRawOffset) } new Date(localMillis - timeZoneOffset) } + def toJavaDate(daysSinceEpoch: SQLDate): Date = { + toJavaDate(daysSinceEpoch, TimeZone.getDefault) + } + /** * Converts microseconds since the epoch to an instance of `java.sql.Timestamp` * via creating a local timestamp at the system time zone in Proleptic Gregorian diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala index 4892deae92b3d..0def7fd169e7b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateFormatterSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.util -import java.time.{DateTimeException, LocalDate} +import java.time.{DateTimeException, LocalDate, ZoneId} +import java.util.{Calendar, TimeZone} import org.apache.spark.SparkUpgradeException import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ @@ -31,26 +32,35 @@ class DateFormatterSuite extends DatetimeFormatterSuite { DateFormatter(pattern, UTC, isParsing) } - test("parsing dates") { - outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter(getZoneId(timeZone)) - val daysSinceEpoch = formatter.parse("2018-12-02") - assert(daysSinceEpoch === 17867) + private def withOutstandingZoneIds(f: ZoneId => Unit): Unit = { + for { + jvmZoneId <- outstandingZoneIds + sessionZoneId <- outstandingZoneIds + } { + withDefaultTimeZone(jvmZoneId) { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionZoneId.getId) { + f(sessionZoneId) + } } } } + test("parsing dates") { + withOutstandingZoneIds { zoneId => + val formatter = DateFormatter(zoneId) + val daysSinceEpoch = formatter.parse("2018-12-02") + assert(daysSinceEpoch === 17867) + } + } + test("format dates") { - outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter(getZoneId(timeZone)) - val (days, expected) = (17867, "2018-12-02") - val date = formatter.format(days) - assert(date === expected) - assert(formatter.format(daysToLocalDate(days)) === expected) - assert(formatter.format(toJavaDate(days)) === expected) - } + withOutstandingZoneIds { zoneId => + val formatter = DateFormatter(zoneId) + val (days, expected) = (17867, "2018-12-02") + val date = formatter.format(days) + assert(date === expected) + assert(formatter.format(daysToLocalDate(days)) === expected) + assert(formatter.format(toJavaDate(days, TimeZone.getTimeZone(zoneId))) === expected) } } @@ -69,19 +79,17 @@ class DateFormatterSuite extends DatetimeFormatterSuite { "2018-12-12", "2038-01-01", "5010-11-17").foreach { date => - outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter( - DateFormatter.defaultPattern, - getZoneId(timeZone), - DateFormatter.defaultLocale, - legacyFormat, - isParsing = false) - val days = formatter.parse(date) - assert(date === formatter.format(days)) - assert(date === formatter.format(daysToLocalDate(days))) - assert(date === formatter.format(toJavaDate(days))) - } + withOutstandingZoneIds { zoneId => + val formatter = DateFormatter( + DateFormatter.defaultPattern, + zoneId, + DateFormatter.defaultLocale, + legacyFormat, + isParsing = false) + val days = formatter.parse(date) + assert(date === formatter.format(days)) + assert(date === formatter.format(daysToLocalDate(days))) + assert(date === formatter.format(toJavaDate(days, TimeZone.getTimeZone(zoneId)))) } } } @@ -104,18 +112,16 @@ class DateFormatterSuite extends DatetimeFormatterSuite { 17877, 24837, 1110657).foreach { days => - outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter( - DateFormatter.defaultPattern, - getZoneId(timeZone), - DateFormatter.defaultLocale, - legacyFormat, - isParsing = false) - val date = formatter.format(days) - val parsed = formatter.parse(date) - assert(days === parsed) - } + withOutstandingZoneIds { zoneId => + val formatter = DateFormatter( + DateFormatter.defaultPattern, + zoneId, + DateFormatter.defaultLocale, + legacyFormat, + isParsing = false) + val date = formatter.format(days) + val parsed = formatter.parse(date) + assert(days === parsed) } } } @@ -172,19 +178,22 @@ class DateFormatterSuite extends DatetimeFormatterSuite { test("SPARK-31557: rebasing in legacy formatters/parsers") { withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> LegacyBehaviorPolicy.LEGACY.toString) { LegacyDateFormats.values.foreach { legacyFormat => - outstandingTimezonesIds.foreach { timeZone => - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter( - DateFormatter.defaultPattern, - getZoneId(timeZone), - DateFormatter.defaultLocale, - legacyFormat, - isParsing = false) - assert(LocalDate.ofEpochDay(formatter.parse("1000-01-01")) === LocalDate.of(1000, 1, 1)) - assert(formatter.format(LocalDate.of(1000, 1, 1)) === "1000-01-01") - assert(formatter.format(localDateToDays(LocalDate.of(1000, 1, 1))) === "1000-01-01") - assert(formatter.format(java.sql.Date.valueOf("1000-01-01")) === "1000-01-01") - } + withOutstandingZoneIds { zoneId => + val formatter = DateFormatter( + DateFormatter.defaultPattern, + zoneId, + DateFormatter.defaultLocale, + legacyFormat, + isParsing = false) + assert(LocalDate.ofEpochDay(formatter.parse("1000-01-01")) === LocalDate.of(1000, 1, 1)) + assert(formatter.format(LocalDate.of(1000, 1, 1)) === "1000-01-01") + assert(formatter.format(localDateToDays(LocalDate.of(1000, 1, 1))) === "1000-01-01") + val cal = new Calendar.Builder() + .setCalendarType("gregory") + .setTimeZone(TimeZone.getTimeZone(zoneId)) + .setDate(1000, 0, 1) + .build() + assert(formatter.format(cal.getTime) === "1000-01-01") } } }