Skip to content

[SPARK-31710][SQL] Fail casting numeric to timestamp by default #28593

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

Closed
wants to merge 101 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
101 commits
Select commit Hold shift + click to select a range
bb1efa2
Merge pull request #1 from apache/master
GuoPhilipse May 20, 2020
1459d5b
Merge pull request #2 from apache/master
GuoPhilipse May 22, 2020
88c40fe
Merge pull request #3 from apache/master
GuoPhilipse May 22, 2020
8d1deee
fail casting from integral to timestamp
GuoPhilipse May 22, 2020
df22083
Merge pull request #4 from apache/master
GuoPhilipse May 25, 2020
a02c6c7
'fix-testcase-fail'
GuoPhilipse May 25, 2020
e61c484
'add-restore-flag'
GuoPhilipse May 25, 2020
d99cc30
'fix-space'
GuoPhilipse May 25, 2020
0436611
Merge pull request #5 from apache/master
GuoPhilipse May 27, 2020
0a1a6a5
fix code style
GuoPhilipse May 27, 2020
ed80c84
Merge pull request #6 from apache/master
GuoPhilipse May 27, 2020
bae99b1
fix code style
GuoPhilipse May 27, 2020
39da7be
improve code
GuoPhilipse May 27, 2020
39ca87c
Merge pull request #7 from apache/master
GuoPhilipse May 28, 2020
0189b91
fix test cases failure
GuoPhilipse May 28, 2020
2c29c5b
fix test name
GuoPhilipse May 28, 2020
fd3efa2
fix test case error
GuoPhilipse May 28, 2020
69198c6
Merge branch 'master' into 31710-fix-compatibility
GuoPhilipse May 28, 2020
c3b3c89
Merge pull request #8 from apache/master
GuoPhilipse May 28, 2020
38f6e64
fix test case error
GuoPhilipse May 29, 2020
43f9b4e
Merge branch '31710-fix-compatibility' of https://github.com/GuoPhili…
GuoPhilipse May 29, 2020
852d730
fix cmd to bash
GuoPhilipse May 29, 2020
2e55be3
fix scala style
GuoPhilipse May 29, 2020
7637256
fix error test case
GuoPhilipse May 29, 2020
87cada7
fix scala style
GuoPhilipse May 29, 2020
13ae816
fix fail test cases
GuoPhilipse May 29, 2020
4abd8d3
fix scala style
GuoPhilipse May 29, 2020
b13316f
fix error test case
GuoPhilipse May 29, 2020
50cc0cf
fix scala style
GuoPhilipse May 29, 2020
22638e4
fix test case error
GuoPhilipse May 30, 2020
b1dda01
fix test case error
GuoPhilipse May 30, 2020
38f0e76
fix test case error
GuoPhilipse May 30, 2020
e478c90
fix pyspark test error
GuoPhilipse May 31, 2020
7348917
fix python style
GuoPhilipse May 31, 2020
e50e5cb
annotation examples casting from long to timestamp
GuoPhilipse Jun 1, 2020
2344e1f
fix pyspark test error
GuoPhilipse Jun 1, 2020
c3546eb
Merge pull request #9 from apache/master
GuoPhilipse Jun 2, 2020
c7cf8b9
Merge branch 'master' of https://github.com/GuoPhilipse/spark into 31…
GuoPhilipse Jun 2, 2020
5f138af
fix to_date using cast long to timestamp
GuoPhilipse Jun 2, 2020
77a339a
Merge pull request #10 from apache/master
GuoPhilipse Jun 3, 2020
3a3b93e
Merge branch 'master' of https://github.com/GuoPhilipse/spark into 31…
GuoPhilipse Jun 3, 2020
a6b4f74
Merge pull request #11 from apache/master
GuoPhilipse Jun 5, 2020
1b52fd6
submit to local
GuoPhilipse Jun 5, 2020
293fa9f
Merge branch 'master' of https://github.com/GuoPhilipse/spark into 31…
GuoPhilipse Jun 5, 2020
9cf87e8
extend functions and use new functions
GuoPhilipse Jun 5, 2020
cb21d5d
fix code style.etc
GuoPhilipse Jun 5, 2020
726371c
fix code style
GuoPhilipse Jun 5, 2020
8ed4fc4
fix doc format.etc
GuoPhilipse Jun 6, 2020
96c197c
fix testcase error
GuoPhilipse Jun 6, 2020
3d1819d
failed test case
GuoPhilipse Jun 6, 2020
be88f01
fix testcase error
GuoPhilipse Jun 6, 2020
fd7815a
fix find no test result problem
GuoPhilipse Jun 7, 2020
e14c4f9
fix find no test result problem
GuoPhilipse Jun 7, 2020
7cb0a54
fix testcase error
GuoPhilipse Jun 7, 2020
7ad82da
fix found no test result file
GuoPhilipse Jun 7, 2020
00ed960
fix test case error
GuoPhilipse Jun 7, 2020
664277e
Merge pull request #12 from apache/master
GuoPhilipse Jun 7, 2020
6d9eecb
Merge branch 'master' of https://github.com/GuoPhilipse/spark into 31…
GuoPhilipse Jun 7, 2020
b8b2919
fix test case fail
GuoPhilipse Jun 7, 2020
eeb0a61
fix test case fail
GuoPhilipse Jun 7, 2020
409c821
fix check spark streaming ui timeout
GuoPhilipse Jun 8, 2020
ec2cf54
fix additional space
GuoPhilipse Jun 8, 2020
3fd6d02
fix test case fail
GuoPhilipse Jun 8, 2020
fd677c9
Merge pull request #13 from apache/master
GuoPhilipse Jun 8, 2020
108dfea
merge
GuoPhilipse Jun 8, 2020
47f210c
fix test error.etc
GuoPhilipse Jun 8, 2020
860500d
fix test error.etc
GuoPhilipse Jun 8, 2020
4c7ddb4
revert support fraction types.etc
GuoPhilipse Jun 9, 2020
b43a140
remove useless space
GuoPhilipse Jun 9, 2020
1d5c750
fix code style.etc
GuoPhilipse Jun 9, 2020
7b26d0c
remove duplicated test cases
GuoPhilipse Jun 10, 2020
c7f2a9b
fix python style
GuoPhilipse Jun 10, 2020
8a5e9be
fix code style
GuoPhilipse Jun 10, 2020
93b1f63
Merge pull request #14 from apache/master
GuoPhilipse Jun 10, 2020
a6a9bd4
fix failed test case
GuoPhilipse Jun 10, 2020
a5b5474
Merge pull request #15 from apache/master
GuoPhilipse Jun 11, 2020
4fecf9b
Merge branch 'master' of https://github.com/GuoPhilipse/spark into 31…
GuoPhilipse Jun 11, 2020
f995f46
rebase
GuoPhilipse Jun 11, 2020
c8d5aa5
rebase
GuoPhilipse Jun 11, 2020
b4f4d53
delete test file
GuoPhilipse Jun 11, 2020
f4556a4
Merge pull request #16 from apache/master
GuoPhilipse Jun 11, 2020
f4da70b
rebase latest code
GuoPhilipse Jun 11, 2020
7132fca
rebase lastet code
GuoPhilipse Jun 11, 2020
7ffcde2
fix error cases
GuoPhilipse Jun 12, 2020
f84caa1
fix error cases
GuoPhilipse Jun 12, 2020
6d56a83
fix error cases
GuoPhilipse Jun 12, 2020
5716f40
remove cmd test
GuoPhilipse Jun 12, 2020
64acff3
'empty'
GuoPhilipse Jun 12, 2020
d26bd34
fix test case error
GuoPhilipse Jun 12, 2020
9545c9f
delete test file
GuoPhilipse Jun 12, 2020
01f42e6
fix python style
GuoPhilipse Jun 13, 2020
e29409d
fix python style
GuoPhilipse Jun 13, 2020
183d418
trigger rebuild
GuoPhilipse Jun 13, 2020
1764c76
fix python style
GuoPhilipse Jun 13, 2020
74eb6ca
improve python style
GuoPhilipse Jun 13, 2020
5f06714
fix python style
GuoPhilipse Jun 13, 2020
8fe1960
improve test queality
GuoPhilipse Jun 15, 2020
ffa3079
remove unnecessary commit
GuoPhilipse Jun 15, 2020
bc4b62c
remove test file
GuoPhilipse Jun 15, 2020
08aee30
improve test case
GuoPhilipse Jun 15, 2020
12b4239
improve test case
GuoPhilipse Jun 15, 2020
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
2 changes: 2 additions & 0 deletions docs/sql-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@ license: |

