Skip to content

Commit 0dcd5df

Browse files
committed
forbid mixed patterns in formatter
1 parent 9e91729 commit 0dcd5df

File tree

4 files changed

+127
-165
lines changed

4 files changed

+127
-165
lines changed

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

Lines changed: 36 additions & 78 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,6 @@ import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverSt
2323
import java.time.temporal._
2424
import java.util.Locale
2525

26-
import scala.util.control.NonFatal
27-
2826
import com.google.common.cache.CacheBuilder
2927

3028
import org.apache.spark.SparkUpgradeException
@@ -33,91 +31,51 @@ import org.apache.spark.sql.internal.SQLConf
3331
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._
3432

3533
trait DateTimeFormatterHelper {
36-
private def getFieldValue(
37-
accessor: TemporalAccessor,
38-
field: TemporalField): Option[Int] = {
34+
35+
private def getOrDefault(accessor: TemporalAccessor, field: TemporalField, default: Int): Int = {
3936
if (accessor.isSupported(field)) {
40-
try {
41-
Option(accessor.get(field))
42-
} catch {
43-
case NonFatal(_) => None
44-
}
37+
accessor.get(field)
4538
} else {
46-
None
39+
default
4740
}
4841
}
4942

43+
private def mayNonWeekBased(accessor: TemporalAccessor): Boolean = {
44+
val has = accessor.isSupported _
45+
has(ChronoField.YEAR) || has(ChronoField.MONTH_OF_YEAR) || has(ChronoField.DAY_OF_MONTH)
46+
}
47+
48+
private def mayWeekBased(accessor: TemporalAccessor, wf: WeekFields): Boolean = {
49+
val has = accessor.isSupported _
50+
has(wf.weekBasedYear) || has(wf.weekOfMonth) || has(wf.weekOfWeekBasedYear) || has(wf.dayOfWeek)
51+
}
52+
5053
@throws[DateTimeException]
5154
protected def toLocalDate(accessor: TemporalAccessor, locale: Locale): LocalDate = {
5255
val localDate = accessor.query(TemporalQueries.localDate())
5356
// If all the date fields are resolved(yMd or Ywu), return the local date directly.
5457
if (localDate != null) return localDate
5558

56-
var res = LocalDate.of(1970, 1, 1)
57-
5859
val weekFields = WeekFields.of(locale)
59-
val weekBasedYearField = weekFields.weekBasedYear
60-
var weekBasedYearEnabled = false
61-
62-
val year = getFieldValue(accessor, weekBasedYearField).map { y =>
63-
weekBasedYearEnabled = true
64-
y
65-
}.orElse {
66-
getFieldValue(accessor, ChronoField.YEAR)
67-
}
68-
69-
val week = getFieldValue(accessor, weekFields.weekOfWeekBasedYear())
70-
val dayOfWeek = getFieldValue(accessor, weekFields.dayOfWeek())
71-
72-
// TODO: How to check 'W' week-of-month field, not like other week-based field, it always throw
73-
// UnsupportedTemporalTypeException to get it even `accessor.isSupported` passed.
74-
75-
if (weekBasedYearEnabled) {
76-
// If the week-based-year field exists, only the week-based fields matters.
77-
res.`with`(weekFields.weekOfWeekBasedYear, week.getOrElse(1).toLong)
78-
.`with`(weekFields.dayOfWeek(), dayOfWeek.getOrElse(1).toLong)
79-
.`with`(weekBasedYearField, year.get)
60+
val weekBased = mayWeekBased(accessor, weekFields)
61+
if (weekBased && mayNonWeekBased(accessor)) {
62+
throw new DateTimeException(
63+
s"Can not mix week-based and non-week-based date fields together for parsing dates")
64+
} else if (weekBased) {
65+
66+
val weekBasedYear = getOrDefault(accessor, weekFields.weekBasedYear, 1970)
67+
val week = getOrDefault(accessor, weekFields.weekOfWeekBasedYear(), 1)
68+
val dayOfWeek = getOrDefault(accessor, weekFields.dayOfWeek(), 1)
69+
70+
LocalDate.of(1970, 1, 1)
71+
.`with`(weekFields.weekOfWeekBasedYear, week)
72+
.`with`(weekFields.dayOfWeek, dayOfWeek)
73+
.`with`(weekFields.weekBasedYear, weekBasedYear)
8074
} else {
81-
if (year.isDefined) {
82-
res = res.withYear(year.get)
83-
}
84-
85-
val month = getFieldValue(accessor, ChronoField.MONTH_OF_YEAR)
86-
val day = getFieldValue(accessor, ChronoField.DAY_OF_MONTH)
87-
88-
if (month.isDefined && week.isDefined) {
89-
// check the week fall into the correct month of the year
90-
res = res.`with`(weekFields.weekOfWeekBasedYear(), week.get)
91-
if (res.getMonthValue != month.get) {
92-
throw new DateTimeException(
93-
s"week-of-week-based-year value: ${week.get} conflicts with month-of-year value:" +
94-
s" ${month.get} which should be ${res.getMonthValue} instead.")
95-
}
96-
} else if (month.isDefined) {
97-
res = res.withMonth(month.get)
98-
} else if (week.isDefined) {
99-
if (day.isDefined) {
100-
throw new DateTimeException(
101-
s"Can not use week-of-week-based-year and day-of-month together in non-week-based" +
102-
s" mode.")
103-
}
104-
res = res.`with`(weekFields.weekOfWeekBasedYear, week.get)
105-
}
106-
107-
if (dayOfWeek.isDefined && day.isDefined) {
108-
// check whether the days matches
109-
res = res.`with`(weekFields.dayOfWeek(), dayOfWeek.get)
110-
if (res.getDayOfMonth != day.get) {
111-
throw new DateTimeException(
112-
s"day-of-week value: ${dayOfWeek.get} conflicts with day-of-month value:" +
113-
s" ${day.get} which should be ${res.getDayOfMonth} instead.")
114-
}
115-
} else if (day.isDefined) {
116-
res = res.withDayOfMonth(day.get)
117-
} else if (dayOfWeek.isDefined) {
118-
res = res.`with`(weekFields.dayOfWeek(), dayOfWeek.get)
119-
}
120-
res
75+
val year = getOrDefault(accessor, ChronoField.YEAR, 1970)
76+
val month = getOrDefault(accessor, ChronoField.MONTH_OF_YEAR, 1)
77+
val day = getOrDefault(accessor, ChronoField.DAY_OF_MONTH, 1)
78+
LocalDate.of(year, month, day)
12179
}
12280
}
12381

@@ -134,10 +92,10 @@ trait DateTimeFormatterHelper {
13492
} else {
13593
0
13694
}
137-
val minute = getFieldValue(accessor, ChronoField.MINUTE_OF_HOUR)
138-
val second = getFieldValue(accessor, ChronoField.SECOND_OF_MINUTE)
139-
val nanoSecond = getFieldValue(accessor, ChronoField.NANO_OF_SECOND)
140-
LocalTime.of(hour, minute.getOrElse(0), second.getOrElse(0), nanoSecond.getOrElse(0))
95+
val minute = getOrDefault(accessor, ChronoField.MINUTE_OF_HOUR, 0)
96+
val second = getOrDefault(accessor, ChronoField.SECOND_OF_MINUTE, 0)
97+
val nanoSecond = getOrDefault(accessor, ChronoField.NANO_OF_SECOND, 0)
98+
LocalTime.of(hour, minute, second, nanoSecond)
14199
}
142100

143101
// Converts the parsed temporal object to ZonedDateTime. It sets time components to zeros

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

Lines changed: 9 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -1170,63 +1170,41 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
11701170
}
11711171

11721172
test("SPARK-31868: Restore the behaviour week-based-year for 2.4") {
1173-
// TODO: Locale.US is Sunday started day of week, which affects the new formatter,
1174-
// while in 2.4 it's Monday first.
1175-
checkEvaluation(
1176-
new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-MM-dd")).child,
1177-
Timestamp.valueOf("2017-12-31 00:00:00.0"))
1178-
1179-
checkEvaluation(
1180-
new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child,
1181-
Timestamp.valueOf("2017-12-31 13:33:33.0"))
1182-
11831173
checkEvaluation(
11841174
new ParseToTimestamp(Literal("1969 1 2"), Literal("YYYY w u")).child,
11851175
Timestamp.valueOf("1968-12-30 00:00:00.0"))
11861176

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

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

1197-
// // the legacy parser does not support 'W' and results null, so SparkUpgradeException will come
1198-
// checkExceptionInExpression[SparkUpgradeException](
1199-
// new ParseToTimestamp(Literal("1969 4 2"), Literal("YYYY W u")).child, "3.0")
1200-
// checkExceptionInExpression[SparkUpgradeException](
1201-
// new ParseToTimestamp(Literal("5"), Literal("W")).child, "3.0")
1202-
1187+
checkEvaluation(
1188+
new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-dd")).child,
1189+
null)
12031190
// https://bugs.openjdk.java.net/browse/JDK-8145633
12041191
// Adjacent value parsing not supported for Localized Patterns
12051192
checkExceptionInExpression[SparkUpgradeException](
12061193
new ParseToTimestamp(Literal("196940"), Literal("YYYYww")).child, "3.0")
12071194

1208-
checkEvaluation(
1209-
new ParseToTimestamp(Literal("2020 1 3 2"), Literal("yyyy M w u")).child,
1210-
Timestamp.valueOf("2020-01-13 00:00:00.0"))
1195+
checkExceptionInExpression[SparkUpgradeException](
1196+
new ParseToTimestamp(Literal("2018-11-17 13:33:33"), Literal("YYYY-MM-dd HH:mm:ss")).child,
1197+
"3.0")
12111198

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

1216-
checkEvaluation(
1217-
new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child,
1218-
Timestamp.valueOf("2018-03-11 00:00:00.0"))
1219-
1220-
checkEvaluation(
1221-
new ParseToTimestamp(Literal("2018-11-2-17"), Literal("yyyy-ww-dd")).child,
1222-
null)
1203+
checkExceptionInExpression[SparkUpgradeException](
1204+
new ParseToTimestamp(Literal("2018-11-17"), Literal("YYYY-ww-dd")).child, "3.0")
12231205

1224-
// problem of first day of week change
12251206
checkExceptionInExpression[SparkUpgradeException](
12261207
new ParseToTimestamp(Literal("1969 1 6 1"), Literal("yyyy M d u")).child, "3.0")
12271208

1228-
checkEvaluation(
1229-
new ParseToTimestamp(Literal("1969 5 11 11"), Literal("YYYY M ww dd")).child,
1230-
Timestamp.valueOf("1969-03-09 00:00:00.0"))
12311209
}
12321210
}

sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out

Lines changed: 41 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -1030,73 +1030,81 @@ You may get a different result due to the upgrading of Spark 3.0: Fail to recogn
10301030
-- !query
10311031
select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss')
10321032
-- !query schema
1033-
struct<to_timestamp(2018-11-17 13:33:33, YYYY-MM-dd HH:mm:ss):timestamp>
1033+
struct<>
10341034
-- !query output
1035-
2017-12-31 13:33:33
1035+
org.apache.spark.SparkUpgradeException
1036+
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.
10361037

10371038

10381039
-- !query
10391040
select to_timestamp('1969-01-01', 'YYYY-MM-dd')
10401041
-- !query schema
1041-
struct<to_timestamp(1969-01-01, YYYY-MM-dd):timestamp>
1042+
struct<>
10421043
-- !query output
1043-
1968-12-29 00:00:00
1044+
org.apache.spark.SparkUpgradeException
1045+
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.
10441046

10451047

10461048
-- !query
10471049
select to_timestamp('1969-12-31', 'YYYY-MM-dd')
10481050
-- !query schema
1049-
struct<to_timestamp(1969-12-31, YYYY-MM-dd):timestamp>
1051+
struct<>
10501052
-- !query output
1051-
1968-12-29 00:00:00
1053+
org.apache.spark.SparkUpgradeException
1054+
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.
10521055

