-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-35780][SQL] Support DATE/TIMESTAMP literals across the full range #32959
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-35780][SQL] Support DATE/TIMESTAMP literals across the full range #32959
Conversation
|
Test build #139960 has finished for PR 32959 at commit
|
|
Kubernetes integration test unable to build dist. exiting with code: 1 |
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
Outdated
Show resolved
Hide resolved
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
Outdated
Show resolved
Hide resolved
6966058 to
633781f
Compare
| -- !query output | ||
| java.time.DateTimeException | ||
| Cannot cast 7 to DateType. | ||
| 728567 00:00:00.000000000 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
cc @cloud-fan, this is an existing mismatch between ANSI and non-ANSI mode. For non-ANSI mode, this query will throw exception
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's still request at least 4 digits for year
|
Test build #140505 has finished for PR 32959 at commit
|
|
Test build #140510 has finished for PR 32959 at commit
|
|
Kubernetes integration test starting |
|
Kubernetes integration test status success |
|
Kubernetes integration test starting |
|
Test build #140526 has finished for PR 32959 at commit
|
|
Kubernetes integration test status success |
| "date.sql", | ||
| "datetime.sql", | ||
| "datetime-legacy.sql", | ||
| "ansi/datetime.sql", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
same reason to "date.sql" that thriftserver couldn't handle negative year
| test("SPARK-30960, SPARK-31641: parse date/timestamp string with legacy format") { | ||
| val julianDay = -141704 // 1582-01-01 in Julian calendar | ||
| val ds = Seq( | ||
| s"{'t': '2020-1-12 3:23:34.12', 'd': '2020-1-12 T', 'd2': '12345', 'd3': '$julianDay'}" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
'12345' and '-141704' are treated as epoch days before this PR because it's out of the 0000-9999 range.
this is used for backward compatibility with JSON data generated by spark 1.5.
But this compatibility is very confusing, for example, before this PR:
'9999' will be converted to '9999-01-01' while '10000' will be converted to '1997-05-19'
So I suggest just removing this compatibility
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
Show resolved
Hide resolved
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
Show resolved
Hide resolved
|
Kubernetes integration test unable to build dist. exiting with code: 1 |
|
Test build #140566 has finished for PR 32959 at commit
|
|
Kubernetes integration test starting |
|
Kubernetes integration test status success |
|
Kubernetes integration test starting |
|
cc @cloud-fan, comments are addressed and tests are passed |
|
Kubernetes integration test status success |
|
Test build #140922 has finished for PR 32959 at commit
|
|
Kubernetes integration test starting |
|
Test build #140958 has finished for PR 32959 at commit
|
|
Kubernetes integration test status success |
|
@linhongliu-db please fix the code conflicts. |
|
Kubernetes integration test starting |
|
Kubernetes integration test status success |
|
Test build #140976 has finished for PR 32959 at commit
|
|
Kubernetes integration test unable to build dist. exiting with code: 1 |
|
thanks, merging to master/3.2 (since it's timestamp related) |
### What changes were proposed in this pull request?
DATE/TIMESTAMP literals support years 0000 to 9999. However, internally we support a range that is much larger.
We can add or subtract large intervals from a date/timestamp and the system will happily process and display large negative and positive dates.
Since we obviously cannot put this genie back into the bottle the only thing we can do is allow matching DATE/TIMESTAMP literals.
### Why are the changes needed?
make spark more usable and bug fix
### Does this PR introduce _any_ user-facing change?
Yes, after this PR, below SQL will have different results
```sql
select cast('-10000-1-2' as date) as date_col
-- before PR: NULL
-- after PR: -10000-1-2
```
```sql
select cast('2021-4294967297-11' as date) as date_col
-- before PR: 2021-01-11
-- after PR: NULL
```
### How was this patch tested?
newly added test cases
Closes #32959 from linhongliu-db/SPARK-35780-full-range-datetime.
Lead-authored-by: Linhong Liu <[email protected]>
Co-authored-by: Linhong Liu <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
(cherry picked from commit b866457)
Signed-off-by: Wenchen Fan <[email protected]>
|
Test build #141008 has finished for PR 32959 at commit
|
### What changes were proposed in this pull request? In PR #32959, we found some weird datetime strings that can be parsed. ([details](#32959 (comment))) This PR blocks the invalid datetime string. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? Yes, below strings will have different results when cast to datetime. ```sql select cast('12::' as timestamp); -- Before: 2021-07-07 12:00:00, After: NULL select cast('T' as timestamp); -- Before: 2021-07-07 00:00:00, After: NULL ``` ### How was this patch tested? some new test cases Closes #33490 from linhongliu-db/SPARK-35780-block-invalid-format. Authored-by: Linhong Liu <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
### What changes were proposed in this pull request? In PR #32959, we found some weird datetime strings that can be parsed. ([details](#32959 (comment))) This PR blocks the invalid datetime string. ### Why are the changes needed? bug fix ### Does this PR introduce _any_ user-facing change? Yes, below strings will have different results when cast to datetime. ```sql select cast('12::' as timestamp); -- Before: 2021-07-07 12:00:00, After: NULL select cast('T' as timestamp); -- Before: 2021-07-07 00:00:00, After: NULL ``` ### How was this patch tested? some new test cases Closes #33490 from linhongliu-db/SPARK-35780-block-invalid-format. Authored-by: Linhong Liu <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit ed0e351) Signed-off-by: Wenchen Fan <[email protected]>
| if (s == null) { | ||
| def isValidDigits(segment: Int, digits: Int): Boolean = { | ||
| // An integer is able to represent a date within [+-]5 million years. | ||
| var maxDigitsYear = 7 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can I implement a configuration item that configures the range of digits allowed for the year?
I found that it was writing to tables in different formats and the results would behave differently.
create table t(c1 date) stored as textfile;
insert overwrite table t select cast( '22022-05-01' as date);
select * from t1; -- output nullcreate table t(c1 date) stored as orcfile;
insert overwrite table t select cast( '22022-05-01' as date);
select * from t1; -- output +22022-05-01Because orc/parquet date stores integers, but textfile and sequencefile store text.
But if you use hive jdbc, the query will fail, because java.sql.Date only supports 4-digit years.
Caused by: java.lang.IllegalArgumentException
at java.sql.Date.valueOf(Date.java:143)
at org.apache.hive.jdbc.HiveBaseResultSet.evaluate(HiveBaseResultSet.java:447
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's expected that not all the data sources and BI clients support datetime values larger than 10000-01-01, the question is when the failure should happen.
It looks to me that the Hive table should fail to write 22022-05-01 with textfile source, and the hive jdbc should fail at the client-side saying 22022-05-01 is not supported.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BTW, I don't think it's possible to add a Spark config to forbid large datetime values. The literal is just one place, there are many other datetime operations that may produce large datetime values, which have been there before this PR.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your explanation, make sense.
There may be some dates that were treated as abnormal by users in previous Spark versions, and can be handled normally in Spark 3.2, although they are normal dates.
Because I didn't see this behavior change in the migration guide before noticing this PR.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yea the impact on BI clients was missed, though strictly speaking BI clients are not part of Spark.
…g dates in "yyyyMMdd" format with CORRECTED time parser policy
### What changes were proposed in this pull request?
This PR fixes a correctness issue when reading a CSV or a JSON file with dates in "yyyyMMdd" format:
```
name,mydate
1,2020011
2,20201203
```
or
```
{"date": "2020011"}
{"date": "20201203"}
```
Prior to #32959, reading this CSV file would return:
```
+----+--------------+
|name|mydate |
+----+--------------+
|1 |null |
|2 |2020-12-03 |
+----+--------------+
```
However, after the patch, the invalid date is parsed because of the much more lenient parsing in `DateTimeUtils.stringToDate`, the method treats `2020011` as a full year:
```
+----+--------------+
|name|mydate |
+----+--------------+
|1 |+2020011-01-01|
|2 |2020-12-03 |
+----+--------------+
```
Similar result would be observed in JSON.
This PR attempts to address correctness issue by introducing a new configuration option `enableDateTimeParsingFallback` which allows to enable/disable the backward compatible parsing.
Currently, by default we will fall back to the backward compatible behavior only if parser policy is legacy and no custom pattern was set (this is defined in `UnivocityParser` and `JacksonParser` for csv and json respectively).
### Why are the changes needed?
Fixes a correctness issue in Spark 3.4.
### Does this PR introduce _any_ user-facing change?
In order to avoid correctness issues when reading CSV or JSON files with a custom pattern, a new configuration option `enableDateTimeParsingFallback` has been added to control whether or not the code would fall back to the backward compatible behavior of parsing dates and timestamps in CSV and JSON data sources.
- If the config is enabled and the date cannot be parsed, we will fall back to `DateTimeUtils.stringToDate`.
- If the config is enabled and the timestamp cannot be parsed, `DateTimeUtils.stringToTimestamp` will be used.
- Otherwise, depending on the parser policy and a custom pattern, the value will be parsed as null.
### How was this patch tested?
I added unit tests for CSV and JSON to verify the fix and the config option.
Closes #37147 from sadikovi/fix-csv-date-inference.
Authored-by: Ivan Sadikov <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>

What changes were proposed in this pull request?
DATE/TIMESTAMP literals support years 0000 to 9999. However, internally we support a range that is much larger.
We can add or subtract large intervals from a date/timestamp and the system will happily process and display large negative and positive dates.
Since we obviously cannot put this genie back into the bottle the only thing we can do is allow matching DATE/TIMESTAMP literals.
Why are the changes needed?
make spark more usable and bug fix
Does this PR introduce any user-facing change?
Yes, after this PR, below SQL will have different results
How was this patch tested?
newly added test cases