Skip to content

Commit f0e6d0e

Browse files
GuoPhilipsecloud-fan
authored andcommitted
[SPARK-31710][SQL] Fail casting numeric to timestamp by default
## What changes were proposed in this pull request? we fail casting from numeric to timestamp by default. ## Why are the changes needed? casting from numeric to timestamp is not a non-standard,meanwhile it may generate different result between spark and other systems,for example hive ## Does this PR introduce any user-facing change? Yes,user cannot cast numeric to timestamp directly,user have to use the following function to achieve the same effect:TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS ## How was this patch tested? unit test added Closes #28593 from GuoPhilipse/31710-fix-compatibility. Lead-authored-by: GuoPhilipse <guofei_ok@126.com> Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent fe68e95 commit f0e6d0e

File tree

30 files changed

+233
-150
lines changed

30 files changed

+233
-150
lines changed

docs/sql-migration-guide.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,8 @@ license: |
3030

3131
- 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`.
3232

33+
- 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.
34+
3335
## Upgrading from Spark SQL 2.4 to 3.0
3436

3537
### Dataset/DataFrame APIs

python/pyspark/sql/dataframe.py

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -534,7 +534,10 @@ def withWatermark(self, eventTime, delayThreshold):
534534
535535
.. note:: Evolving
536536
537-
>>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes')
537+
>>> from pyspark.sql.functions import timestamp_seconds
538+
>>> sdf.select(
539+
... 'name',
540+
... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes')
538541
DataFrame[name: string, time: timestamp]
539542
"""
540543
if not eventTime or type(eventTime) is not str:

python/pyspark/sql/functions.py

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1427,6 +1427,19 @@ def to_utc_timestamp(timestamp, tz):
14271427
return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz))
14281428

14291429

1430+
@since(3.1)
1431+
def timestamp_seconds(col):
1432+
"""
1433+
>>> from pyspark.sql.functions import timestamp_seconds
1434+
>>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time'])
1435+
>>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect()
1436+
[Row(ts=datetime.datetime(2008, 12, 25, 7, 30))]
1437+
"""
1438+
1439+
sc = SparkContext._active_spark_context
1440+
return Column(sc._jvm.functions.timestamp_seconds(_to_java_column(col)))
1441+
1442+
14301443
@since(2.0)
14311444
@ignore_unicode_prefix
14321445
def window(timeColumn, windowDuration, slideDuration=None, startTime=None):

python/pyspark/sql/tests/test_dataframe.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -644,7 +644,7 @@ def test_to_pandas_from_mixed_dataframe(self):
644644
CAST(col6 AS DOUBLE) AS double,
645645
CAST(col7 AS BOOLEAN) AS boolean,
646646
CAST(col8 AS STRING) AS string,
647-
CAST(col9 AS TIMESTAMP) AS timestamp
647+
timestamp_seconds(col9) AS timestamp
648648
FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1),
649649
(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
650650
"""

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,8 @@ object Cast {
5959
case (StringType, TimestampType) => true
6060
case (BooleanType, TimestampType) => true
6161
case (DateType, TimestampType) => true
62-
case (_: NumericType, TimestampType) => true
62+
case (_: NumericType, TimestampType) =>
63+
SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP)
6364

6465
case (StringType, DateType) => true
6566
case (TimestampType, DateType) => true
@@ -266,7 +267,15 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
266267
TypeCheckResult.TypeCheckSuccess
267268
} else {
268269
TypeCheckResult.TypeCheckFailure(
269-
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}")
270+
if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) {
271+
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," +
272+
"you can enable the casting by setting " +
273+
s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," +
274+
"but we strongly recommend using function " +
275+
"TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead."
276+
} else {
277+
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}"
278+
})
270279
}
271280
}
272281

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1549,7 +1549,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr
15491549

15501550
def this(left: Expression, format: Expression) {
15511551
this(left, Option(format),
1552-
Cast(Cast(UnixTimestamp(left, format), TimestampType), DateType))
1552+
Cast(SecondsToTimestamp(UnixTimestamp(left, format)), DateType))
15531553
}
15541554

15551555
def this(left: Expression) = {

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2608,6 +2608,15 @@ object SQLConf {
26082608
.checkValue(_ > 0, "The timeout value must be positive")
26092609
.createWithDefault(10L)
26102610

2611+
val LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP =
2612+
buildConf("spark.sql.legacy.allowCastNumericToTimestamp")
2613+
.internal()
2614+
.doc("When true, allow casting numeric to timestamp," +
2615+
"when false, forbid the cast, more details in SPARK-31710")
2616+
.version("3.1.0")
2617+
.booleanConf
2618+
.createWithDefault(false)
2619+
26112620
/**
26122621
* Holds information about keys that have been deprecated.
26132622
*
@@ -3196,6 +3205,9 @@ class SQLConf extends Serializable with Logging {
31963205

31973206
def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID)
31983207

3208+
def legacyAllowCastNumericToTimestamp: Boolean =
3209+
getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP)
3210+
31993211
/** ********************** SQLConf functionality methods ************ */
32003212

32013213
/** Set Spark SQL configuration properties. */

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

Lines changed: 63 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
5050
}
5151

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

5658
test("null cast") {
@@ -239,7 +241,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
239241
checkCast(1.5, 1.5f)
240242
checkCast(1.5, "1.5")
241243

242-
checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble)
244+
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
245+
checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble)
246+
}
243247
}
244248

245249
test("cast from string") {
@@ -305,17 +309,20 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
305309
checkEvaluation(cast(cast(cast(cast(
306310
cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType),
307311
5.toLong)
308-
checkEvaluation(
309-
cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType),
310-
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
311-
5.toShort)
312-
checkEvaluation(
313-
cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType),
314-
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
315-
null)
316-
checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT),
317-
ByteType), TimestampType), LongType), StringType), ShortType),
318-
5.toShort)
312+
313+
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
314+
checkEvaluation(
315+
cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType),
316+
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
317+
5.toShort)
318+
checkEvaluation(
319+
cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType),
320+
DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType),
321+
null)
322+
checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT),
323+
ByteType), TimestampType), LongType), StringType), ShortType),
324+
5.toShort)
325+
}
319326

320327
checkEvaluation(cast("23", DoubleType), 23d)
321328
checkEvaluation(cast("23", IntegerType), 23)
@@ -376,29 +383,32 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
376383
checkEvaluation(cast(ts, LongType), 15.toLong)
377384
checkEvaluation(cast(ts, FloatType), 15.003f)
378385
checkEvaluation(cast(ts, DoubleType), 15.003)
379-
checkEvaluation(cast(cast(tss, ShortType), TimestampType),
380-
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
381-
checkEvaluation(cast(cast(tss, IntegerType), TimestampType),
382-
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
383-
checkEvaluation(cast(cast(tss, LongType), TimestampType),
384-
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
385-
checkEvaluation(
386-
cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType),
387-
millis.toFloat / MILLIS_PER_SECOND)
388-
checkEvaluation(
389-
cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType),
390-
millis.toDouble / MILLIS_PER_SECOND)
391-
checkEvaluation(
392-
cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT),
393-
Decimal(1))
394386

395-
// A test for higher precision than millis
396-
checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001)
387+
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
388+
checkEvaluation(cast(cast(tss, ShortType), TimestampType),
389+
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
390+
checkEvaluation(cast(cast(tss, IntegerType), TimestampType),
391+
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
392+
checkEvaluation(cast(cast(tss, LongType), TimestampType),
393+
DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND)
394+
checkEvaluation(
395+
cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType),
396+
millis.toFloat / MILLIS_PER_SECOND)
397+
checkEvaluation(
398+
cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType),
399+
millis.toDouble / MILLIS_PER_SECOND)
400+
checkEvaluation(
401+
cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT),
402+
Decimal(1))
397403

398-
checkEvaluation(cast(Double.NaN, TimestampType), null)
399-
checkEvaluation(cast(1.0 / 0.0, TimestampType), null)
400-
checkEvaluation(cast(Float.NaN, TimestampType), null)
401-
checkEvaluation(cast(1.0f / 0.0f, TimestampType), null)
404+
// A test for higher precision than millis
405+
checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001)
406+
407+
checkEvaluation(cast(Double.NaN, TimestampType), null)
408+
checkEvaluation(cast(1.0 / 0.0, TimestampType), null)
409+
checkEvaluation(cast(Float.NaN, TimestampType), null)
410+
checkEvaluation(cast(1.0f / 0.0f, TimestampType), null)
411+
}
402412
}
403413

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

10271037
test("cast from int 2") {
10281038
checkEvaluation(cast(1, LongType), 1.toLong)
1029-
checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong)
1030-
checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong)
1039+
1040+
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") {
1041+
checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong)
1042+
checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong)
1043+
}
10311044

10321045
checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123))
10331046
checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123))
@@ -1310,6 +1323,20 @@ class CastSuite extends CastSuiteBase {
13101323
checkEvaluation(cast(negativeTs, LongType), expectedSecs)
13111324
}
13121325
}
1326+
1327+
test("SPARK-31710:fail casting from numeric to timestamp by default") {
1328+
Seq(true, false).foreach { enable =>
1329+
withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) {
1330+
assert(cast(2.toByte, TimestampType).resolved == enable)
1331+
assert(cast(10.toShort, TimestampType).resolved == enable)
1332+
assert(cast(3, TimestampType).resolved == enable)
1333+
assert(cast(10L, TimestampType).resolved == enable)
1334+
assert(cast(Decimal(1.2), TimestampType).resolved == enable)
1335+
assert(cast(1.7f, TimestampType).resolved == enable)
1336+
assert(cast(2.3d, TimestampType).resolved == enable)
1337+
}
1338+
}
1339+
}
13131340
}
13141341

13151342
/**

sql/core/src/main/scala/org/apache/spark/sql/functions.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3358,6 +3358,15 @@ object functions {
33583358
window(timeColumn, windowDuration, windowDuration, "0 second")
33593359
}
33603360

3361+
/**
3362+
* Creates timestamp from the number of seconds since UTC epoch.
3363+
* @group = datetime_funcs
3364+
* @since = 3.1.0
3365+
*/
3366+
def timestamp_seconds(e: Column): Column = withExpr {
3367+
SecondsToTimestamp(e.expr)
3368+
}
3369+
33613370
//////////////////////////////////////////////////////////////////////////////////////////////
33623371
// Collection functions
33633372
//////////////////////////////////////////////////////////////////////////////////////////////

sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1,15 +1,15 @@
11
--This test file was converted from window.sql.
22
-- Test data.
33
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
4-
(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
5-
(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"),
6-
(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"),
7-
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"),
8-
(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"),
9-
(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"),
10-
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"),
4+
(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
5+
(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"),
6+
(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"),
7+
(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"),
8+
(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"),
9+
(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"),
10+
(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"),
1111
(null, null, null, null, null, null),
12-
(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null)
12+
(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null)
1313
AS testData(val, val_long, val_double, val_date, val_timestamp, cate);
1414

1515
-- RowsBetween

0 commit comments

Comments
 (0)