10531056

10541057
-- !query
10551058
select to_timestamp('2018-01-01', 'YYYY-MM-dd')
10561059
-- !query schema
1057-
struct<to_timestamp(2018-01-01, YYYY-MM-dd):timestamp>
1060+
struct<>
10581061
-- !query output
1059-
2017-12-31 00:00:00
1062+
org.apache.spark.SparkUpgradeException
1063+
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.
10601064

10611065

10621066
-- !query
10631067
select to_timestamp('2018-11-17 13:33:33', 'YYYY-MM-dd HH:mm:ss')
10641068
-- !query schema
1065-
struct<to_timestamp(2018-11-17 13:33:33, YYYY-MM-dd HH:mm:ss):timestamp>
1069+
struct<>
10661070
-- !query output
1067-
2017-12-31 13:33:33
1071+
org.apache.spark.SparkUpgradeException
1072+
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.
10681073

10691074

10701075
-- !query
10711076
select to_timestamp('1969 1 1', 'yyyy w u')
10721077
-- !query schema
10731078
struct<to_timestamp(1969 1 1, yyyy w u):timestamp>
10741079
-- !query output
1075-
1968-12-29 00:00:00
1080+
NULL
10761081

10771082

10781083
-- !query
10791084
select to_timestamp('1969 1 1', 'yyyy M u')
10801085
-- !query schema
1081-
struct<to_timestamp(1969 1 1, yyyy M u):timestamp>
1086+
struct<>
10821087
-- !query output
1083-
1968-12-29 00:00:00
1088+
org.apache.spark.SparkUpgradeException
1089+
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.
10841090

