Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ class Iso8601DateFormatter(
val specialDate = convertSpecialDate(s.trim, zoneId)
specialDate.getOrElse {
try {
val localDate = toLocalDate(formatter.parse(s))
val localDate = toLocalDate(formatter.parse(s), locale)
localDateToDays(localDate)
} catch checkDiffResult(s, legacyFormatter.parse)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.util
import java.time._
import java.time.chrono.IsoChronology
import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverStyle}
import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries}
import java.time.temporal._
import java.util.Locale

import com.google.common.cache.CacheBuilder
Expand All @@ -31,26 +31,52 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._

trait DateTimeFormatterHelper {
private def getOrDefault(accessor: TemporalAccessor, field: ChronoField, default: Int): Int = {

private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = {
if (accessor.isSupported(field)) {
accessor.get(field)
} else {
default
}
}

protected def toLocalDate(accessor: TemporalAccessor): LocalDate = {
private def mayNonWeekBased(accessor: TemporalAccessor): Boolean = {
val has = accessor.isSupported _
has(ChronoField.YEAR) || has(ChronoField.MONTH_OF_YEAR) || has(ChronoField.DAY_OF_MONTH)
}

private def mayWeekBased(accessor: TemporalAccessor, wf: WeekFields): Boolean = {
val has = accessor.isSupported _
has(wf.weekBasedYear) || has(wf.weekOfMonth) || has(wf.weekOfWeekBasedYear) || has(wf.dayOfWeek)
}

@throws[DateTimeException]
protected def toLocalDate(accessor: TemporalAccessor, locale: Locale): LocalDate = {
val localDate = accessor.query(TemporalQueries.localDate())
// If all the date fields are specified, return the local date directly.
// If all the date fields are resolved(yMd or Ywu), return the local date directly.
if (localDate != null) return localDate

// Users may want to parse only a few datetime fields from a string and extract these fields
// later, and we should provide default values for missing fields.
// To be compatible with Spark 2.4, we pick 1970 as the default value of year.
val year = getOrDefault(accessor, ChronoField.YEAR, 1970)
val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1)
val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1)
LocalDate.of(year, month, day)
val weekFields = WeekFields.of(locale)
val weekBased = mayWeekBased(accessor, weekFields)
if (weekBased && mayNonWeekBased(accessor)) {
throw new DateTimeException(
s"Can not mix week-based and non-week-based date fields together for parsing dates")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should this happen when we create the formatter?

} else if (weekBased) {

val weekBasedYear = getOrDefault(accessor, weekFields.weekBasedYear, 1970)
val week = getOrDefault(accessor, weekFields.weekOfWeekBasedYear(), 1)
val dayOfWeek = getOrDefault(accessor, weekFields.dayOfWeek(), 1)

LocalDate.of(1970, 1, 1)
.`with`(weekFields.weekOfWeekBasedYear, week)
.`with`(weekFields.dayOfWeek, dayOfWeek)
.`with`(weekFields.weekBasedYear, weekBasedYear)
} else {
val year = getOrDefault(accessor, ChronoField.YEAR, 1970)
val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1)
val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1)
LocalDate.of(year, month, day)
}
}

