You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2020/05/22 14:17:28 UTC
[spark] branch master updated: [SPARK-31710][SQL] Adds
TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions
This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 2115c55 [SPARK-31710][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions
2115c55 is described below
commit 2115c55efe5d4bfe60ff4271f15624bd16a01929
Author: TJX2014 <xi...@gmail.com>
AuthorDate: Fri May 22 14:16:30 2020 +0000
[SPARK-31710][SQL] Adds TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions
### What changes were proposed in this pull request?
Add and register three new functions: `TIMESTAMP_SECONDS`, `TIMESTAMP_MILLIS` and `TIMESTAMP_MICROS`
A test is added.
Reference: [BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/timestamp_functions?hl=en#timestamp_seconds)
### Why are the changes needed?
People will have convenient way to get timestamps from seconds,milliseconds and microseconds.
### Does this PR introduce _any_ user-facing change?
Yes, people will have the following ways to get timestamp:
```scala
sql("select TIMESTAMP_SECONDS(t.a) as timestamp from values(1230219000),(-1230219000) as t(a)").show(false)
```
```
+-------------------------+
|timestamp |
+-------------------------+
|2008-12-25 23:30:00|
|1931-01-07 16:30:00|
+-------------------------+
```
```scala
sql("select TIMESTAMP_MILLIS(t.a) as timestamp from values(1230219000123),(-1230219000123) as t(a)").show(false)
```
```
+-------------------------------+
|timestamp |
+-------------------------------+
|2008-12-25 23:30:00.123|
|1931-01-07 16:29:59.877|
+-------------------------------+
```
```scala
sql("select TIMESTAMP_MICROS(t.a) as timestamp from values(1230219000123123),(-1230219000123123) as t(a)").show(false)
```
```
+------------------------------------+
|timestamp |
+------------------------------------+
|2008-12-25 23:30:00.123123|
|1931-01-07 16:29:59.876877|
+------------------------------------+
```
### How was this patch tested?
Unit test.
Closes #28534 from TJX2014/master-SPARK-31710.
Authored-by: TJX2014 <xi...@gmail.com>
Signed-off-by: Wenchen Fan <we...@databricks.com>
---
.../sql/catalyst/analysis/FunctionRegistry.scala | 3 +
.../catalyst/expressions/datetimeExpressions.scala | 77 ++++++++++++++++++++++
.../expressions/DateExpressionsSuite.scala | 22 +++++++
.../sql-functions/sql-expression-schema.md | 5 +-
.../test/resources/sql-tests/inputs/datetime.sql | 10 +++
.../sql-tests/results/ansi/datetime.sql.out | 62 ++++++++++++++++-
.../resources/sql-tests/results/datetime.sql.out | 60 +++++++++++++++++
7 files changed, 237 insertions(+), 2 deletions(-)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 26d7b3e..5e53927 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -424,6 +424,9 @@ object FunctionRegistry {
expression[MakeInterval]("make_interval"),
expression[DatePart]("date_part"),
expression[Extract]("extract"),
+ expression[SecondsToTimestamp]("timestamp_seconds"),
+ expression[MillisToTimestamp]("timestamp_millis"),
+ expression[MicrosToTimestamp]("timestamp_micros"),
// collection functions
expression[CreateArray]("array"),
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index ccedcb4..afc57aa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -401,6 +401,83 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas
}
}
+abstract class NumberToTimestampBase extends UnaryExpression
+ with ExpectsInputTypes {
+
+ protected def upScaleFactor: Long
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType)
+
+ override def dataType: DataType = TimestampType
+
+ override def nullSafeEval(input: Any): Any = {
+ Math.multiplyExact(input.asInstanceOf[Number].longValue(), upScaleFactor)
+ }
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ if (upScaleFactor == 1) {
+ defineCodeGen(ctx, ev, c => c)
+ } else {
+ defineCodeGen(ctx, ev, c => s"java.lang.Math.multiplyExact($c, ${upScaleFactor}L)")
+ }
+ }
+}
+
+@ExpressionDescription(
+ usage = "_FUNC_(seconds) - Creates timestamp from the number of seconds since UTC epoch.",
+ examples = """
+ Examples:
+ > SELECT _FUNC_(1230219000);
+ 2008-12-25 07:30:00
+ """,
+ group = "datetime_funcs",
+ since = "3.1.0")
+case class SecondsToTimestamp(child: Expression)
+ extends NumberToTimestampBase {
+
+ override def upScaleFactor: Long = MICROS_PER_SECOND
+
+ override def prettyName: String = "timestamp_seconds"
+}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = "_FUNC_(milliseconds) - Creates timestamp from the number of milliseconds since UTC epoch.",
+ examples = """
+ Examples:
+ > SELECT _FUNC_(1230219000123);
+ 2008-12-25 07:30:00.123
+ """,
+ group = "datetime_funcs",
+ since = "3.1.0")
+// scalastyle:on line.size.limit
+case class MillisToTimestamp(child: Expression)
+ extends NumberToTimestampBase {
+
+ override def upScaleFactor: Long = MICROS_PER_MILLIS
+
+ override def prettyName: String = "timestamp_millis"
+}
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = "_FUNC_(microseconds) - Creates timestamp from the number of microseconds since UTC epoch.",
+ examples = """
+ Examples:
+ > SELECT _FUNC_(1230219000123123);
+ 2008-12-25 07:30:00.123123
+ """,
+ group = "datetime_funcs",
+ since = "3.1.0")
+// scalastyle:on line.size.limit
+case class MicrosToTimestamp(child: Expression)
+ extends NumberToTimestampBase {
+
+ override def upScaleFactor: Long = 1L
+
+ override def prettyName: String = "timestamp_micros"
+}
+
@ExpressionDescription(
usage = "_FUNC_(date) - Returns the year component of the date/timestamp.",
examples = """
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 6e8397d..87062f2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -1146,4 +1146,26 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
Literal("yyyy-MM-dd'T'HH:mm:ss.SSSz")), "Fail to parse")
}
}
+
+ test("SPARK-31710:Adds TIMESTAMP_SECONDS, " +
+ "TIMESTAMP_MILLIS and TIMESTAMP_MICROS functions") {
+ checkEvaluation(SecondsToTimestamp(Literal(1230219000)), 1230219000L * MICROS_PER_SECOND)
+ checkEvaluation(SecondsToTimestamp(Literal(-1230219000)), -1230219000L * MICROS_PER_SECOND)
+ checkEvaluation(SecondsToTimestamp(Literal(null, IntegerType)), null)
+ checkEvaluation(MillisToTimestamp(Literal(1230219000123L)), 1230219000123L * MICROS_PER_MILLIS)
+ checkEvaluation(MillisToTimestamp(
+ Literal(-1230219000123L)), -1230219000123L * MICROS_PER_MILLIS)
+ checkEvaluation(MillisToTimestamp(Literal(null, IntegerType)), null)
+ checkEvaluation(MicrosToTimestamp(Literal(1230219000123123L)), 1230219000123123L)
+ checkEvaluation(MicrosToTimestamp(Literal(-1230219000123123L)), -1230219000123123L)
+ checkEvaluation(MicrosToTimestamp(Literal(null, IntegerType)), null)
+ checkExceptionInExpression[ArithmeticException](
+ SecondsToTimestamp(Literal(1230219000123123L)), "long overflow")
+ checkExceptionInExpression[ArithmeticException](
+ SecondsToTimestamp(Literal(-1230219000123123L)), "long overflow")
+ checkExceptionInExpression[ArithmeticException](
+ MillisToTimestamp(Literal(92233720368547758L)), "long overflow")
+ checkExceptionInExpression[ArithmeticException](
+ MillisToTimestamp(Literal(-92233720368547758L)), "long overflow")
+ }
}
diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
index 3570fb6..0efc2a6 100644
--- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
+++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md
@@ -1,6 +1,6 @@
<!-- Automatically generated by ExpressionsSchemaSuite -->
## Summary
- - Number of queries: 333
+ - Number of queries: 336
- Number of expressions that missing example: 34
- Expressions missing examples: and,string,tinyint,double,smallint,date,decimal,boolean,float,binary,bigint,int,timestamp,cume_dist,dense_rank,input_file_block_length,input_file_block_start,input_file_name,lag,lead,monotonically_increasing_id,ntile,struct,!,not,or,percent_rank,rank,row_number,spark_partition_id,version,window,positive,count_min_sketch
## Schema of Built-in Functions
@@ -175,6 +175,8 @@
| org.apache.spark.sql.catalyst.expressions.MapValues | map_values | SELECT map_values(map(1, 'a', 2, 'b')) | struct<map_values(map(1, a, 2, b)):array<string>> |
| org.apache.spark.sql.catalyst.expressions.MapZipWith | map_zip_with | SELECT map_zip_with(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)) | struct<map_zip_with(map(1, a, 2, b), map(1, x, 2, y), lambdafunction(concat(namedlambdavariable(), namedlambdavariable()), namedlambdavariable(), namedlambdavariable(), namedlambdavariable())):map<int,string>> |
| org.apache.spark.sql.catalyst.expressions.Md5 | md5 | SELECT md5('Spark') | struct<md5(CAST(Spark AS BINARY)):string> |
+| org.apache.spark.sql.catalyst.expressions.MicrosToTimestamp | timestamp_micros | SELECT timestamp_micros(1230219000123123) | struct<timestamp_micros(1230219000123123):timestamp> |
+| org.apache.spark.sql.catalyst.expressions.MillisToTimestamp | timestamp_millis | SELECT timestamp_millis(1230219000123) | struct<timestamp_millis(1230219000123):timestamp> |
| org.apache.spark.sql.catalyst.expressions.Minute | minute | SELECT minute('2009-07-30 12:58:59') | struct<minute(CAST(2009-07-30 12:58:59 AS TIMESTAMP)):int> |
| org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID | monotonically_increasing_id | N/A | N/A |
| org.apache.spark.sql.catalyst.expressions.Month | month | SELECT month('2016-07-30') | struct<month(CAST(2016-07-30 AS DATE)):int> |
@@ -223,6 +225,7 @@
| org.apache.spark.sql.catalyst.expressions.SchemaOfCsv | schema_of_csv | SELECT schema_of_csv('1,abc') | struct<schema_of_csv(1,abc):string> |
| org.apache.spark.sql.catalyst.expressions.SchemaOfJson | schema_of_json | SELECT schema_of_json('[{"col":0}]') | struct<schema_of_json([{"col":0}]):string> |
| org.apache.spark.sql.catalyst.expressions.Second | second | SELECT second('2009-07-30 12:58:59') | struct<second(CAST(2009-07-30 12:58:59 AS TIMESTAMP)):int> |
+| org.apache.spark.sql.catalyst.expressions.SecondsToTimestamp | timestamp_seconds | SELECT timestamp_seconds(1230219000) | struct<timestamp_seconds(1230219000):timestamp> |
| org.apache.spark.sql.catalyst.expressions.Sentences | sentences | SELECT sentences('Hi there! Good morning.') | struct<sentences(Hi there! Good morning., , ):array<array<string>>> |
| org.apache.spark.sql.catalyst.expressions.Sequence | sequence | SELECT sequence(1, 5) | struct<sequence(1, 5):array<int>> |
| org.apache.spark.sql.catalyst.expressions.Sha1 | sha1 | SELECT sha1('Spark') | struct<sha1(CAST(Spark AS BINARY)):string> |
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
index 9be857e..0fb373f 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
@@ -1,5 +1,15 @@
-- date time functions
+-- [SPARK-31710] TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS to timestamp transfer
+select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null);
+select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null);
+select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null);
+-- overflow exception:
+select TIMESTAMP_SECONDS(1230219000123123);
+select TIMESTAMP_SECONDS(-1230219000123123);
+select TIMESTAMP_MILLIS(92233720368547758);
+select TIMESTAMP_MILLIS(-92233720368547758);
+
-- [SPARK-16836] current_date and current_timestamp literals
select current_date = current_date(), current_timestamp = current_timestamp();
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
index ee155ac..2e61cb8 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
@@ -1,5 +1,65 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 91
+-- Number of queries: 92
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
+-- !query schema
+struct<timestamp_seconds(1230219000):timestamp,timestamp_seconds(-1230219000):timestamp,timestamp_seconds(CAST(NULL AS INT)):timestamp>
+-- !query output
+2008-12-25 07:30:00 1931-01-07 00:30:00 NULL
+
+
+-- !query
+select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null)
+-- !query schema
+struct<timestamp_millis(1230219000123):timestamp,timestamp_millis(-1230219000123):timestamp,timestamp_millis(CAST(NULL AS INT)):timestamp>
+-- !query output
+2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL
+
+
+-- !query
+select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null)
+-- !query schema
+struct<timestamp_micros(1230219000123123):timestamp,timestamp_micros(-1230219000123123):timestamp,timestamp_micros(CAST(NULL AS INT)):timestamp>
+-- !query output
+2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000123123)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_SECONDS(-1230219000123123)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_MILLIS(92233720368547758)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_MILLIS(-92233720368547758)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
-- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
index f3fc6a5..4b879fc 100755
--- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
@@ -3,6 +3,66 @@
-- !query
+select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null)
+-- !query schema
+struct<timestamp_seconds(1230219000):timestamp,timestamp_seconds(-1230219000):timestamp,timestamp_seconds(CAST(NULL AS INT)):timestamp>
+-- !query output
+2008-12-25 07:30:00 1931-01-07 00:30:00 NULL
+
+
+-- !query
+select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null)
+-- !query schema
+struct<timestamp_millis(1230219000123):timestamp,timestamp_millis(-1230219000123):timestamp,timestamp_millis(CAST(NULL AS INT)):timestamp>
+-- !query output
+2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL
+
+
+-- !query
+select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null)
+-- !query schema
+struct<timestamp_micros(1230219000123123):timestamp,timestamp_micros(-1230219000123123):timestamp,timestamp_micros(CAST(NULL AS INT)):timestamp>
+-- !query output
+2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL
+
+
+-- !query
+select TIMESTAMP_SECONDS(1230219000123123)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_SECONDS(-1230219000123123)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_MILLIS(92233720368547758)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
+select TIMESTAMP_MILLIS(-92233720368547758)
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+long overflow
+
+
+-- !query
select current_date = current_date(), current_timestamp = current_timestamp()
-- !query schema
struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean>
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org