[SPARK-35734][SQL][FOLLOWUP] IntervalUtils.toDayTimeIntervalString should consider the case a day-time type is casted as another day-time type
### What changes were proposed in this pull request? This PR fixes an issue that `IntervalUtils.toDayTimeIntervalString` doesn't consider the case that a day-time interval type is casted as another day-time interval type. if data of `interval day to second` is casted as `interval hour to second`, the value of the day is multiplied by 24 and added to the value of hour. For example, `INTERVAL '1 2' DAY TO HOUR` will be `INTERVAL '26' HOUR` if it's casted. If this behavior is intended, it should be stringified as `INTERVAL '26' HOUR` but currently, it will be `INTERVAL '2' HOUR` ### Why are the changes needed? t's a bug if the behavior of cast is intended. ### Does this PR introduce _any_ user-facing change? No, because this feature is not released yet. ### How was this patch tested? Modified the tests added in SPARK-35734 (#32891) Closes #33031 from sarutak/fix-toDayTimeIntervalString. Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com> Signed-off-by: Max Gekk <max.gekk@gmail.com>
This commit is contained in:
parent
960a7e5fce
commit
4416b4b8ba
|
@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros
|
|||
import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, HIVE_STYLE, IntervalStyle}
|
||||
import org.apache.spark.sql.errors.QueryExecutionErrors
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.{DayTimeIntervalType, Decimal, YearMonthIntervalType}
|
||||
import org.apache.spark.sql.types.{DayTimeIntervalType => DT, Decimal, YearMonthIntervalType => YM}
|
||||
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
|
||||
|
||||
// The style of textual representation of intervals
|
||||
|
@ -892,7 +892,7 @@ object IntervalUtils {
|
|||
* @throws ArithmeticException If numeric overflow occurs
|
||||
*/
|
||||
def durationToMicros(duration: Duration): Long = {
|
||||
durationToMicros(duration, DayTimeIntervalType.SECOND)
|
||||
durationToMicros(duration, DT.SECOND)
|
||||
}
|
||||
|
||||
def durationToMicros(duration: Duration, endField: Byte): Long = {
|
||||
|
@ -910,10 +910,10 @@ object IntervalUtils {
|
|||
}
|
||||
|
||||
endField match {
|
||||
case DayTimeIntervalType.DAY => micros - micros % MICROS_PER_DAY
|
||||
case DayTimeIntervalType.HOUR => micros - micros % MICROS_PER_HOUR
|
||||
case DayTimeIntervalType.MINUTE => micros - micros % MICROS_PER_MINUTE
|
||||
case DayTimeIntervalType.SECOND => micros
|
||||
case DT.DAY => micros - micros % MICROS_PER_DAY
|
||||
case DT.HOUR => micros - micros % MICROS_PER_HOUR
|
||||
case DT.MINUTE => micros - micros % MICROS_PER_MINUTE
|
||||
case DT.SECOND => micros
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -936,13 +936,13 @@ object IntervalUtils {
|
|||
* @throws ArithmeticException If numeric overflow occurs
|
||||
*/
|
||||
def periodToMonths(period: Period): Int = {
|
||||
periodToMonths(period, YearMonthIntervalType.MONTH)
|
||||
periodToMonths(period, YM.MONTH)
|
||||
}
|
||||
|
||||
def periodToMonths(period: Period, endField: Byte): Int = {
|
||||
val monthsInYears = Math.multiplyExact(period.getYears, MONTHS_PER_YEAR)
|
||||
val months = Math.addExact(monthsInYears, period.getMonths)
|
||||
if (endField == YearMonthIntervalType.YEAR) {
|
||||
if (endField == YM.YEAR) {
|
||||
months - months % MONTHS_PER_YEAR
|
||||
} else {
|
||||
months
|
||||
|
@ -993,8 +993,8 @@ object IntervalUtils {
|
|||
val formatBuilder = new StringBuilder("INTERVAL '")
|
||||
if (startField == endField) {
|
||||
startField match {
|
||||
case YearMonthIntervalType.YEAR => formatBuilder.append(s"$year' YEAR")
|
||||
case YearMonthIntervalType.MONTH => formatBuilder.append(s"$months' MONTH")
|
||||
case YM.YEAR => formatBuilder.append(s"$year' YEAR")
|
||||
case YM.MONTH => formatBuilder.append(s"$months' MONTH")
|
||||
}
|
||||
} else {
|
||||
formatBuilder.append(s"$yearAndMonth' YEAR TO MONTH")
|
||||
|
@ -1021,31 +1021,43 @@ object IntervalUtils {
|
|||
endField: Byte): String = {
|
||||
var sign = ""
|
||||
var rest = micros
|
||||
val from = DayTimeIntervalType.fieldToString(startField).toUpperCase
|
||||
val to = DayTimeIntervalType.fieldToString(endField).toUpperCase
|
||||
val from = DT.fieldToString(startField).toUpperCase
|
||||
val to = DT.fieldToString(endField).toUpperCase
|
||||
val prefix = "INTERVAL '"
|
||||
val postfix = s"' ${if (startField == endField) from else s"$from TO $to"}"
|
||||
|
||||
if (micros < 0) {
|
||||
if (micros == Long.MinValue) {
|
||||
// Especial handling of minimum `Long` value because negate op overflows `Long`.
|
||||
// seconds = 106751991 * (24 * 60 * 60) + 4 * 60 * 60 + 54 = 9223372036854
|
||||
// microseconds = -9223372036854000000L-775808 == Long.MinValue
|
||||
val baseStr = "-106751991 04:00:54.775808000"
|
||||
val minIntervalString = style match {
|
||||
case ANSI_STYLE =>
|
||||
val baseStr = "-106751991 04:00:54.775808"
|
||||
val fromPos = startField match {
|
||||
case DayTimeIntervalType.DAY => 0
|
||||
case DayTimeIntervalType.HOUR => 11
|
||||
case DayTimeIntervalType.MINUTE => 14
|
||||
case DayTimeIntervalType.SECOND => 17
|
||||
val firstStr = startField match {
|
||||
case DT.DAY => "-106751991"
|
||||
case DT.HOUR => "-2562047788"
|
||||
case DT.MINUTE => "-153722867280"
|
||||
case DT.SECOND => "-9223372036854.775808"
|
||||
}
|
||||
val toPos = endField match {
|
||||
case DayTimeIntervalType.DAY => 10
|
||||
case DayTimeIntervalType.HOUR => 13
|
||||
case DayTimeIntervalType.MINUTE => 16
|
||||
case DayTimeIntervalType.SECOND => baseStr.length
|
||||
val followingStr = if (startField == endField) {
|
||||
""
|
||||
} else {
|
||||
val substrStart = startField match {
|
||||
case DT.DAY => 10
|
||||
case DT.HOUR => 13
|
||||
case DT.MINUTE => 16
|
||||
}
|
||||
val substrEnd = endField match {
|
||||
case DT.HOUR => 13
|
||||
case DT.MINUTE => 16
|
||||
case DT.SECOND => 26
|
||||
}
|
||||
baseStr.substring(substrStart, substrEnd)
|
||||
}
|
||||
val postfix = if (startField == endField) from else s"$from TO $to"
|
||||
s"INTERVAL '${baseStr.substring(fromPos, toPos)}' $postfix"
|
||||
case HIVE_STYLE => "-106751991 04:00:54.775808000"
|
||||
|
||||
s"$prefix$firstStr$followingStr$postfix"
|
||||
case HIVE_STYLE => baseStr
|
||||
}
|
||||
return minIntervalString
|
||||
} else {
|
||||
|
@ -1053,53 +1065,51 @@ object IntervalUtils {
|
|||
rest = -rest
|
||||
}
|
||||
}
|
||||
val secondsWithFraction = rest % MICROS_PER_MINUTE
|
||||
rest /= MICROS_PER_MINUTE
|
||||
val minutes = rest % MINUTES_PER_HOUR
|
||||
rest /= MINUTES_PER_HOUR
|
||||
val hours = rest % HOURS_PER_DAY
|
||||
val days = rest / HOURS_PER_DAY
|
||||
val leadSecZero = if (secondsWithFraction < 10 * MICROS_PER_SECOND) "0" else ""
|
||||
val intervalString = style match {
|
||||
case ANSI_STYLE =>
|
||||
val secStr = java.math.BigDecimal.valueOf(secondsWithFraction, 6)
|
||||
.stripTrailingZeros()
|
||||
.toPlainString()
|
||||
val formatBuilder = new StringBuilder("INTERVAL '")
|
||||
if (startField == endField) {
|
||||
startField match {
|
||||
case DayTimeIntervalType.DAY => formatBuilder.append(s"$sign$days' ")
|
||||
case DayTimeIntervalType.HOUR => formatBuilder.append(f"$hours%02d' ")
|
||||
case DayTimeIntervalType.MINUTE => formatBuilder.append(f"$minutes%02d' ")
|
||||
case DayTimeIntervalType.SECOND => formatBuilder.append(s"$leadSecZero$secStr' ")
|
||||
}
|
||||
formatBuilder.append(from).toString
|
||||
} else {
|
||||
val formatArgs = new mutable.ArrayBuffer[Long]
|
||||
if (startField <= DayTimeIntervalType.DAY && DayTimeIntervalType.DAY < endField) {
|
||||
formatBuilder.append(s"$sign$days ")
|
||||
}
|
||||
if (startField <= DayTimeIntervalType.HOUR && DayTimeIntervalType.HOUR < endField) {
|
||||
formatBuilder.append("%02d:")
|
||||
formatArgs.append(hours)
|
||||
}
|
||||
if (startField <= DayTimeIntervalType.MINUTE && DayTimeIntervalType.MINUTE < endField) {
|
||||
formatBuilder.append("%02d:")
|
||||
formatArgs.append(minutes)
|
||||
}
|
||||
endField match {
|
||||
case DayTimeIntervalType.HOUR =>
|
||||
formatBuilder.append("%02d' ")
|
||||
formatArgs.append(hours)
|
||||
case DayTimeIntervalType.MINUTE =>
|
||||
formatBuilder.append("%02d' ")
|
||||
formatArgs.append(minutes)
|
||||
case DayTimeIntervalType.SECOND =>
|
||||
formatBuilder.append(s"$leadSecZero$secStr' ")
|
||||
}
|
||||
formatBuilder.append(s"$from TO $to").toString.format(formatArgs.toSeq: _*)
|
||||
val formatBuilder = new mutable.StringBuilder(sign)
|
||||
val formatArgs = new mutable.ArrayBuffer[Long]()
|
||||
startField match {
|
||||
case DT.DAY =>
|
||||
formatBuilder.append(rest / MICROS_PER_DAY)
|
||||
rest %= MICROS_PER_DAY
|
||||
case DT.HOUR =>
|
||||
formatBuilder.append("%02d")
|
||||
formatArgs.append(rest / MICROS_PER_HOUR)
|
||||
rest %= MICROS_PER_HOUR
|
||||
case DT.MINUTE =>
|
||||
formatBuilder.append("%02d")
|
||||
formatArgs.append(rest / MICROS_PER_MINUTE)
|
||||
rest %= MICROS_PER_MINUTE
|
||||
case DT.SECOND =>
|
||||
val leadZero = if (rest < 10 * MICROS_PER_SECOND) "0" else ""
|
||||
formatBuilder.append(s"$leadZero" +
|
||||
s"${java.math.BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString}")
|
||||
}
|
||||
|
||||
if (startField < DT.HOUR && DT.HOUR <= endField) {
|
||||
formatBuilder.append(" %02d")
|
||||
formatArgs.append(rest / MICROS_PER_HOUR)
|
||||
rest %= MICROS_PER_HOUR
|
||||
}
|
||||
if (startField < DT.MINUTE && DT.MINUTE <= endField) {
|
||||
formatBuilder.append(":%02d")
|
||||
formatArgs.append(rest / MICROS_PER_MINUTE)
|
||||
rest %= MICROS_PER_MINUTE
|
||||
}
|
||||
if (startField < DT.SECOND && DT.SECOND <= endField) {
|
||||
val leadZero = if (rest < 10 * MICROS_PER_SECOND) "0" else ""
|
||||
formatBuilder.append(
|
||||
s":$leadZero${java.math.BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString}")
|
||||
}
|
||||
s"$prefix${formatBuilder.toString.format(formatArgs.toSeq: _*)}$postfix"
|
||||
case HIVE_STYLE =>
|
||||
val secondsWithFraction = rest % MICROS_PER_MINUTE
|
||||
rest /= MICROS_PER_MINUTE
|
||||
val minutes = rest % MINUTES_PER_HOUR
|
||||
rest /= MINUTES_PER_HOUR
|
||||
val hours = rest % HOURS_PER_DAY
|
||||
val days = rest / HOURS_PER_DAY
|
||||
val seconds = secondsWithFraction / MICROS_PER_SECOND
|
||||
val nanos = (secondsWithFraction % MICROS_PER_SECOND) * NANOS_PER_MICROS
|
||||
f"$sign$days $hours%02d:$minutes%02d:$seconds%02d.$nanos%09d"
|
||||
|
|
|
@ -554,13 +554,13 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper {
|
|||
("INTERVAL '-0 00:00:00.000001' DAY TO SECOND",
|
||||
"INTERVAL '-0 00:00' DAY TO MINUTE",
|
||||
"INTERVAL '-0 00' DAY TO HOUR",
|
||||
"INTERVAL '00:00:00.000001' HOUR TO SECOND",
|
||||
"INTERVAL '00:00' HOUR TO MINUTE",
|
||||
"INTERVAL '00:00.000001' MINUTE TO SECOND",
|
||||
"INTERVAL '-00:00:00.000001' HOUR TO SECOND",
|
||||
"INTERVAL '-00:00' HOUR TO MINUTE",
|
||||
"INTERVAL '-00:00.000001' MINUTE TO SECOND",
|
||||
"INTERVAL '-0' DAY",
|
||||
"INTERVAL '00' HOUR",
|
||||
"INTERVAL '00' MINUTE",
|
||||
"INTERVAL '00.000001' SECOND"),
|
||||
"INTERVAL '-00' HOUR",
|
||||
"INTERVAL '-00' MINUTE",
|
||||
"INTERVAL '-00.000001' SECOND"),
|
||||
10 * MICROS_PER_MILLIS ->
|
||||
("INTERVAL '0 00:00:00.01' DAY TO SECOND",
|
||||
"INTERVAL '0 00:00' DAY TO MINUTE",
|
||||
|
@ -576,24 +576,46 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper {
|
|||
("INTERVAL '-123 00:00:03' DAY TO SECOND",
|
||||
"INTERVAL '-123 00:00' DAY TO MINUTE",
|
||||
"INTERVAL '-123 00' DAY TO HOUR",
|
||||
"INTERVAL '00:00:03' HOUR TO SECOND",
|
||||
"INTERVAL '00:00' HOUR TO MINUTE",
|
||||
"INTERVAL '00:03' MINUTE TO SECOND",
|
||||
"INTERVAL '-2952:00:03' HOUR TO SECOND",
|
||||
"INTERVAL '-2952:00' HOUR TO MINUTE",
|
||||
"INTERVAL '-177120:03' MINUTE TO SECOND",
|
||||
"INTERVAL '-123' DAY",
|
||||
"INTERVAL '00' HOUR",
|
||||
"INTERVAL '00' MINUTE",
|
||||
"INTERVAL '03' SECOND"),
|
||||
"INTERVAL '-2952' HOUR",
|
||||
"INTERVAL '-177120' MINUTE",
|
||||
"INTERVAL '-10627203' SECOND"),
|
||||
Long.MinValue ->
|
||||
("INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND",
|
||||
"INTERVAL '-106751991 04:00' DAY TO MINUTE",
|
||||
"INTERVAL '-106751991 04' DAY TO HOUR",
|
||||
"INTERVAL '04:00:54.775808' HOUR TO SECOND",
|
||||
"INTERVAL '04:00' HOUR TO MINUTE",
|
||||
"INTERVAL '00:54.775808' MINUTE TO SECOND",
|
||||
"INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND",
|
||||
"INTERVAL '-2562047788:00' HOUR TO MINUTE",
|
||||
"INTERVAL '-153722867280:54.775808' MINUTE TO SECOND",
|
||||
"INTERVAL '-106751991' DAY",
|
||||
"INTERVAL '04' HOUR",
|
||||
"INTERVAL '00' MINUTE",
|
||||
"INTERVAL '54.775808' SECOND")
|
||||
"INTERVAL '-2562047788' HOUR",
|
||||
"INTERVAL '-153722867280' MINUTE",
|
||||
"INTERVAL '-9223372036854.775808' SECOND"),
|
||||
69159782123456L ->
|
||||
("INTERVAL '800 11:03:02.123456' DAY TO SECOND",
|
||||
"INTERVAL '800 11:03' DAY TO MINUTE",
|
||||
"INTERVAL '800 11' DAY TO HOUR",
|
||||
"INTERVAL '19211:03:02.123456' HOUR TO SECOND",
|
||||
"INTERVAL '19211:03' HOUR TO MINUTE",
|
||||
"INTERVAL '1152663:02.123456' MINUTE TO SECOND",
|
||||
"INTERVAL '800' DAY",
|
||||
"INTERVAL '19211' HOUR",
|
||||
"INTERVAL '1152663' MINUTE",
|
||||
"INTERVAL '69159782.123456' SECOND"),
|
||||
-69159782123456L ->
|
||||
("INTERVAL '-800 11:03:02.123456' DAY TO SECOND",
|
||||
"INTERVAL '-800 11:03' DAY TO MINUTE",
|
||||
"INTERVAL '-800 11' DAY TO HOUR",
|
||||
"INTERVAL '-19211:03:02.123456' HOUR TO SECOND",
|
||||
"INTERVAL '-19211:03' HOUR TO MINUTE",
|
||||
"INTERVAL '-1152663:02.123456' MINUTE TO SECOND",
|
||||
"INTERVAL '-800' DAY",
|
||||
"INTERVAL '-19211' HOUR",
|
||||
"INTERVAL '-1152663' MINUTE",
|
||||
"INTERVAL '-69159782.123456' SECOND")
|
||||
).foreach {
|
||||
case (
|
||||
micros, (
|
||||
|
|
Loading…
Reference in a new issue