- In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`.

- In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can set `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710.

## Upgrading from Spark SQL 2.4 to 3.0

### Dataset/DataFrame APIs
Expand Down
5 changes: 4 additions & 1 deletion python/pyspark/sql/dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -534,7 +534,10 @@ def withWatermark(self, eventTime, delayThreshold):

.. note:: Evolving

>>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes')
>>> from pyspark.sql.functions import timestamp_seconds
>>> sdf.select(
... 'name',
... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes')
DataFrame[name: string, time: timestamp]
"""
if not eventTime or type(eventTime) is not str:
Expand Down
13 changes: 13 additions & 0 deletions python/pyspark/sql/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -1427,6 +1427,19 @@ def to_utc_timestamp(timestamp, tz):
return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz))


@since(3.1)
def timestamp_seconds(col):
"""
>>> from pyspark.sql.functions import timestamp_seconds
Copy link
Member

Choose a reason for hiding this comment

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

Can you set the session timezone? It caused SPARK-32088

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks HyukjinKwon, will fix soon

>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect()
[Row(ts=datetime.datetime(2008, 12, 25, 7, 30))]
"""

sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.timestamp_seconds(_to_java_column(col)))


@since(2.0)
@ignore_unicode_prefix
def window(timeColumn, windowDuration, slideDuration=None, startTime=None):
Expand Down
2 changes: 1 addition & 1 deletion python/pyspark/sql/tests/test_dataframe.py
Original file line number Diff line number Diff line change
Expand Up @@ -644,7 +644,7 @@ def test_to_pandas_from_mixed_dataframe(self):
CAST(col6 AS DOUBLE) AS double,
CAST(col7 AS BOOLEAN) AS boolean,
CAST(col8 AS STRING) AS string,
CAST(col9 AS TIMESTAMP) AS timestamp
timestamp_seconds(col9) AS timestamp
FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1),
(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
"""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,8 @@ object Cast {
case (StringType, TimestampType) => true
case (BooleanType, TimestampType) => true
case (DateType, TimestampType) => true
case (_: NumericType, TimestampType) => true
case (_: NumericType, TimestampType) =>
SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP)

case (StringType, DateType) => true
case (TimestampType, DateType) => true
Expand Down Expand Up @@ -266,7 +267,15 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
TypeCheckResult.TypeCheckSuccess
} else {
TypeCheckResult.TypeCheckFailure(
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}")
if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) {
Copy link
Member

Choose a reason for hiding this comment

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

btw, we need the check here? I think its okay just to update our migration guide.

Copy link
Contributor

Choose a reason for hiding this comment

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

it's better to let users know why it's wrong and how to fix, in the error message.

s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," +
"you can enable the casting by setting " +
s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," +
"but we strongly recommend using function " +
"TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead."
} else {
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}"
})
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1549,7 +1549,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr

def this(left: Expression, format: Expression) {
this(left, Option(format),
Cast(Cast(UnixTimestamp(left, format), TimestampType), DateType))
Cast(SecondsToTimestamp(UnixTimestamp(left, format)), DateType))
}

def this(left: Expression) = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2595,6 +2595,15 @@ object SQLConf {
.checkValue(_ > 0, "The timeout value must be positive")
.createWithDefault(10L)

val LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP =
buildConf("spark.sql.legacy.allowCastNumericToTimestamp")
.internal()
.doc("When true, allow casting numeric to timestamp," +
"when false, forbid the cast, more details in SPARK-31710")
.version("3.1.0")
.booleanConf
.createWithDefault(false)

/**
* Holds information about keys that have been deprecated.
*
Expand Down Expand Up @@ -3181,6 +3190,9 @@ class SQLConf extends Serializable with Logging {

def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID)

def legacyAllowCastNumericToTimestamp: Boolean =
getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP)

/** ********************** SQLConf functionality methods ************ */

/** Set Spark SQL configuration properties. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
}

protected def checkNullCast(from: DataType, to: DataType): Unit = {
checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null)
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null)
}
}

test("null cast") {
Expand Down Expand Up @@ -239,7 +241,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
checkCast(1.5, 1.5f)
checkCast(1.5, "1.5")

checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble)
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble)
}
}

test("cast from string") {
Expand Down Expand Up @@ -305,17 +309,20 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(cast(cast(cast(cast(
cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType),
5.toLong)
checkEvaluation(
cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType),
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
5.toShort)
checkEvaluation(
cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType),
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
null)
checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT),
ByteType), TimestampType), LongType), StringType), ShortType),
5.toShort)

withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
checkEvaluation(
cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType),
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
5.toShort)
checkEvaluation(
cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType),
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
null)
checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT),
ByteType), TimestampType), LongType), StringType), ShortType),
5.toShort)
}

checkEvaluation(cast("23", DoubleType), 23d)
checkEvaluation(cast("23", IntegerType), 23)
Expand Down Expand Up @@ -376,29 +383,32 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(cast(ts, LongType), 15.toLong)
checkEvaluation(cast(ts, FloatType), 15.003f)
checkEvaluation(cast(ts, DoubleType), 15.003)
checkEvaluation(cast(cast(tss, ShortType), TimestampType),
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
checkEvaluation(cast(cast(tss, IntegerType), TimestampType),
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
checkEvaluation(cast(cast(tss, LongType), TimestampType),
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
checkEvaluation(
cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType),
millis.toFloat / MILLIS_PER_SECOND)
checkEvaluation(
cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType),
millis.toDouble / MILLIS_PER_SECOND)
checkEvaluation(
cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT),
Decimal(1))

// A test for higher precision than millis
checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001)
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
checkEvaluation(cast(cast(tss, ShortType), TimestampType),
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
checkEvaluation(cast(cast(tss, IntegerType), TimestampType),
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
checkEvaluation(cast(cast(tss, LongType), TimestampType),
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
checkEvaluation(
cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType),
millis.toFloat / MILLIS_PER_SECOND)
checkEvaluation(
cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType),
millis.toDouble / MILLIS_PER_SECOND)
checkEvaluation(
cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT),
Decimal(1))

checkEvaluation(cast(Double.NaN, TimestampType), null)
checkEvaluation(cast(1.0 / 0.0, TimestampType), null)
checkEvaluation(cast(Float.NaN, TimestampType), null)
checkEvaluation(cast(1.0f / 0.0f, TimestampType), null)
// A test for higher precision than millis
checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001)

checkEvaluation(cast(Double.NaN, TimestampType), null)
checkEvaluation(cast(1.0 / 0.0, TimestampType), null)
checkEvaluation(cast(Float.NaN, TimestampType), null)
checkEvaluation(cast(1.0f / 0.0f, TimestampType), null)
}
}

test("cast from array") {
Expand Down Expand Up @@ -1026,8 +1036,11 @@ class CastSuite extends CastSuiteBase {

test("cast from int 2") {
checkEvaluation(cast(1, LongType), 1.toLong)
checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong)
checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong)

withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong)
checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong)
}

checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123))
checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123))
Expand Down Expand Up @@ -1310,6 +1323,20 @@ class CastSuite extends CastSuiteBase {
checkEvaluation(cast(negativeTs, LongType), expectedSecs)
}
}

test("SPARK-31710:fail casting from numeric to timestamp by default") {
Seq(true, false).foreach { enable =>
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) {
assert(cast(2.toByte, TimestampType).resolved == enable)
assert(cast(10.toShort, TimestampType).resolved == enable)
assert(cast(3, TimestampType).resolved == enable)
assert(cast(10L, TimestampType).resolved == enable)
assert(cast(Decimal(1.2), TimestampType).resolved == enable)
assert(cast(1.7f, TimestampType).resolved == enable)
assert(cast(2.3d, TimestampType).resolved == enable)
}
}
}
}

/**
Expand Down
9 changes: 9 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/functions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3358,6 +3358,15 @@ object functions {
window(timeColumn, windowDuration, windowDuration, "0 second")
}

/**
* Creates timestamp from the number of seconds since UTC epoch.
* @group = datetime_funcs
* @since = 3.1.0
Copy link
Member

Choose a reason for hiding this comment

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

nit:

@group datetime_funcs
@since 3.1.0

*/
def timestamp_seconds(e: Column): Column = withExpr {
SecondsToTimestamp(e.expr)
}

//////////////////////////////////////////////////////////////////////////////////////////////
// Collection functions
//////////////////////////////////////////////////////////////////////////////////////////////
Expand Down
16 changes: 8 additions & 8 deletions sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql
Original file line number Diff line number Diff line change
@@ -1,15 +1,15 @@
--This test file was converted from window.sql.
-- Test data.
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"),
(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"),
(null, null, null, null, null, null),
(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null)
(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null)
AS testData(val, val_long, val_double, val_date, val_timestamp, cate);

-- RowsBetween
Expand Down
16 changes: 8 additions & 8 deletions sql/core/src/test/resources/sql-tests/inputs/window.sql
Original file line number Diff line number Diff line change
Expand Up @@ -5,15 +5,15 @@

-- Test data.
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"),
(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"),
(null, null, null, null, null, null),
(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null)
(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null)
AS testData(val, val_long, val_double, val_date, val_timestamp, cate);

-- RowsBetween
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,15 +4,15 @@

-- !query
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"),
(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"),
(null, null, null, null, null, null),
(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null)
(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null)
AS testData(val, val_long, val_double, val_date, val_timestamp, cate)
-- !query schema
struct<>
Expand Down
16 changes: 8 additions & 8 deletions sql/core/src/test/resources/sql-tests/results/window.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -4,15 +4,15 @@

-- !query
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"),
(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"),
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"),
(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"),
(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"),
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"),
(null, null, null, null, null, null),
(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null)
(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null)
AS testData(val, val_long, val_double, val_date, val_timestamp, cate)
-- !query schema
struct<>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -639,7 +639,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {

val now = sql("select unix_timestamp()").collect().head.getLong(0)
checkAnswer(
sql(s"select cast ($now as timestamp)"),
sql(s"select timestamp_seconds($now)"),
Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now))))
}
}
Expand Down Expand Up @@ -716,7 +716,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss")

checkAnswer(df.select(to_timestamp(col("ss"))),
df.select(unix_timestamp(col("ss")).cast("timestamp")))
df.select(timestamp_seconds(unix_timestamp(col("ss")))))
checkAnswer(df.select(to_timestamp(col("ss"))), Seq(
Row(ts1), Row(ts2)))
if (legacyParserPolicy == "legacy") {
Expand Down
Loading