Skip to content
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ object Cast {

case (StringType, DateType) => true
case (TimestampType, DateType) => true
case (_: IntegralType, DateType) => true

case (StringType, CalendarIntervalType) => true

Expand Down Expand Up @@ -474,6 +475,10 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String
// throw valid precision more than seconds, according to Hive.
// Timestamp.nanos is in 0 to 999,999,999, no more than a second.
buildCast[Long](_, t => microsToEpochDays(t, zoneId))
case LongType => buildCast[Long](_, l => l.toInt)
case IntegerType => buildCast[Int](_, identity)
case ShortType => buildCast[Short](_, s => s.toInt)
case ByteType => buildCast[Byte](_, b => b.toInt)
}

// IntervalConverter
Expand Down Expand Up @@ -1076,6 +1081,8 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String
(c, evPrim, evNull) =>
code"""$evPrim =
org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);"""
case _: IntegralType =>
(c, evPrim, _) => code"$evPrim = (int)$c;"
case _ =>
(c, evPrim, evNull) => code"$evNull = true;"
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -855,10 +855,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
assert(cast("1234", DateType).checkInputDataTypes().isSuccess)
assert(cast(new Timestamp(1), DateType).checkInputDataTypes().isSuccess)
assert(cast(false, DateType).checkInputDataTypes().isFailure)
assert(cast(1.toByte, DateType).checkInputDataTypes().isFailure)
assert(cast(1.toShort, DateType).checkInputDataTypes().isFailure)
assert(cast(1, DateType).checkInputDataTypes().isFailure)
assert(cast(1L, DateType).checkInputDataTypes().isFailure)
assert(cast(1.0.toFloat, DateType).checkInputDataTypes().isFailure)
assert(cast(1.0, DateType).checkInputDataTypes().isFailure)
}
Expand Down Expand Up @@ -1200,4 +1196,11 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(cast(Long.MinValue - 0.9D, LongType), Long.MinValue)
}
}

test("cast integral types to date") {
checkCast(0.toByte, Date.valueOf("1970-01-01"))
checkCast(-1.toShort, Date.valueOf("1969-12-31"))
checkCast(1, Date.valueOf("1970-01-02"))
checkCast(365, Date.valueOf("1971-01-01"))
}
}
Loading