private def toLocalTime(accessor: TemporalAccessor): LocalTime = {
Expand All @@ -74,8 +100,11 @@ trait DateTimeFormatterHelper {

// Converts the parsed temporal object to ZonedDateTime. It sets time components to zeros
// if they does not exist in the parsed object.
protected def toZonedDateTime(accessor: TemporalAccessor, zoneId: ZoneId): ZonedDateTime = {
val localDate = toLocalDate(accessor)
protected def toZonedDateTime(
accessor: TemporalAccessor,
zoneId: ZoneId,
locale: Locale): ZonedDateTime = {
val localDate = toLocalDate(accessor, locale)
val localTime = toLocalTime(accessor)
ZonedDateTime.of(localDate, localTime, zoneId)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ class Iso8601TimestampFormatter(
val parsed = formatter.parse(s)
val parsedZoneId = parsed.query(TemporalQueries.zone())
val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId
val zonedDateTime = toZonedDateTime(parsed, timeZoneId)
val zonedDateTime = toZonedDateTime(parsed, timeZoneId, locale)
val epochSeconds = zonedDateTime.toEpochSecond
val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1168,4 +1168,43 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkExceptionInExpression[ArithmeticException](
MillisToTimestamp(Literal(-92233720368547758L)), "long overflow")
}

test("SPARK-31868: Restore the behaviour week-based-year for 2.4") {
checkEvaluation(
new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child,
Timestamp.valueOf("1968-12-30 00:00:00.0"))

// the existence of 'W' is not for generating the timestamp but likely for checking whether
// the timestamp falling into it.
checkEvaluation(
new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child,
Timestamp.valueOf("1968-12-30 00:00:00.0"))

checkEvaluation(
new ParseToTimestamp(Literal("1969 5 2"), Literal("YYYY W u")).child,
Timestamp.valueOf("1968-12-30 00:00:00.0"))

checkEvaluation(
new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-dd")).child,
null)
// https://bugs.openjdk.java.net/browse/JDK-8145633
// Adjacent value parsing not supported for Localized Patterns
checkExceptionInExpression[SparkUpgradeException](
new ParseToTimestamp(Literal("196940"), Literal("YYYYww")).child, "3.0")

checkExceptionInExpression[SparkUpgradeException](
new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child,
"3.0")

checkEvaluation(
new ParseToTimestamp(Literal("2018-46-7 13:33:33"), Literal("YYYY-ww-u HH:mm:ss")).child,
Timestamp.valueOf("2018-11-17 13:33:33.0"))

checkExceptionInExpression[SparkUpgradeException](
new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, "3.0")

checkExceptionInExpression[SparkUpgradeException](
new ParseToTimestamp(Literal("1969 1 6 1"), Literal("yyyy M d u")).child, "3.0")

}
}
28 changes: 28 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/datetime.sql
Original file line number Diff line number Diff line change
Expand Up @@ -164,3 +164,31 @@ select from_csv('26/October/2015', 'date Date', map('dateFormat', 'dd/MMMMM/yyyy
select from_unixtime(1, 'yyyyyyyyyyy-MM-dd');
select date_format(timestamp '2018-11-17 13:33:33', 'yyyyyyyyyy-MM-dd HH:mm:ss');
select date_format(date '2018-11-17', 'yyyyyyyyyyy-MM-dd');

-- SPARK-31868: Restore the behaviour week-based-year for 2.4
select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss'); -- only the week-based year matters for the date part
select to_timestamp('1969-01-01', 'YYYY-MM-dd');
select to_timestamp('1969-12-31', 'YYYY-MM-dd');
select to_timestamp('2018-01-01', 'YYYY-MM-dd');
select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss');
select to_timestamp('1969 1 1', 'yyyy w u');
select to_timestamp('1969 1 1', 'yyyy M u');
select to_timestamp('1 1969 1', 'M YYYY u');
select to_timestamp('1969 1 1', 'YYYY M u');
select to_timestamp('1969 1 6 1', 'yyyy M d u');
select to_timestamp('1969 1 5 1', 'yyyy M d u');
-- YYYY-ww-dd
select to_timestamp('2018 11 17', 'YYYY ww dd');
select to_timestamp('1969 2 11', 'YYYY W dd');
select to_timestamp('1969 5 11 11', 'YYYY M ww dd');

select to_timestamp('2020 1 3 2', 'yyyy M w u');
select to_timestamp('2018-11-2-17', 'yyyy-ww-W-dd');
select to_timestamp('2018-11-2-11', 'yyyy-ww-W-dd');
select to_timestamp('2018-11-3-12', 'yyyy-ww-W-dd');
select to_timestamp('2018-11-10', 'yyyy-ww-dd');
select to_timestamp('1', 'u');
select to_timestamp('5 2', 'u d');
select to_timestamp('5 3', 'u d');
select to_timestamp('2020 2020', 'YYYY yyyy');
select to_timestamp('2020 2020', 'YYYY yyyy');
209 changes: 208 additions & 1 deletion sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 119
-- Number of queries: 143


-- !query
Expand Down Expand Up @@ -1025,3 +1025,210 @@ struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to recognize 'yyyyyyyyyyy-MM-dd' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html


-- !query
select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-11-17 13:33:33' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969-01-01', 'YYYY-MM-dd')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969-01-01' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969-12-31', 'YYYY-MM-dd')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969-12-31' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('2018-01-01', 'YYYY-MM-dd')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-01-01' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018-11-17 13:33:33' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969 1 1', 'yyyy w u')
-- !query schema
struct<to_timestamp(1969 1 1, yyyy w u):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('1969 1 1', 'yyyy M u')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1 1969 1', 'M YYYY u')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1 1969 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969 1 1', 'YYYY M u')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969 1 6 1', 'yyyy M d u')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 1 6 1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969 1 5 1', 'yyyy M d u')
-- !query schema
struct<to_timestamp(1969 1 5 1, yyyy M d u):timestamp>
-- !query output
1969-01-05 00:00:00


-- !query
select to_timestamp('2018 11 17', 'YYYY ww dd')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2018 11 17' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('1969 2 11', 'YYYY W dd')
-- !query schema
struct<to_timestamp(1969 2 11, YYYY W dd):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('1969 5 11 11', 'YYYY M ww dd')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '1969 5 11 11' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('2020 1 3 2', 'yyyy M w u')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 1 3 2' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('2018-11-2-17', 'yyyy-ww-W-dd')
-- !query schema
struct<to_timestamp(2018-11-2-17, yyyy-ww-W-dd):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('2018-11-2-11', 'yyyy-ww-W-dd')
-- !query schema
struct<to_timestamp(2018-11-2-11, yyyy-ww-W-dd):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('2018-11-3-12', 'yyyy-ww-W-dd')
-- !query schema
struct<to_timestamp(2018-11-3-12, yyyy-ww-W-dd):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('2018-11-10', 'yyyy-ww-dd')
-- !query schema
struct<to_timestamp(2018-11-10, yyyy-ww-dd):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('1', 'u')
-- !query schema
struct<to_timestamp(1, u):timestamp>
-- !query output
1969-12-28 00:00:00


-- !query
select to_timestamp('5 2', 'u d')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '5 2' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('5 3', 'u d')
-- !query schema
struct<to_timestamp(5 3, u d):timestamp>
-- !query output
NULL


-- !query
select to_timestamp('2020 2020', 'YYYY yyyy')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 2020' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.


-- !query
select to_timestamp('2020 2020', 'YYYY yyyy')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
You may get a different result due to the upgrading of Spark 3.0: Fail to parse '2020 2020' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
Loading