10851091

10861092
-- !query
10871093
select to_timestamp('1 1969 1', 'M YYYY u')
10881094
-- !query schema
1089-
struct<to_timestamp(1 1969 1, M YYYY u):timestamp>
1095+
struct<>
10901096
-- !query output
1091-
1968-12-29 00:00:00
1097+
org.apache.spark.SparkUpgradeException
1098+
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.
10921099

10931100

10941101
-- !query
10951102
select to_timestamp('1969 1 1', 'YYYY M u')
10961103
-- !query schema
1097-
struct<to_timestamp(1969 1 1, YYYY M u):timestamp>
1104+
struct<>
10981105
-- !query output
1099-
1968-12-29 00:00:00
1106+
org.apache.spark.SparkUpgradeException
1107+
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.
11001108

11011109

11021110
-- !query
@@ -1119,33 +1127,36 @@ struct<to_timestamp(1969 1 5 1, yyyy M d u):timestamp>
11191127
-- !query
11201128
select to_timestamp('2018 11 17', 'YYYY ww dd')
11211129
-- !query schema
1122-
struct<to_timestamp(2018 11 17, YYYY ww dd):timestamp>
1130+
struct<>
11231131
-- !query output
1124-
2018-03-11 00:00:00
1132+
org.apache.spark.SparkUpgradeException
1133+
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.
11251134

11261135

11271136
-- !query
11281137
select to_timestamp('1969 2 11', 'YYYY W dd')
11291138
-- !query schema
11301139
struct<to_timestamp(1969 2 11, YYYY W dd):timestamp>
11311140
-- !query output
1132-
1968-12-29 00:00:00
1141+
NULL
11331142

11341143

11351144
-- !query
11361145
select to_timestamp('1969 5 11 11', 'YYYY M ww dd')
11371146
-- !query schema
1138-
struct<to_timestamp(1969 5 11 11, YYYY M ww dd):timestamp>
1147+
struct<>
11391148
-- !query output
1140-
1969-03-09 00:00:00
1149+
org.apache.spark.SparkUpgradeException
1150+
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.
11411151

11421152

11431153
-- !query
11441154
select to_timestamp('2020 1 3 2', 'yyyy M w u')
11451155
-- !query schema
1146-
struct<to_timestamp(2020 1 3 2, yyyy M w u):timestamp>
1156+
struct<>
11471157
-- !query output
1148-
2020-01-13 00:00:00
1158+
org.apache.spark.SparkUpgradeException
1159+
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.
11491160

11501161

11511162
-- !query
@@ -1208,14 +1219,16 @@ NULL
12081219
-- !query
12091220
select to_timestamp('2020 2020', 'YYYY yyyy')
12101221
-- !query schema
1211-
struct<to_timestamp(2020 2020, YYYY yyyy):timestamp>
1222+
struct<>
12121223
-- !query output
1213-
2019-12-29 00:00:00
1224+
org.apache.spark.SparkUpgradeException
1225+
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.
12141226

12151227

12161228
-- !query
12171229
select to_timestamp('2020 2020', 'YYYY yyyy')
12181230
-- !query schema
1219-
struct<to_timestamp(2020 2020, YYYY yyyy):timestamp>
1231+
struct<>
12201232
-- !query output
1221-
2019-12-29 00:00:00
1233+
org.apache.spark.SparkUpgradeException
1234+
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.

0 commit comments

Comments
 (0)