Skip to content

Commit

Permalink
fail casting from integral to timestamp
Browse files Browse the repository at this point in the history
  • Loading branch information
GuoPhilipse committed May 22, 2020
1 parent 1459d5b commit 8d1deee
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ object Cast {
case (StringType, TimestampType) => true
case (BooleanType, TimestampType) => true
case (DateType, TimestampType) => true
case (_: NumericType, TimestampType) => true
case (_: FractionalType, TimestampType) => true

case (StringType, DateType) => true
case (TimestampType, DateType) => true
Expand Down Expand Up @@ -138,10 +138,8 @@ object Cast {
case (_: CalendarIntervalType, StringType) => true
case (NullType, _) => true

// Spark supports casting between long and timestamp, please see `longToTimestamp` and
// `timestampToLong` for details.
// spark forbid casting from integral to timestamp, more details in [SPARK-31790]
case (TimestampType, LongType) => true
case (LongType, TimestampType) => true

case (ArrayType(fromType, fn), ArrayType(toType, tn)) =>
resolvableNullability(fn, tn) && canUpCast(fromType, toType)
Expand Down Expand Up @@ -266,7 +264,12 @@ 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[IntegralType] && dataType.isInstanceOf[TimestampType]) {
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," +
s"please use function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instand"
} else {
s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}"
})
}
}

Expand Down Expand Up @@ -425,14 +428,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs, zoneId).orNull)
case BooleanType =>
buildCast[Boolean](_, b => if (b) 1L else 0)
case LongType =>
buildCast[Long](_, l => longToTimestamp(l))
case IntegerType =>
buildCast[Int](_, i => longToTimestamp(i.toLong))
case ShortType =>
buildCast[Short](_, s => longToTimestamp(s.toLong))
case ByteType =>
buildCast[Byte](_, b => longToTimestamp(b.toLong))
case DateType =>
buildCast[Int](_, d => epochDaysToMicros(d, zoneId))
// TimestampWritable.decimalToTimestamp
Expand All @@ -453,8 +448,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong
}

// converting seconds to us
private[this] def longToTimestamp(t: Long): Long = SECONDS.toMicros(t)
// converting us to seconds
private[this] def timestampToLong(ts: Long): Long = {
Math.floorDiv(ts, MICROS_PER_SECOND)
Expand Down Expand Up @@ -1229,8 +1222,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
"""
case BooleanType =>
(c, evPrim, evNull) => code"$evPrim = $c ? 1L : 0L;"
case _: IntegralType =>
(c, evPrim, evNull) => code"$evPrim = ${longToTimeStampCode(c)};"
case DateType =>
val zoneIdClass = classOf[ZoneId]
val zid = JavaCode.global(
Expand Down Expand Up @@ -1277,7 +1268,6 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
val block = inline"new java.math.BigDecimal($MICROS_PER_SECOND)"
code"($d.toBigDecimal().bigDecimal().multiply($block)).longValue()"
}
private[this] def longToTimeStampCode(l: ExprValue): Block = code"$l * (long)$MICROS_PER_SECOND"
private[this] def timestampToLongCode(ts: ExprValue): Block =
code"java.lang.Math.floorDiv($ts, $MICROS_PER_SECOND)"
private[this] def timestampToDoubleCode(ts: ExprValue): Block =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1311,6 +1311,13 @@ class CastSuite extends CastSuiteBase {
checkEvaluation(cast(negativeTs, LongType), expectedSecs)
}
}

test("SPARK-31710:fail casting from integral to timestamp by default") {
assert(!cast(2.toByte, TimestampType).resolved)
assert(!cast(10.toShort, TimestampType).resolved)
assert(!cast(3, TimestampType).resolved)
assert(!cast(10L, TimestampType).resolved)
}
}

/**
Expand Down

0 comments on commit 8d1deee

Please sign in to comment.