-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-31797][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions #28534
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
Changes from 1 commit
5e60ec5
27a162b
f879987
4c17d49
3411935
17a33f5
344e5e1
407dcfc
944c644
642668c
62bf8b5
7da94b5
3704fee
f5b0a5a
f4f7c35
3c949ac
fdfd75c
f0cc631
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||
|---|---|---|---|---|---|---|---|---|
|
|
@@ -402,11 +402,11 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas | |||||||
| } | ||||||||
|
|
||||||||
| @ExpressionDescription( | ||||||||
| usage = "_FUNC_(date) - Returns timestamp from seconds.", | ||||||||
| usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.", | ||||||||
| examples = """ | ||||||||
| Examples: | ||||||||
| > SELECT _FUNC_(1230219000); | ||||||||
| "2008-12-25 07:30:00.0" | ||||||||
| "2008-12-25 07:30:00" | ||||||||
| """, | ||||||||
| group = "datetime_funcs", | ||||||||
| since = "3.1.0") | ||||||||
|
|
@@ -419,11 +419,12 @@ case class SecondsToTimestamp(child: Expression) | |||||||
| } | ||||||||
|
|
||||||||
| @ExpressionDescription( | ||||||||
| usage = "_FUNC_(date) - Returns timestamp from milliseconds.", | ||||||||
| usage = "_FUNC_(milliseconds) - " + | ||||||||
| "Creates timestamp from the number of milliseconds since UTC epoch.", | ||||||||
| examples = """ | ||||||||
| Examples: | ||||||||
| > SELECT _FUNC_(1230219000000); | ||||||||
|
||||||||
| > SELECT _FUNC_(1230219000000); | |
| > SELECT _FUNC_(12302190000123); | |
| "2008-12-25 07:30:00.123" |
Same for micros below.
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.
+1
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.
Ok, datetime.sql is also syncronized.
Outdated
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.
we don't need to check the type. The analyzer should guarantee it. We can just write
Math.multiplyExact(input.asInstanceOf[Long], upScaleFactor)
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.
Cool
Outdated
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.
ditto
Outdated
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.
this should be ev.value, otherwise you get class cast exception, as ev.value is a string.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2,6 +2,30 @@ | |
| -- Number of queries: 91 | ||
|
|
||
|
|
||
| -- !query | ||
| select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) | ||
| -- !query schema | ||
| struct<timestamp_seconds(CAST(1230219000 AS BIGINT)):timestamp,timestamp_seconds(CAST(-1230219000 AS BIGINT)):timestamp,timestamp_seconds(CAST(NULL AS BIGINT)):timestamp> | ||
|
||
| -- !query output | ||
| 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL | ||
|
|
||
|
|
||
| -- !query | ||
| select TIMESTAMP_MILLISECONDS(1230219000000),TIMESTAMP_MILLISECONDS(-1230219000000),TIMESTAMP_MILLISECONDS(null) | ||
| -- !query schema | ||
| struct<timestamp_milliseconds(1230219000000):timestamp,timestamp_milliseconds(-1230219000000):timestamp,timestamp_milliseconds(CAST(NULL AS BIGINT)):timestamp> | ||
| -- !query output | ||
| 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL | ||
|
|
||
|
|
||
| -- !query | ||
| select TIMESTAMP_MICROSECONDS(1230219000000000),TIMESTAMP_MICROSECONDS(-1230219000000000),TIMESTAMP_MICROSECONDS(null) | ||
| -- !query schema | ||
| struct<timestamp_microseconds(1230219000000000):timestamp,timestamp_microseconds(-1230219000000000):timestamp,timestamp_microseconds(CAST(NULL AS BIGINT)):timestamp> | ||
| -- !query output | ||
| 2008-12-25 07:30:00 1931-01-07 00:30:00 NULL | ||
|
|
||
|
|
||
| -- !query | ||
| select current_date = current_date(), current_timestamp = current_timestamp() | ||
| -- !query schema | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -3498,24 +3498,20 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark | |
|
|
||
| test("SPARK-31710: " + | ||
|
||
| "TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS to timestamp transfer") { | ||
| val df1 = sql("select TIMESTAMP_SECONDS(1230219000) as timestamp") | ||
| checkAnswer(df1, Row(Timestamp.valueOf("2008-12-25 07:30:00.0"))) | ||
|
|
||
| val df2 = sql("select TIMESTAMP_MILLISECONDS(1230219000000) as timestamp") | ||
| checkAnswer(df2, Row(Timestamp.valueOf("2008-12-25 07:30:00.0"))) | ||
|
|
||
| val df3 = sql("select TIMESTAMP_MICROSECONDS(1230219000000000) as timestamp") | ||
| checkAnswer(df3, Row(Timestamp.valueOf("2008-12-25 07:30:00.0"))) | ||
|
|
||
| val df4 = sql("select TIMESTAMP_SECONDS(-1230219000) as timestamp") | ||
| checkAnswer(df4, Row(Timestamp.valueOf("1931-01-07 00:30:00.0"))) | ||
|
|
||
| val df5 = sql("select TIMESTAMP_MILLISECONDS(-1230219000000) as timestamp") | ||
| checkAnswer(df5, Row(Timestamp.valueOf("1931-01-07 00:30:00.0"))) | ||
|
|
||
| val df6 = sql("select TIMESTAMP_MICROSECONDS(-1230219000000000) as timestamp") | ||
| checkAnswer(df6, Row(Timestamp.valueOf("1931-01-07 00:30:00.0"))) | ||
|
|
||
| val df1 = sql("select TIMESTAMP_SECONDS(1230219000)," + | ||
| "TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)") | ||
| checkAnswer(df1, Row(Timestamp.valueOf("2008-12-25 07:30:00"), | ||
| Timestamp.valueOf("1931-01-07 00:30:00"), null)) | ||
|
|
||
| val df2 = sql("select TIMESTAMP_MILLISECONDS(1230219000000)," + | ||
| "TIMESTAMP_MILLISECONDS(-1230219000000),TIMESTAMP_MILLISECONDS(null)") | ||
| checkAnswer(df2, Row(Timestamp.valueOf("2008-12-25 07:30:00"), | ||
| Timestamp.valueOf("1931-01-07 00:30:00"), null)) | ||
|
|
||
| val df3 = sql("select TIMESTAMP_MICROSECONDS(1230219000000000)," + | ||
| "TIMESTAMP_MICROSECONDS(-1230219000000000),TIMESTAMP_MICROSECONDS(null)") | ||
| checkAnswer(df3, Row(Timestamp.valueOf("2008-12-25 07:30:00"), | ||
| Timestamp.valueOf("1931-01-07 00:30:00"), null)) | ||
|
||
| } | ||
| } | ||
|
|
||
|
|
||
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.
nit: let's do either:
or
just for the sake of consistency.