diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 27d6ef09a69..9635034c9a4 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -485,3 +485,29 @@ def do_csv_scan(spark): do_csv_scan, exist_classes= "FileSourceScanExec", non_exist_classes= "GpuBatchScanExec") + +@pytest.mark.skipif(is_before_spark_330(), reason='Reading day-time interval type is supported from Spark3.3.0') +@pytest.mark.parametrize('v1_enabled_list', ["", "csv"]) +def test_round_trip_for_interval(spark_tmp_path, v1_enabled_list): + csv_interval_gens = [ + DayTimeIntervalGen(start_field="day", end_field="day"), + DayTimeIntervalGen(start_field="day", end_field="hour"), + DayTimeIntervalGen(start_field="day", end_field="minute"), + DayTimeIntervalGen(start_field="day", end_field="second"), + DayTimeIntervalGen(start_field="hour", end_field="hour"), + DayTimeIntervalGen(start_field="hour", end_field="minute"), + DayTimeIntervalGen(start_field="hour", end_field="second"), + DayTimeIntervalGen(start_field="minute", end_field="minute"), + DayTimeIntervalGen(start_field="minute", end_field="second"), + DayTimeIntervalGen(start_field="second", end_field="second"), + ] + + gen = StructGen([('_c' + str(i), csv_interval_gens[i]) for i in range(0, len(csv_interval_gens))], nullable=False) + data_path = spark_tmp_path + '/CSV_DATA' + schema = gen.data_type + updated_conf = copy_and_update(_enable_all_types_conf, {'spark.sql.sources.useV1SourceList': v1_enabled_list}) + with_cpu_session( + lambda spark: gen_df(spark, gen).write.csv(data_path)) + assert_gpu_and_cpu_are_equal_collect( + lambda spark: spark.read.schema(schema).csv(data_path), + conf=updated_conf) diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 819d5b644d6..3bbc29d7dd9 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -613,31 +613,43 @@ def make_null(): self._start(rand, make_null) # DayTimeIntervalGen is for Spark 3.3.0+ -# DayTimeIntervalType(startField, endField): Represents a day-time interval which is made up of a contiguous subset of the following fields: +# DayTimeIntervalType(startField, endField): +# Represents a day-time interval which is made up of a contiguous subset of the following fields: # SECOND, seconds within minutes and possibly fractions of a second [0..59.999999], +# Note Spark now uses 99 as max second, see issue https://issues.apache.org/jira/browse/SPARK-38324 +# If second is start field, its max value is long.max / microseconds in one second # MINUTE, minutes within hours [0..59], +# If minute is start field, its max value is long.max / microseconds in one minute # HOUR, hours within days [0..23], -# DAY, days in the range [0..106751991]. +# If hour is start field, its max value is long.max / microseconds in one hour +# DAY, days in the range [0..106751991]. 106751991 is long.max / microseconds in one day # For more details: https://spark.apache.org/docs/latest/sql-ref-datatypes.html -# Note: 106751991/365 = 292471 years which is much bigger than 9999 year, seems something is wrong +MIN_DAY_TIME_INTERVAL = timedelta(microseconds=-pow(2, 63)) +MAX_DAY_TIME_INTERVAL = timedelta(microseconds=(pow(2, 63) - 1)) class DayTimeIntervalGen(DataGen): """Generate DayTimeIntervalType values""" - def __init__(self, max_days = None, nullable=True, special_cases =[timedelta(seconds = 0)]): - super().__init__(DayTimeIntervalType(), nullable=nullable, special_cases=special_cases) - if max_days is None: - self._max_days = 106751991 - else: - self._max_days = max_days + def __init__(self, min_value=MIN_DAY_TIME_INTERVAL, max_value=MAX_DAY_TIME_INTERVAL, start_field="day", end_field="second", + nullable=True, special_cases=[timedelta(seconds=0)]): + # Note the nano seconds are truncated for min_value and max_value + self._min_micros = (math.floor(min_value.total_seconds()) * 1000000) + min_value.microseconds + self._max_micros = (math.floor(max_value.total_seconds()) * 1000000) + max_value.microseconds + fields = ["day", "hour", "minute", "second"] + start_index = fields.index(start_field) + end_index = fields.index(end_field) + if start_index > end_index: + raise RuntimeError('Start field {}, end field {}, valid fields is {}, start field index should <= end ' + 'field index'.format(start_field, end_field, fields)) + super().__init__(DayTimeIntervalType(start_index, end_index), nullable=nullable, special_cases=special_cases) + + def _gen_random(self, rand): + micros = rand.randint(self._min_micros, self._max_micros) + # issue: Interval types are not truncated to the expected endField when creating a DataFrame via Duration + # https://issues.apache.org/jira/browse/SPARK-38577 + # If above issue is fixed, should update this DayTimeIntervalGen. + return timedelta(microseconds=micros) + def start(self, rand): - self._start(rand, - lambda : timedelta( - microseconds = rand.randint(0, 999999), - seconds = rand.randint(0, 59), - minutes = rand.randint(0, 59), - hours = rand.randint(0, 23), - days = rand.randint(0, self._max_days), - ) - ) + self._start(rand, lambda: self._gen_random(rand)) def skip_if_not_utc(): if (not is_tz_utc()): diff --git a/integration_tests/src/main/python/date_time_test.py b/integration_tests/src/main/python/date_time_test.py index 0b40afdbe57..37931ac75bd 100644 --- a/integration_tests/src/main/python/date_time_test.py +++ b/integration_tests/src/main/python/date_time_test.py @@ -45,9 +45,9 @@ def test_timeadd(data_gen): def test_timeadd_daytime_column(): gen_list = [ # timestamp column max year is 1000 - ('t', TimestampGen(end = datetime(1000, 1, 1, tzinfo=timezone.utc))), + ('t', TimestampGen(end=datetime(1000, 1, 1, tzinfo=timezone.utc))), # max days is 8000 year, so added result will not be out of range - ('d', DayTimeIntervalGen(max_days = 8000 * 365))] + ('d', DayTimeIntervalGen(min_value=timedelta(days=0), max_value=timedelta(days=8000 * 365)))] assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, gen_list).selectExpr("t + d", "t + INTERVAL '1 02:03:04' DAY TO SECOND")) diff --git a/sql-plugin/src/main/301until330-all/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala b/sql-plugin/src/main/301until330-all/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala index 553c7aaf712..d4fd9fdbcc8 100644 --- a/sql-plugin/src/main/301until330-all/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala +++ b/sql-plugin/src/main/301until330-all/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala @@ -15,6 +15,7 @@ */ package com.nvidia.spark.rapids.shims +import ai.rapids.cudf import ai.rapids.cudf.DType import com.nvidia.spark.rapids.GpuRowToColumnConverter.TypeConverter @@ -74,4 +75,9 @@ object GpuTypeShims { def toScalarForType(t: DataType, v: Any) = { throw new RuntimeException(s"Can not convert $v to scalar for type $t.") } + + def supportCsvRead(dt: DataType) : Boolean = false + + def csvRead(cv: cudf.ColumnVector, dt: DataType): cudf.ColumnVector = + throw new RuntimeException(s"Not support type $dt.") } diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala new file mode 100644 index 00000000000..96545216cbb --- /dev/null +++ b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuIntervalUtils.scala @@ -0,0 +1,560 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.nvidia.spark.rapids.shims + +import java.util.concurrent.TimeUnit.{DAYS, HOURS, MINUTES, SECONDS} + +import ai.rapids.cudf.{ColumnVector, DType, Scalar} +import com.nvidia.spark.rapids.Arm + +import org.apache.spark.sql.types.{DayTimeIntervalType => DT} + +/** + * Parse DayTimeIntervalType string column to long column of micro seconds + * Spark DayTimeIntervalType type: + * https://spark.apache.org/docs/latest/sql-ref-datatypes.html + * Spark parse DayTime: + * https://github.com/apache/spark/blob/v3.2.1/ + * sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala#L275 + * + * DayTimeIntervalType have 10 sub types, 10 types and examples are as following: + * + * INTERVAL DAY: INTERVAL '100' DAY + * INTERVAL DAY TO HOUR: INTERVAL '100 10' DAY TO HOUR + * INTERVAL DAY TO MINUTE: INTERVAL '100 10:30' DAY TO MINUTE + * INTERVAL DAY TO SECOND: INTERVAL '100 10:30:40.999999' DAY TO SECOND + * INTERVAL HOUR: INTERVAL '123' HOUR + * INTERVAL HOUR TO MINUTE: INTERVAL '123:10' HOUR TO MINUTE + * INTERVAL HOUR TO SECOND: INTERVAL '123:10:59' HOUR TO SECOND + * INTERVAL MINUTE: INTERVAL '1000' MINUTE + * INTERVAL MINUTE TO SECOND: INTERVAL '1000:01.001' MINUTE TO SECOND + * INTERVAL SECOND: INTERVAL '1000.000001' SECOND + * + * For each sub type, there 2 valid forms, take DAY TO SECOND for example: + * INTERVAL '100 10:30:40.999999' DAY TO SECOND the normal mode + * 100 10:30:40.999999 the short mode + * + * Note: Currently not supporting the short mode, because of CSV writing generates the normal mode + * + * DAY, days in the range [0..106751991] + * HOUR, hours within days [0..23] + * Note if HOUR is leading item, range is [0, Long.max/ micros in one hour] + * MINUTE, minutes within hours [0..59] + * Note if MINUTE is leading item, range is [0, Long.max/ micros in one minute] + * SECOND, seconds within minutes and possibly fractions of a second [0..59.999999] + * Note if SECOND is leading item, range is [0, Long.max/ micros in one second] + * Max second within minutes should be 59, + * but Spark use 99, see this issue: https://issues.apache.org/jira/browse/SPARK-38324, + * should update correspondingly if Spark fixes this issue + * + */ +object GpuIntervalUtils extends Arm { + val MAX_DAY: Long = Long.MaxValue / DAYS.toMicros(1) + val MAX_HOUR: Long = Long.MaxValue / HOURS.toMicros(1) + val MAX_MINUTE: Long = Long.MaxValue / MINUTES.toMicros(1) + val MAX_SECOND: Long = Long.MaxValue / SECONDS.toMicros(1) + val MAX_HOUR_IN_DAY = 23L + val MAX_MINUTE_IN_HOUR = 59L + + // literals ignore upper and lower cases + private val INTERVAL = "[iI][nN][tT][eE][rR][vV][aA][lL]" + private val DAY = "[dD][aA][yY]" + private val HOUR = "[hH][oO][uU][rR]" + private val MINUTE = "[mM][iI][nN][uU][tT][eE]" + private val SECOND = "[sS][eE][cC][oO][nN][dD]" + private val TO = "[tT][oO]" + + // + or - + private val sign = "([+\\-])?" + private val blanks = "\\s+" + private val normalPattern = "(\\d{1,2})" + private val dayBoundPattern = "(\\d{1,9})" + private val hourBoundPattern = "(\\d{1,10})" + private val minuteBoundPattern = "(\\d{1,12})" + private val secondBoundPattern = "(\\d{1,13})" + private val microPattern = "(\\.\\d{1,9})?" + + private val dayPatternString = s"$sign$dayBoundPattern" + private val dayLiteralRegex = s"^$INTERVAL$blanks$sign'$dayPatternString'$blanks$DAY$$" + + private val dayHourPatternString = s"$sign$dayBoundPattern $normalPattern" + private val dayHourLiteralRegex = + s"^$INTERVAL$blanks$sign'$dayHourPatternString'$blanks$DAY$blanks$TO$blanks$HOUR$$" + + private val dayMinutePatternString = s"$sign$dayBoundPattern $normalPattern:$normalPattern" + private val dayMinuteLiteralRegex = + s"^$INTERVAL$blanks$sign'$dayMinutePatternString'$blanks$DAY$blanks$TO$blanks$MINUTE$$" + + private val daySecondPatternString = + s"$sign$dayBoundPattern $normalPattern:$normalPattern:$normalPattern$microPattern" + private val daySecondLiteralRegex = + s"^$INTERVAL$blanks$sign'$daySecondPatternString'$blanks$DAY$blanks$TO$blanks$SECOND$$" + + private val hourPatternString = s"$sign$hourBoundPattern" + private val hourLiteralRegex = s"^$INTERVAL$blanks$sign'$hourPatternString'$blanks$HOUR$$" + + private val hourMinutePatternString = s"$sign$hourBoundPattern:$normalPattern" + private val hourMinuteLiteralRegex = + s"^$INTERVAL$blanks$sign'$hourMinutePatternString'$blanks$HOUR$blanks$TO$blanks$MINUTE$$" + + private val hourSecondPatternString = + s"$sign$hourBoundPattern:$normalPattern:$normalPattern$microPattern" + private val hourSecondLiteralRegex = + s"^$INTERVAL$blanks$sign'$hourSecondPatternString'$blanks$HOUR$blanks$TO$blanks$SECOND$$" + + private val minutePatternString = s"$sign$minuteBoundPattern" + private val minuteLiteralRegex = s"^$INTERVAL$blanks$sign'$minutePatternString'$blanks$MINUTE$$" + + private val minuteSecondPatternString = + s"$sign$minuteBoundPattern:$normalPattern$microPattern" + private val minuteSecondLiteralRegex = + s"^$INTERVAL$blanks$sign'$minuteSecondPatternString'$blanks$MINUTE$blanks$TO$blanks$SECOND$$" + + private val secondPatternString = s"$sign$secondBoundPattern$microPattern" + private val secondLiteralRegex = s"^$INTERVAL$blanks$sign'$secondPatternString'$blanks$SECOND$$" + + def checkAllValid(cv: ColumnVector, regexp: String): Unit = { + if (cv.hasNulls) { + throw new RuntimeException(s"Do not match the regular expression: $regexp") + } + } + + /** + * Cast string column to long column, if the casting of a row failed set null by default. + * Fail reason includes: regexp not match, range check failed, overflow when adding + * + * @param cv string column + * @param t day-time interval type + * @param throwException throw exception if failed when throwException is true + * TODO: checked regexp match, should also check range, check overflow, + * seems there are no operators requiring this. + * @return long column of micros + */ + def castStringToDTInterval( + cv: ColumnVector, t: DT, throwException: Boolean = false): ColumnVector = { + (t.startField, t.endField) match { + case (DT.DAY, DT.DAY) => withResource(cv.extractRe(dayLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), dayLiteralRegex) + } + + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromDayToDay(sign, + groupsTable.getColumn(2) // day + ) + } + } + } + + case (DT.DAY, DT.HOUR) => withResource(cv.extractRe(dayHourLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), dayHourLiteralRegex) + } + + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromDayToHour(sign, + groupsTable.getColumn(2), // day + groupsTable.getColumn(3) // hour + ) + } + } + } + + case (DT.DAY, DT.MINUTE) => withResource(cv.extractRe(dayMinuteLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), dayMinuteLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromDayToMinute(sign, + groupsTable.getColumn(2), // day + groupsTable.getColumn(3), // hour + groupsTable.getColumn(4) // minute + ) + } + } + } + + case (DT.DAY, DT.SECOND) => withResource(cv.extractRe(daySecondLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), daySecondLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromDayToSecond(sign, + groupsTable.getColumn(2), // day + groupsTable.getColumn(3), // hour + groupsTable.getColumn(4), // minute + groupsTable.getColumn(5), // second + groupsTable.getColumn(6) // micro + ) + } + } + } + + case (DT.HOUR, DT.HOUR) => withResource(cv.extractRe(hourLiteralRegex)) { groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), hourLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromHourToHour(sign, + groupsTable.getColumn(2) // hour + ) + } + } + } + + case (DT.HOUR, DT.MINUTE) => withResource(cv.extractRe(hourMinuteLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), hourMinuteLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromHourToMinute(sign, + groupsTable.getColumn(2), // hour + groupsTable.getColumn(3) // minute + ) + } + } + } + + case (DT.HOUR, DT.SECOND) => withResource(cv.extractRe(hourSecondLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), hourSecondLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { + sign => + addFromHourToSecond(sign, + groupsTable.getColumn(2), // hour + groupsTable.getColumn(3), // minute + groupsTable.getColumn(4), // second + groupsTable.getColumn(5) // micros + ) + } + } + } + + case (DT.MINUTE, DT.MINUTE) => withResource(cv.extractRe(minuteLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), minuteLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromMinuteToMinute(sign, + groupsTable.getColumn(2) // minute + ) + } + } + } + + case (DT.MINUTE, DT.SECOND) => withResource(cv.extractRe(minuteSecondLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), minuteSecondLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromMinuteToSecond(sign, + groupsTable.getColumn(2), // minute + groupsTable.getColumn(3), // second + groupsTable.getColumn(4) // micro + ) + } + } + } + + case (DT.SECOND, DT.SECOND) => withResource(cv.extractRe(secondLiteralRegex)) { + groupsTable => { + if (throwException) { + // check all match the regexp + checkAllValid(groupsTable.getColumn(2), secondLiteralRegex) + } + withResource(finalSign(groupsTable.getColumn(0), groupsTable.getColumn(1))) { sign => + addFromSecondToSecond(sign, + groupsTable.getColumn(2), // second + groupsTable.getColumn(3) // micro + ) + } + } + } + + case _ => + throw new RuntimeException( + s"Not supported DayTimeIntervalType(${t.startField}, ${t.endField})") + } + } + + // get sign column of long type with 1L or -1L in it + // not close firstSignInTable and secondSignInTable here, outer table.close will close them + private def finalSign( + firstSignInTable: ColumnVector, secondSignInTable: ColumnVector): ColumnVector = { + withResource(Scalar.fromString("-")) { negScalar => + withResource(negScalar.equalTo(firstSignInTable)) { neg1 => + withResource(negScalar.equalTo(secondSignInTable)) { neg2 => + withResource(neg1.bitXor(neg2)) { s => + withResource(Scalar.fromLong(1L)) { one => + withResource(Scalar.fromLong(-1L)) { negOne => + s.ifElse(negOne, one) + } + } + } + } + } + } + } + + /** + * get micro seconds from decimal string column and truncate the nano seconds + * e.g.: .123456789 => 123456 + * + * @param decimal string column + * @return micros column + */ + private def getMicrosFromDecimal(sign: ColumnVector, decimal: ColumnVector): ColumnVector = { + withResource(decimal.castTo(DType.create(DType.DTypeEnum.DECIMAL64, -6))) { decimal => + withResource(Scalar.fromLong(1000000L)) { million => + withResource(decimal.mul(million)) { r => + withResource(r.asLongs()) { l => + l.mul(sign) + } + } + } + } + } + + private def addFromDayToDay( + sign: ColumnVector, + daysInTable: ColumnVector + ): ColumnVector = { + daysToMicros(sign, daysInTable, MAX_DAY) + } + + private def addFromDayToHour( + sign: ColumnVector, + daysInTable: ColumnVector, + hoursInTable: ColumnVector + ): ColumnVector = { + add(daysToMicros(sign, daysInTable, MAX_DAY), + hoursToMicros(sign, hoursInTable, MAX_HOUR_IN_DAY)) + } + + private def addFromDayToMinute( + sign: ColumnVector, + daysInTable: ColumnVector, + hoursInTable: ColumnVector, + minutesInTable: ColumnVector + ): ColumnVector = { + add(daysToMicros(sign, daysInTable, MAX_DAY), + add(hoursToMicros(sign, hoursInTable, MAX_HOUR_IN_DAY), + minutesToMicros(sign, minutesInTable, MAX_MINUTE_IN_HOUR))) + } + + private def addFromDayToSecond( + sign: ColumnVector, + daysInTable: ColumnVector, + hoursInTable: ColumnVector, + minutesInTable: ColumnVector, + secondsInTable: ColumnVector, + microsInTable: ColumnVector + ): ColumnVector = { + add(daysToMicros(sign, daysInTable, MAX_DAY), + add(hoursToMicros(sign, hoursInTable, MAX_HOUR_IN_DAY), + add(minutesToMicros(sign, minutesInTable, MAX_MINUTE_IN_HOUR), + add(secondsToMicros(sign, secondsInTable), // max value is 99, no overflow + getMicrosFromDecimal(sign, microsInTable))))) // max value is 999999999, no overflow + } + + private def addFromHourToHour( + sign: ColumnVector, + hoursInTable: ColumnVector + ): ColumnVector = { + hoursToMicros(sign, hoursInTable, MAX_HOUR) + } + + private def addFromHourToMinute( + sign: ColumnVector, + hoursInTable: ColumnVector, + minutesInTable: ColumnVector + ): ColumnVector = { + add(hoursToMicros(sign, hoursInTable, MAX_HOUR), + minutesToMicros(sign, minutesInTable, MAX_MINUTE_IN_HOUR)) + } + + private def addFromHourToSecond( + sign: ColumnVector, + hoursInTable: ColumnVector, + minutesInTable: ColumnVector, + secondsInTable: ColumnVector, + microsInTable: ColumnVector + ): ColumnVector = { + add(hoursToMicros(sign, hoursInTable, MAX_HOUR), + add(minutesToMicros(sign, minutesInTable, MAX_MINUTE_IN_HOUR), + add(secondsToMicros(sign, secondsInTable), + getMicrosFromDecimal(sign, microsInTable)))) + } + + private def addFromMinuteToMinute( + sign: ColumnVector, + minutesInTable: ColumnVector + ): ColumnVector = { + minutesToMicros(sign, minutesInTable, MAX_MINUTE) + } + + private def addFromMinuteToSecond( + sign: ColumnVector, + minutesInTable: ColumnVector, + secondsInTable: ColumnVector, + microsInTable: ColumnVector + ): ColumnVector = { + add(minutesToMicros(sign, minutesInTable, MAX_MINUTE), + add(secondsToMicros(sign, secondsInTable), + getMicrosFromDecimal(sign, microsInTable))) + } + + private def addFromSecondToSecond( + sign: ColumnVector, + secondsInTable: ColumnVector, + microsInTable: ColumnVector + ): ColumnVector = { + add(secondsToMicros(sign, secondsInTable, MAX_SECOND), + getMicrosFromDecimal(sign, microsInTable)) + } + + // Check overflow. It is true when both arguments have the opposite sign of the result. + // Which is equal to "((x ^ r) & (y ^ r)) < 0" in the form of arithmetic. + private def getOverflow(lhs: ColumnVector, rhs: ColumnVector, ret: ColumnVector): ColumnVector = { + val signCV = withResource(ret.bitXor(lhs)) { lXor => + withResource(ret.bitXor(rhs)) { rXor => + lXor.bitAnd(rXor) + } + } + withResource(signCV) { sign => + withResource(Scalar.fromInt(0)) { zero => + sign.lessThan(zero) + } + } + } + + // set null if overflow + private def setNullIfOverflow( + lhs: ColumnVector, rhs: ColumnVector, ret: ColumnVector): ColumnVector = { + withResource(getOverflow(lhs, rhs, ret)) { overflow => + withResource(Scalar.fromNull(DType.INT64)) { nullScalar => + // if overflow, set as null + overflow.ifElse(nullScalar, ret) + } + } + } + + // Add left and right to a new one and then close them. + // Check overflow and set null + private def add(left: ColumnVector, right: ColumnVector): ColumnVector = { + withResource(left) { l => + withResource(right) { r => + withResource(l.add(r)) { result => + setNullIfOverflow(l, r, result) + } + } + } + } + + private def daysToMicros( + sign: ColumnVector, daysInGroupTable: ColumnVector, maxDay: Long): ColumnVector = { + multiple(sign, daysInGroupTable, DAYS.toMicros(1), maxDay) + } + + private def hoursToMicros( + sign: ColumnVector, hoursInGroupTable: ColumnVector, maxHour: Long): ColumnVector = { + multiple(sign, hoursInGroupTable, HOURS.toMicros(1), maxHour) + } + + private def minutesToMicros( + sign: ColumnVector, minutesInGroupTable: ColumnVector, maxMinute: Long): ColumnVector = { + multiple(sign, minutesInGroupTable, MINUTES.toMicros(1), maxMinute) + } + + private def secondsToMicros( + sign: ColumnVector, secondsInGroupTable: ColumnVector, maxSecond: Long): ColumnVector = { + multiple(sign, secondsInGroupTable, SECONDS.toMicros(1), maxSecond) + } + + private def secondsToMicros( + sign: ColumnVector, secondsInGroupTable: ColumnVector): ColumnVector = { + multiple(sign, secondsInGroupTable, SECONDS.toMicros(1)) + } + + /** + * Check range, return sign * base * multiple. + * + * @param sign long column with 1L or -1L in it + * @param base string column contains positive long + * @param multiple const long value + * @param maxInBase the max value for base column + * @return + */ + private def multiple( + sign: ColumnVector, base: ColumnVector, multiple: Long, maxInBase: Long): ColumnVector = { + // check max limit, set null if exceeds the max value + val baseWithFixCv = withResource(Scalar.fromLong(maxInBase)) { maxScalar => + withResource(Scalar.fromNull(DType.INT64)) { nullScalar => + withResource(base.castTo(DType.INT64)) { baseLong => + withResource(baseLong.greaterThan(maxScalar)) { greater => + greater.ifElse(nullScalar, baseLong) + } + } + } + } + val baseWithSignCv = withResource(baseWithFixCv) { baseWithFix => + baseWithFix.mul(sign) + } + withResource(baseWithSignCv) { baseWithSign => + withResource(Scalar.fromLong(multiple)) { multipleScalar => + baseWithSign.mul(multipleScalar) + } + } + } + + /** + * Return sign * base * multiple. + * + * @param sign long column + * @param groupInTable string column contains positive long + * @param multiple const long value + * @return sign * base * multiple + */ + private def multiple( + sign: ColumnVector, groupInTable: ColumnVector, multiple: Long): ColumnVector = { + val baseWithSignCv = withResource(groupInTable.castTo(DType.INT64)) { baseLong => + baseLong.mul(sign) + } + withResource(baseWithSignCv) { baseWithSign => + withResource(Scalar.fromLong(multiple)) { multipleScalar => + baseWithSign.mul(multipleScalar) + } + } + } +} diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala index f9e81277341..6e5de65bf81 100644 --- a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala +++ b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/GpuTypeShims.scala @@ -15,6 +15,7 @@ */ package com.nvidia.spark.rapids.shims +import ai.rapids.cudf import ai.rapids.cudf.{DType, Scalar} import com.nvidia.spark.rapids.ColumnarCopyHelper import com.nvidia.spark.rapids.GpuRowToColumnConverter.{LongConverter, NotNullLongConverter, TypeConverter} @@ -143,4 +144,20 @@ object GpuTypeShims { throw new RuntimeException(s"Can not convert $v to scalar for type $t.") } } + + + def supportCsvRead(dt: DataType) : Boolean = { + dt match { + case DayTimeIntervalType(_, _) => true + case _ => false + } + } + + def csvRead(cv: cudf.ColumnVector, dt: DataType): cudf.ColumnVector = { + dt match { + case d: DayTimeIntervalType => GpuIntervalUtils.castStringToDTInterval(cv, d) + case _ => throw new RuntimeException(s"Not support type $dt.") + } + } + } diff --git a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/Spark33XShims.scala b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/Spark33XShims.scala index 55d9cae2078..a938d84a75a 100644 --- a/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/Spark33XShims.scala +++ b/sql-plugin/src/main/330+/scala/com/nvidia/spark/rapids/shims/Spark33XShims.scala @@ -18,7 +18,6 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.InMemoryTableScanMeta import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuOverrides import org.apache.parquet.schema.MessageType import org.apache.spark.rdd.RDD @@ -31,8 +30,11 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FilePartition, FileScanRDD, HadoopFsRelation, PartitionedFile} import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids._ +import org.apache.spark.sql.rapids.execution.GpuShuffleMeta import org.apache.spark.sql.rapids.shims.GpuTimeAdd import org.apache.spark.sql.types.{CalendarIntervalType, DayTimeIntervalType, StructType} import org.apache.spark.unsafe.types.CalendarInterval @@ -84,6 +86,10 @@ trait Spark33XShims extends Spark33XFileOptionsShims { // 330+ supports DAYTIME interval types override def getFileFormats: Map[FileFormatType, Map[FileFormatOp, FileFormatChecks]] = { Map( + (CsvFormatType, FileFormatChecks( + cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.DAYTIME, + cudfWrite = TypeSig.none, + sparkSig = TypeSig.cpuAtomics)), (ParquetFormatType, FileFormatChecks( cudfRead = (TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP + TypeSig.DAYTIME).nested(), @@ -259,6 +265,31 @@ trait Spark33XShims extends Spark33XFileOptionsShims { override def getExecs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = { val _gpuCommonTypes = TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_64 val map: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( + GpuOverrides.exec[ShuffleExchangeExec]( + "The backend for most data being exchanged between processes", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.NULL + TypeSig.DECIMAL_128 + + TypeSig.ARRAY + TypeSig.STRUCT + TypeSig.MAP + TypeSig.DAYTIME).nested() + .withPsNote(TypeEnum.STRUCT, "Round-robin partitioning is not supported for nested " + + s"structs if ${SQLConf.SORT_BEFORE_REPARTITION.key} is true") + .withPsNote(TypeEnum.ARRAY, "Round-robin partitioning is not supported if " + + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true") + .withPsNote(TypeEnum.MAP, "Round-robin partitioning is not supported if " + + s"${SQLConf.SORT_BEFORE_REPARTITION.key} is true"), + TypeSig.all), + (shuffle, conf, p, r) => new GpuShuffleMeta(shuffle, conf, p, r)), + GpuOverrides.exec[BatchScanExec]( + "The backend for most file input", + ExecChecks( + (TypeSig.commonCudfTypes + TypeSig.STRUCT + TypeSig.MAP + TypeSig.ARRAY + + TypeSig.DECIMAL_128 + TypeSig.DAYTIME).nested(), + TypeSig.all), + (p, conf, parent, r) => new SparkPlanMeta[BatchScanExec](p, conf, parent, r) { + override val childScans: scala.Seq[ScanMeta[_]] = + Seq(GpuOverrides.wrapScan(p.scan, conf, Some(this))) + + override def convertToGpu(): GpuExec = + GpuBatchScanExec(p.output, childScans.head.convertToGpu()) + }), GpuOverrides.exec[CoalesceExec]( "The backend for the dataframe coalesce method", ExecChecks((_gpuCommonTypes + TypeSig.DECIMAL_128 + TypeSig.STRUCT + TypeSig.ARRAY + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala index d1317f14a6f..b965bfedccf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ListBuffer import scala.math.max import ai.rapids.cudf.{ColumnVector, DType, HostMemoryBuffer, NvtxColor, NvtxRange, Scalar, Schema, Table} +import com.nvidia.spark.rapids.shims.GpuTypeShims import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.compress.CompressionCodecFactory @@ -177,6 +178,8 @@ abstract class GpuTextBasedPartitionReader( DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType | DataTypes.DoubleType | _: DecimalType | DataTypes.DateType => f.copy(dataType = DataTypes.StringType) + case other if GpuTypeShims.supportCsvRead(other) => + f.copy(dataType = DataTypes.StringType) case _ => f } @@ -219,6 +222,8 @@ abstract class GpuTextBasedPartitionReader( castStringToDecimal(table.getColumn(i), dt) case DataTypes.DateType => castStringToDate(table.getColumn(i)) + case other if GpuTypeShims.supportCsvRead(other) => + GpuTypeShims.csvRead(table.getColumn(i), other) case _ => table.getColumn(i).incRefCount() } diff --git a/tests/pom.xml b/tests/pom.xml index 7247c7b37d4..83d4c44df19 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -238,6 +238,34 @@ + + release330 + + + buildver + 330 + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-330-test-src + add-test-source + + + ${project.basedir}/src/test/330/scala + + + + + + + + diff --git a/tests/src/test/330/scala/com/nvidia/spark/rapids/CsvScanForIntervalSuite.scala b/tests/src/test/330/scala/com/nvidia/spark/rapids/CsvScanForIntervalSuite.scala new file mode 100644 index 00000000000..ad852ae4813 --- /dev/null +++ b/tests/src/test/330/scala/com/nvidia/spark/rapids/CsvScanForIntervalSuite.scala @@ -0,0 +1,637 @@ +/* + * Copyright (c) 2022, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.nvidia.spark.rapids + +import ai.rapids.cudf.ColumnVector +import com.nvidia.spark.rapids.shims.GpuIntervalUtils + +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.types.{DayTimeIntervalType, StructField, StructType} + +class CsvScanForIntervalSuite extends SparkQueryCompareTestSuite { + test("test castStringToDTInterval format valid") { + withResource(ColumnVector.fromLongs( + 86400000000L, + -86400000000L, + 86400000000L, + + 86400000000L, + -86400000000L, + 86400000000L, + + 86400000000L, + 86400000000L, + 86400000000L, + + -86400000000L, + -86400000000L, + 86400000000L + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval '1' day", + "inTERval '-1' DAY", + "INTERVAL -'-1' DAY", + + "interval '01' DAY", + "interval '-01' DAY", + "interval -'-01' DAY", + + "inTerVal +'01' day", + "INTeRVAL '+01' DaY", + "INTERvAL +'+01' dAY", + + "interval +'-01' dAY", + "INTERVAL -'+01' DAy", + "INTERVAL +'+01' DAy" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + } + + test("test castStringToDTInterval for 10 sub types") { + val micros1Day = 86400L * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Day, + -micros1Day, + micros1Day, + micros1Day + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1' day", + "inTERval '-1' DAY", + "INTERVAL -'-1' DAY", + "INTERVAL +'+1' DAY" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Day1Hour = (86400L + 3600L) * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Day1Hour, + -micros1Day1Hour, + micros1Day1Hour, + micros1Day1Hour + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1 1' day to hour", + "inTERval '-1 1' DAY to HOUR", + "INTERVAL -'-1 1' DAY TO HOUR", + "INTERVAL +'+1 1' DAY TO HOUR" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.HOUR))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Day1Hour1Minute = (86400L + 3600L + 60) * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Day1Hour1Minute, + -micros1Day1Hour1Minute, + micros1Day1Hour1Minute, + micros1Day1Hour1Minute + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1 1:1' day to MINUTE", + "inTERval '-1 1:1' DAY to MINUTE", + "INTERVAL -'-1 1:1' DAY TO MINUTE", + "INTERVAL +'+1 1:1' DAY TO MINUTE" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.MINUTE))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Day1Hour1Minute1Second = (86400L + 3600L + 60 + 1) * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Day1Hour1Minute1Second, + -micros1Day1Hour1Minute1Second, + micros1Day1Hour1Minute1Second, + micros1Day1Hour1Minute1Second + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1 1:1:1' day to SECOND", + "inTERval '-1 1:1:1' DAY to SECOND", + "INTERVAL -'-1 1:1:1' DAY TO SECOND", + "INTERVAL +'+1 1:1:1' DAY TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Day1Hour1Minute1Second1Micros = (86400L + 3600L + 60 + 1) * 1000000L + 100000L + withResource(ColumnVector.fromLongs( + -micros1Day1Hour1Minute1Second1Micros, + -micros1Day1Hour1Minute1Second1Micros, + micros1Day1Hour1Minute1Second1Micros, + micros1Day1Hour1Minute1Second1Micros + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1 1:1:1.1' day to SECOND", + "inTERval '-1 1:1:1.1' DAY to SECOND", + "INTERVAL -'-1 1:1:1.1' DAY TO SECOND", + "INTERVAL +'+1 1:1:1.1' DAY TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Hour = 3600L * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Hour, + -micros1Hour, + micros1Hour, + micros1Hour + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1' hour", + "inTERval '-1' hour", + "INTERVAL -'-1' hour", + "INTERVAL +'+1' hour" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.HOUR))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + + val micros1Hour1Minute = (3600L + 60) * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Hour1Minute, + -micros1Hour1Minute, + micros1Hour1Minute, + micros1Hour1Minute + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1:1' hour to minute", + "inTERval '-1:1' hour to MINUTE", + "INTERVAL -'-1:1' hour TO MINUTE", + "INTERVAL +'+1:1' hour TO MINUTE" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.MINUTE))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Hour1Minute1Second = (3600L + 60 + 1) * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Hour1Minute1Second, + -micros1Hour1Minute1Second, + micros1Hour1Minute1Second, + micros1Hour1Minute1Second + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1:1:1' HOUR to SECOND", + "inTERval '-1:1:1' HOUR to SECOND", + "INTERVAL -'-1:1:1' HOUR TO SECOND", + "INTERVAL +'+1:1:1' HOUR TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.SECOND))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Hour1Minute1Second1Micros = (3600L + 60 + 1) * 1000000L + 100000L + withResource(ColumnVector.fromLongs( + -micros1Hour1Minute1Second1Micros, + -micros1Hour1Minute1Second1Micros, + micros1Hour1Minute1Second1Micros, + micros1Hour1Minute1Second1Micros + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1:1:1.1' HOUR to SECOND", + "inTERval '-1:1:1.1' HOUR to SECOND", + "INTERVAL -'-1:1:1.1' HOUR TO SECOND", + "INTERVAL +'+1:1:1.1' HOUR TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.SECOND))) { actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Minute = 60 * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Minute, + -micros1Minute, + micros1Minute, + micros1Minute + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1' MINUTE", + "inTERval '-1' MINUTE", + "INTERVAL -'-1' MINUTE", + "INTERVAL +'+1' MINUTE" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.MINUTE, DayTimeIntervalType.MINUTE))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Minute1Second = (60 + 1) * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Minute1Second, + -micros1Minute1Second, + micros1Minute1Second, + micros1Minute1Second + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1:1' MINUTE to SECOND", + "inTERval '-1:1' MINUTE to SECOND", + "INTERVAL -'-1:1' MINUTE TO SECOND", + "INTERVAL +'+1:1' MINUTE TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.MINUTE, DayTimeIntervalType.SECOND))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Minute1Second1Micros = (60 + 1) * 1000000L + 100000L + withResource(ColumnVector.fromLongs( + -micros1Minute1Second1Micros, + -micros1Minute1Second1Micros, + micros1Minute1Second1Micros, + micros1Minute1Second1Micros + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1:1.1' MINUTE to SECOND", + "inTERval '-1:1.1' MINUTE to SECOND", + "INTERVAL -'-1:1.1' MINUTE TO SECOND", + "INTERVAL +'+1:1.1' MINUTE TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.MINUTE, DayTimeIntervalType.SECOND))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Second = 1 * 1000000L + withResource(ColumnVector.fromLongs( + -micros1Second, + -micros1Second, + micros1Second, + micros1Second + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1' SECOND", + "inTERval '-1' SECOND", + "INTERVAL -'-1' SECOND", + "INTERVAL +'+1' SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.SECOND, DayTimeIntervalType.SECOND))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + val micros1Second1Micros = 1 * 1000000L + 100000L + withResource(ColumnVector.fromLongs( + -micros1Second1Micros, + -micros1Second1Micros, + micros1Second1Micros, + micros1Second1Micros + )) { expectCV => + withResource(ColumnVector.fromStrings( + "interval -'1.1' SECOND", + "inTERval '-1.1' SECOND", + "INTERVAL -'-1.1' SECOND", + "INTERVAL +'+1.1' SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.SECOND, DayTimeIntervalType.SECOND))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + } + + test("test castStringToDTInterval min max") { + withResource(ColumnVector.fromLongs( + Long.MaxValue, + Long.MinValue, + 0L + )) { expectCV => + withResource(ColumnVector.fromStrings( + s"interval +'+${GpuIntervalUtils.MAX_DAY} 4:0:54.775807' day to SECOND", + s"interval -'${GpuIntervalUtils.MAX_DAY} 4:0:54.775808' day to SECOND", + "INTERVAL '-0 00:00:00' day to SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + withResource(ColumnVector.fromLongs( + Long.MaxValue / (86400L * 1000000L) * (86400L * 1000000L), + Long.MinValue / (86400L * 1000000L) * (86400L * 1000000L) + )) { expectCV => + withResource(ColumnVector.fromStrings( + s"interval +'+${GpuIntervalUtils.MAX_DAY}' day", + s"interval -'${GpuIntervalUtils.MAX_DAY}' day" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + withResource(ColumnVector.fromLongs( + Long.MaxValue / (3600L * 1000000L) * (3600L * 1000000L), + Long.MinValue / (3600L * 1000000L) * (3600L * 1000000L) + )) { expectCV => + withResource(ColumnVector.fromStrings( + s"interval +'+${GpuIntervalUtils.MAX_HOUR}' HOUR", + s"interval -'${GpuIntervalUtils.MAX_HOUR}' HOUR" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.HOUR))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + withResource(ColumnVector.fromLongs( + Long.MaxValue / (60L * 1000000L) * (60L * 1000000L), + Long.MinValue / (60L * 1000000L) * (60L * 1000000L) + )) { expectCV => + withResource(ColumnVector.fromStrings( + s"interval +'+${GpuIntervalUtils.MAX_MINUTE}' minute", + s"interval -'${GpuIntervalUtils.MAX_MINUTE}' minute" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.MINUTE, DayTimeIntervalType.MINUTE))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + + withResource(ColumnVector.fromLongs( + Long.MaxValue / 1000000L * 1000000L, + Long.MinValue / 1000000L * 1000000L + )) { expectCV => + withResource(ColumnVector.fromStrings( + s"interval +'+${GpuIntervalUtils.MAX_SECOND}' SECOND", + s"interval -'${GpuIntervalUtils.MAX_SECOND}' SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.SECOND, DayTimeIntervalType.SECOND))) { + actualCV => + CudfTestHelper.assertColumnsAreEqual(expectCV, actualCV) + } + } + } + } + + test("test castStringToDTInterval for overflow and range") { + // check the overflow + withResource(ColumnVector.fromStrings( + s"interval '${GpuIntervalUtils.MAX_DAY} 4:0:54.775808' day to second", // Long.MaxValue + 1 + s"interval '-${GpuIntervalUtils.MAX_DAY} 4:0:54.775809' day to second", // Long.MinValue - 1 + s"interval '${GpuIntervalUtils.MAX_DAY} 5:0:0.0' day to second" // > Long.MaxValue + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max hour and max second which are not leading items + withResource(ColumnVector.fromStrings( + "interval '1 24:0:54.775808' day to second", // 24 hour is invalid + "interval '1 2:60:54.775808' day to second" // 60 minute is invalid + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max hour and max second which are not leading items + withResource(ColumnVector.fromStrings( + "interval '2:60:54.775808' day to second" // 60 minute is invalid + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max leading day + withResource(ColumnVector.fromStrings( + s"interval '${Long.MaxValue / (86400L * 1000000L) + 1} 0:0:0.0' day to second" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max leading hour + withResource(ColumnVector.fromStrings( + s"interval '${Long.MaxValue / (3600L * 1000000L) + 1}:0:0.0' hour to second" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max leading minute + withResource(ColumnVector.fromStrings( + s"interval '${Long.MaxValue / (60L * 1000000L) + 1}:0.0' minute to second" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.MINUTE, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max leading second + withResource(ColumnVector.fromStrings( + s"interval '${Long.MaxValue / 1000000L + 1}' second" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.SECOND, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + + // check max leading second + withResource(ColumnVector.fromStrings( + s"interval '-${Long.MinValue / 1000000L + 1}' second" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.SECOND, DayTimeIntervalType.SECOND))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) // return null because invalid + } + } + } + + test("test castStringToDTInterval format invalid") { + withResource(ColumnVector.fromStrings( + "INTERVAL xx DAY", + "INTERVAL 3 day", + "INTERVAL 3' day", + "INTERVAL '3 day", + "INTERVAL '- 3' day", + "INTERVAL - '3' day", + "INTERVAL '100 10:30:40.' DAY TO SECOND" + )) { intervalCV => + withResource(GpuIntervalUtils.castStringToDTInterval(intervalCV, + DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY))) { actualCV => + for (i <- 0 until actualCV.getRowCount.toInt) + assert(actualCV.isNull.getScalarElement(i).getBoolean) + } + } + } + + def readCsv(spark: SparkSession, path: String): DataFrame = { + def dayTime(s: Byte, e: Byte): DayTimeIntervalType = DayTimeIntervalType(s, e) + + val schema = StructType(Seq( + StructField("c01", dayTime(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY)), + StructField("c02", dayTime(DayTimeIntervalType.DAY, DayTimeIntervalType.HOUR)), + StructField("c03", dayTime(DayTimeIntervalType.DAY, DayTimeIntervalType.MINUTE)), + StructField("c04", dayTime(DayTimeIntervalType.DAY, DayTimeIntervalType.SECOND)), + StructField("c05", dayTime(DayTimeIntervalType.HOUR, DayTimeIntervalType.HOUR)), + StructField("c06", dayTime(DayTimeIntervalType.HOUR, DayTimeIntervalType.MINUTE)), + StructField("c07", dayTime(DayTimeIntervalType.HOUR, DayTimeIntervalType.SECOND)), + StructField("c08", dayTime(DayTimeIntervalType.MINUTE, DayTimeIntervalType.MINUTE)), + StructField("c09", dayTime(DayTimeIntervalType.MINUTE, DayTimeIntervalType.SECOND)), + StructField("c10", dayTime(DayTimeIntervalType.SECOND, DayTimeIntervalType.SECOND)) + )) + fromCsvDf(path, schema)(spark) + } + + testSparkResultsAreEqual( + "test read day-time interval csv file", + spark => readCsv(spark, "day-time-interval.csv") + ) { + df => df + } + + /** + * TODO: Blocked by Spark overflow issue: https://issues.apache.org/jira/browse/SPARK-38520 + * + * // days overflow + * scala> val schema = StructType(Seq(StructField("c1", + * DayTimeIntervalType(DayTimeIntervalType.DAY, DayTimeIntervalType.DAY)))) + * scala> spark.read.csv(path).show(false) + * +------------------------+ + * |_c0 | + * +------------------------+ + * |interval '106751992' day| + * +------------------------+ + * scala> spark.read.schema(schema).csv(path).show(false) + * +-------------------------+ + * |c1 | + * +-------------------------+ + * |INTERVAL '-106751990' DAY| + * +-------------------------+ + * + * // hour overflow + * scala> val schema = StructType(Seq(StructField("c1", + * DayTimeIntervalType(DayTimeIntervalType.HOUR, DayTimeIntervalType.HOUR)))) + * scala> spark.read.csv(path).show(false) + * +----------------------------+ + * |_c0 | + * +----------------------------+ + * |INTERVAL +'+2562047789' hour| + * +----------------------------+ + * scala> spark.read.schema(schema).csv(path).show(false) + * +---------------------------+ + * |c1 | + * +---------------------------+ + * |INTERVAL '-2562047787' HOUR| + * +---------------------------+ + * + * // minute overflow + * scala> val schema = StructType(Seq(StructField("c1", + * DayTimeIntervalType(DayTimeIntervalType.MINUTE, DayTimeIntervalType.MINUTE)))) + * scala> spark.read.csv(path).show(false) + * +------------------------------+ + * |_c0 | + * +------------------------------+ + * |interval '153722867281' minute| + * +------------------------------+ + * scala> spark.read.schema(schema).csv(path).show(false) + * +-------------------------------+ + * |c1 | + * +-------------------------------+ + * |INTERVAL '-153722867280' MINUTE| + * +-------------------------------+ + * + */ + testSparkResultsAreEqual( + "test read day-time interval overflow file", + spark => readCsv(spark, "day-time-interval-to-be-fix.csv"), + assumeCondition = _ => (false, + "check if issue is fixed: https://issues.apache.org/jira/browse/SPARK-38520") + ) { + df => df + } +} diff --git a/tests/src/test/resources/day-time-interval-to-be-fix.csv b/tests/src/test/resources/day-time-interval-to-be-fix.csv new file mode 100644 index 00000000000..1cb7f3c4a23 --- /dev/null +++ b/tests/src/test/resources/day-time-interval-to-be-fix.csv @@ -0,0 +1,2 @@ +interval '106751992' day,,,,interval '2562047789' hour,,,interval '153722867281' minute,, +interval -'106751992' day,,,,interval -'2562047789' hour,,,interval -'153722867281' minute,, diff --git a/tests/src/test/resources/day-time-interval.csv b/tests/src/test/resources/day-time-interval.csv new file mode 100644 index 00000000000..69d0e8433dc --- /dev/null +++ b/tests/src/test/resources/day-time-interval.csv @@ -0,0 +1,43 @@ +INTerval '1' dAY,inTErvAL '1 1' dAY tO HOUR,interval '1 1:1' dAY TO MInUTe,iNtErVaL '1 1:1:1' dAy To seconD,iNTerVaL '1' hoUr,InterVAl '1:1' HouR to mInUtE,iNtERVAl '1:1:1' HOUR to SECoND,InTerVAL '1' minUtE,INterVaL '1:1' mInUTe TO SeconD,INTeRVal '1' SECond +INTerval '0' dAY,inTErvAL '0 0' dAY tO HOUR,interval '0 0:0' dAY TO MInUTe,iNtErVaL '0 0:0:0' dAy To seconD,iNTerVaL '0' hoUr,InterVAl '0:0' HouR to mInUtE,iNtERVAl '0:0:0' HOUR to SECoND,InTerVAL '0' minUtE,INterVaL '0:0' mInUTe TO SeconD,INTeRVal '0' SECond +iNTeRvAL '+1' daY,iNTerVAL '+1 1' Day tO hOUr,iNTeRVaL '+1 1:1' daY to MinuTE,INtERVal '+1 1:1:1' DaY To SECoNd,INtErVaL '+1' hOUr,intErval '+1:1' HoUr tO minUte,INtERVal '+1:1:1' hOur To seCond,iNTERVaL '+1' mInUte,iNtERvAL '+1:1' mInuTE tO sECONd,iNtErvAl '+1' SEcONd +iNTeRvAL '+0' daY,iNTerVAL '+0 0' Day tO hOUr,iNTeRVaL '+0 0:0' daY to MinuTE,INtERVal '+0 0:0:0' DaY To SECoNd,INtErVaL '+0' hOUr,intErval '+0:0' HoUr tO minUte,INtERVal '+0:0:0' hOur To seCond,iNTERVaL '+0' mInUte,iNtERvAL '+0:0' mInuTE tO sECONd,iNtErvAl '+0' SEcONd +INTERval '-1' day,inTeRval '-1 1' Day to hOUr,intErvAL '-1 1:1' Day to mINUte,INTeRVaL '-1 1:1:1' day to SECoNd,iNTeRvAl '-1' HOur,intErVAl '-1:1' HOUR to mInute,InteRvAl '-1:1:1' HoUR to secOnd,intervaL '-1' miNUTe,iNterVAL '-1:1' MinUte tO SECoND,iNTerval '-1' SECoNd +INTERval '-0' day,inTeRval '-0 0' Day to hOUr,intErvAL '-0 0:0' Day to mINUte,INTeRVaL '-0 0:0:0' day to SECoNd,iNTeRvAl '-0' HOur,intErVAl '-0:0' HOUR to mInute,InteRvAl '-0:0:0' HoUR to secOnd,intervaL '-0' miNUTe,iNterVAL '-0:0' MinUte tO SECoND,iNTerval '-0' SECoNd +InteRvAl +'1' dAY,InTerVAL +'1 1' daY tO houR,inTeRvAL +'1 1:1' dAy to mInUte,iNTeRvAL +'1 1:1:1' DaY to SEconD,INTeRval +'1' hoUr,interVal +'1:1' HOUr tO miNutE,inteRvAl +'1:1:1' HOuR To SEConD,IntERVAL +'1' MinuTE,interval +'1:1' MiNuTE to secONd,iNTerVaL +'1' sEcOnD +InteRvAl +'0' dAY,InTerVAL +'0 0' daY tO houR,inTeRvAL +'0 0:0' dAy to mInUte,iNTeRvAL +'0 0:0:0' DaY to SEconD,INTeRval +'0' hoUr,interVal +'0:0' HOUr tO miNutE,inteRvAl +'0:0:0' HOuR To SEConD,IntERVAL +'0' MinuTE,interval +'0:0' MiNuTE to secONd,iNTerVaL +'0' sEcOnD +iNtERVal +'+1' day,IntErval +'+1 1' dAY to hoUr,IntErVAl +'+1 1:1' day TO minute,InterVaL +'+1 1:1:1' DAy to seCOnd,InTeRval +'+1' hOUr,InTERVAL +'+1:1' HOur To MInutE,iNtErVAL +'+1:1:1' HoUr TO sEcOnd,iNTERVaL +'+1' mInute,iNtErVAl +'+1:1' MInutE to SecoNd,InTeRvaL +'+1' seCOnD +iNtERVal +'+0' day,IntErval +'+0 0' dAY to hoUr,IntErVAl +'+0 0:0' day TO minute,InterVaL +'+0 0:0:0' DAy to seCOnd,InTeRval +'+0' hOUr,InTERVAL +'+0:0' HOur To MInutE,iNtErVAL +'+0:0:0' HoUr TO sEcOnd,iNTERVaL +'+0' mInute,iNtErVAl +'+0:0' MInutE to SecoNd,InTeRvaL +'+0' seCOnD +iNTeRVAL +'-1' dAy,INTErVaL +'-1 1' daY tO hOur,INtERVal +'-1 1:1' DaY to mInutE,iNtERvAL +'-1 1:1:1' dAy tO sEcoND,INtErVaL +'-1' hoUr,INTErvAl +'-1:1' HouR TO mINUTe,intervAL +'-1:1:1' Hour To SecOnd,InterVal +'-1' mINutE,InteRVAl +'-1:1' MInuTE tO SeCOnd,inTerval +'-1' seCOnd +iNTeRVAL +'-0' dAy,INTErVaL +'-0 0' daY tO hOur,INtERVal +'-0 0:0' DaY to mInutE,iNtERvAL +'-0 0:0:0' dAy tO sEcoND,INtErVaL +'-0' hoUr,INTErvAl +'-0:0' HouR TO mINUTe,intervAL +'-0:0:0' Hour To SecOnd,InterVal +'-0' mINutE,InteRVAl +'-0:0' MInuTE tO SeCOnd,inTerval +'-0' seCOnd +InTErvAl -'1' daY,IntERVAL -'1 1' DAY To hOur,INTErvAL -'1 1:1' DaY tO MiNuTe,intERvAL -'1 1:1:1' dAY tO secOnd,InTerVaL -'1' hour,INTerVAl -'1:1' hOUR To MINUTE,inteRVAl -'1:1:1' hoUR To SeCond,iNterVaL -'1' MInUTe,INtErvAl -'1:1' mInuTe To secoND,INTErvAl -'1' sECOnD +InTErvAl -'0' daY,IntERVAL -'0 0' DAY To hOur,INTErvAL -'0 0:0' DaY tO MiNuTe,intERvAL -'0 0:0:0' dAY tO secOnd,InTerVaL -'0' hour,INTerVAl -'0:0' hOUR To MINUTE,inteRVAl -'0:0:0' hoUR To SeCond,iNterVaL -'0' MInUTe,INtErvAl -'0:0' mInuTe To secoND,INTErvAl -'0' sECOnD +iNtErval -'+1' daY,InTERvAl -'+1 1' DAY to Hour,iNtervaL -'+1 1:1' Day To mINutE,inTERVAL -'+1 1:1:1' day tO SEcoNd,inteRvAL -'+1' hOUR,INTERval -'+1:1' HOuR TO miNute,IntervAl -'+1:1:1' hOur to sECoND,INtErVaL -'+1' mInUTe,iNTeRVAL -'+1:1' MiNutE TO SecoND,INtERVAL -'+1' SeCONd +iNtErval -'+0' daY,InTERvAl -'+0 0' DAY to Hour,iNtervaL -'+0 0:0' Day To mINutE,inTERVAL -'+0 0:0:0' day tO SEcoNd,inteRvAL -'+0' hOUR,INTERval -'+0:0' HOuR TO miNute,IntervAl -'+0:0:0' hOur to sECoND,INtErVaL -'+0' mInUTe,iNTeRVAL -'+0:0' MiNutE TO SecoND,INtERVAL -'+0' SeCONd +iNTERVal -'-1' Day,INteRval -'-1 1' dAy TO hOUR,iNTERVaL -'-1 1:1' dAY to miNUTE,iNTeRVaL -'-1 1:1:1' dAy to secoNd,iNTeRval -'-1' hOuR,iNtervaL -'-1:1' HOur tO MinuTE,iNTErVAl -'-1:1:1' HOUR to SEcOND,INTeRvAL -'-1' MInUtE,iNtERVal -'-1:1' MINUte tO SecoND,intERvAL -'-1' SeConD +iNTERVal -'-0' Day,INteRval -'-0 0' dAy TO hOUR,iNTERVaL -'-0 0:0' dAY to miNUTE,iNTeRVaL -'-0 0:0:0' dAy to secoNd,iNTeRval -'-0' hOuR,iNtervaL -'-0:0' HOur tO MinuTE,iNTErVAl -'-0:0:0' HOUR to SEcOND,INTeRvAL -'-0' MInUtE,iNtERVal -'-0:0' MINUte tO SecoND,intERvAL -'-0' SeConD +InTerVAl '106751991' DAY,INTErvAL '106751991 1' day to HOUR,INtERVAL '106751991 1:1' dAY tO MiNUTe,iNTeRVal '106751991 1:1:1' DaY tO sEcoNd,InteRVaL '2562047788' hOUR,iNterval '2562047788:1' hOur TO miNuTE,iNTERVAL '2562047788:1:1' hOur TO sECOnd,INteRVAl '153722867280' MiNutE,InTeRVaL '153722867280:1' mInuTe to SECond,iNTErVal '9223372036854' SecoNd +inteRVaL '+106751991' dAY,INTERvAl '+106751991 1' DAy tO hoUr,InTeRVAl '+106751991 1:1' DAy To MiNUTE,inTervAL '+106751991 1:1:1' DaY TO secoNd,inTErVAL '+2562047788' HOur,inteRVAl '+2562047788:1' hoUR TO MINUtE,INTERvaL '+2562047788:1:1' HOuR tO SEcoND,InterVaL '+153722867280' miNutE,interval '+153722867280:1' MiNUte tO SecOND,iNterVaL '+9223372036854' SEcoNd +INtervAL '-106751991' dAY,INTERvAL '-106751991 1' day to hOur,InTeRval '-106751991 1:1' daY tO mINUte,iNTErval '-106751991 1:1:1' dAy tO SeCONd,inTERVaL '-2562047788' Hour,intErVAL '-2562047788:1' hour to mINUte,INTErVal '-2562047788:1:1' hOUr TO second,inTeRVal '-153722867280' MiNuTE,intErvAl '-153722867280:1' minUte To SeCONd,InTervAl '-9223372036854' SeCoND +INtERVAl +'106751991' DaY,INteRVAL +'106751991 1' DAy to HOur,InterVAL +'106751991 1:1' DAY TO mINuTe,intERVAL +'106751991 1:1:1' DAY tO SeCoND,inTErVal +'2562047788' houR,iNTERVal +'2562047788:1' hoUR tO mInUTE,INTErVaL +'2562047788:1:1' HOUr TO SEcond,iNteRvAl +'153722867280' MinUTE,interVAl +'153722867280:1' minute TO seCONd,iNterVAl +'9223372036854' sECoND +iNterVAl +'+106751991' day,iNTeRVaL +'+106751991 1' DaY To hOuR,intErvAL +'+106751991 1:1' dAy TO MINUTE,iNterVaL +'+106751991 1:1:1' DAy TO SECONd,INtErvAl +'+2562047788' HOUR,INTErvAl +'+2562047788:1' hOuR To MiNute,INtErVAl +'+2562047788:1:1' Hour To seCONd,INTeRval +'+153722867280' MiNUte,iNtErVAl +'+153722867280:1' mINuTE tO SEcOnD,iNteRvAL +'+9223372036854' SECOND +INTervaL +'-106751991' day,INTErvAl +'-106751991 1' dAy TO hoUR,intERVAL +'-106751991 1:1' Day TO mInUTe,InTeRVAL +'-106751991 1:1:1' dAY tO sEcOnd,InTervAL +'-2562047788' hour,INtERVal +'-2562047788:1' HOUr to MINUtE,inTervAl +'-2562047788:1:1' hoUr tO SecOND,inTERVal +'-153722867280' MInutE,iNTERVAl +'-153722867280:1' miNUTe TO SEcOnD,INtErvaL +'-9223372036854' SeCONd +iNtERvAl -'106751991' Day,InTeRVAl -'106751991 1' DaY tO hOUr,interVAL -'106751991 1:1' day to MINUTE,IntErvaL -'106751991 1:1:1' day tO SecoNd,iNTerVAl -'2562047788' HOuR,interVaL -'2562047788:1' HOur TO minutE,interval -'2562047788:1:1' HOur to secoNd,INTerVaL -'153722867280' mINUTE,InTerVaL -'153722867280:1' miNute To seCOnd,INTerval -'9223372036854' SEcONd +iNTeRvAL -'+106751991' dAy,INteRVAl -'+106751991 1' daY tO hOur,iNtERVAL -'+106751991 1:1' dAY TO mInUte,iNTERVal -'+106751991 1:1:1' DAY To sECONd,iNTeRvAl -'+2562047788' HoUR,iNTeRVAl -'+2562047788:1' HouR to mInutE,inTervAL -'+2562047788:1:1' hOur tO seConD,IntErVal -'+153722867280' mINute,InTERvAl -'+153722867280:1' MINUTe To SECond,INTerval -'+9223372036854' SecoND +IntERvaL -'-106751991' dAY,iNtErval -'-106751991 1' Day To HouR,iNTerVAl -'-106751991 1:1' daY TO MINute,InTeRvAl -'-106751991 1:1:1' dAy tO SECONd,iNTervAL -'-2562047788' hoUr,inTERVAL -'-2562047788:1' HOUr To minutE,inTErVal -'-2562047788:1:1' hour To SeCoND,IntErVal -'-153722867280' miNuTe,inTERVAl -'-153722867280:1' MINUTe To SeCoNd,iNTerVAL -'-9223372036854' SECONd +iNTerVAl '1' dAY,InTErVaL '1 24' DaY TO hOur,InterVAL '1 1:60' dAy To mInUTe,INTeRvaL '1 24:1:1' day to SecOND,iNteRVaL '1' HOUr,iNtervAL '1:60' hOUR To MiNutE,InTErvAL '1:60:1' hour To SeCOnD,intERvaL '1' mINUtE,intERVAl '1:1' mINutE tO sEcond,iNTErVAl '1' SeCOND +intERVAl '+1' dAy,IntERVaL '+1 24' dAy TO houR,InTERvAL '+1 1:60' dAY TO mINute,inTErval '+1 24:1:1' DAy TO secOnd,INtErvaL '+1' hour,iNtErVAl '+1:60' hOUr to minute,INterVaL '+1:60:1' hOUr To sECOnD,iNTERVAL '+1' mInutE,inteRvAL '+1:1' Minute to SeCoNd,inTeRVAl '+1' SEConD +IntERVal '-1' dAY,INTERvAl '-1 24' day To hOur,iNTeRvaL '-1 1:60' dAY tO minUTe,INteRVal '-1 24:1:1' day To SEconD,iNTervaL '-1' HOUR,iNTeRVaL '-1:60' HOUR tO MinUte,iNTerVal '-1:60:1' hOUR to SeconD,INTerval '-1' MInuTE,inteRvAL '-1:1' MinUTE To SeCoND,InTErval '-1' secoNd +INterVal +'1' dAy,InteRvaL +'1 24' Day TO HOUr,inTervAl +'1 1:60' day TO mINute,INterval +'1 24:1:1' dAY tO SEcoND,iNtErvAl +'1' Hour,interVal +'1:60' hOUR To minuTe,iNtErVAl +'1:60:1' hOUr To seCOND,IntErVaL +'1' miNutE,inteRVAL +'1:1' mInutE To Second,INtErVal +'1' sECond +INteRval +'+1' DAY,INtERVaL +'+1 24' dAY TO HoUR,INterval +'+1 1:60' Day To MInuTE,INTerVal +'+1 24:1:1' DaY to SEcoNd,inTERVaL +'+1' Hour,inTERVAl +'+1:60' hoUR to minUte,IntERVal +'+1:60:1' houR tO SeCOND,INTeRVaL +'+1' MInuTE,inTERvAl +'+1:1' minUtE TO seCOnD,intERVAL +'+1' SeCONd +inTErVAl +'-1' day,iNTERVaL +'-1 24' DAy TO HOuR,inTErVaL +'-1 1:60' dAy TO MiNuTe,iNtErvAl +'-1 24:1:1' dAY tO SeCOND,IntErvAl +'-1' hOUr,intERvAl +'-1:60' hour TO mINuTe,iNtERVal +'-1:60:1' hOUR tO sEcoNd,intErVAl +'-1' mINuTE,INTeRVaL +'-1:1' mInuTE tO SeConD,InTErVal +'-1' sECONd +inTeRvAL -'1' DAy,inteRVAl -'1 24' daY tO hOUr,IntErVal -'1 1:60' DaY tO mInute,intErval -'1 24:1:1' day To seCoNd,IntErVaL -'1' hOUR,InteRVal -'1:60' hOur tO mINUTE,iNtErval -'1:60:1' HoUR TO sECOnd,INTERvaL -'1' MINute,intErvAL -'1:1' mInUtE tO secOND,InTerVAl -'1' SEConD +IntervAl -'+1' dAy,InTeRval -'+1 24' dAy To hoUr,inteRVAl -'+1 1:60' DAy TO mInuTe,IntErVAL -'+1 24:1:1' daY to seCoND,iNterVAL -'+1' HoUr,inTErvaL -'+1:60' hOUr to MiNUTE,interVAl -'+1:60:1' hoUr tO SEconD,iNtERVAL -'+1' miNUTe,inteRvaL -'+1:1' minute To second,intErvAL -'+1' SeCoND +INTErVAL -'-1' dAY,inTErVal -'-1 24' Day to hoUr,InTErvAL -'-1 1:60' DAY TO minUTe,iNTErVAL -'-1 24:1:1' dAy to secOND,InTeRvAl -'-1' hOuR,INTeRVAl -'-1:60' hoUR to MinuTE,IntervAL -'-1:60:1' hoUR To SEcoNd,inTErVAl -'-1' miNute,INTErVAl -'-1:1' MINutE TO SeCoND,iNterVAl -'-1' sEcONd +InTerVAL '1' daY,iNteRvaL '1 24' dAY to HOUr,IntERvAL '1 1:60' DAy To MInUTe,INterVAl '106751991 4:0:54.775807' DAY to SeCoND,intErvaL '1' hoUr,INtervaL '1:60' hOuR TO mINutE,InteRVAL '1:60:1' HOur tO SeconD,iNTErvAL '1' MINutE,iNTERVAL '1:1' mInUte To SEcond,INTerval '1' SecoND +iNtErVAl '1' DAY,INtErval '1 24' DaY to HOUR,INTErVaL '1 1:60' daY To MiNUte,InTerVAl '-106751991 4:0:54.775808' DAY To SecoND,iNTeRvAL '1' hOuR,InteRvaL '1:60' hoUr TO MInUTE,inTErVal '1:60:1' hOUR to sEcONd,INteRvAL '1' minUte,iNTeRVAl '1:1' mINUTE to SEcoND,InTErvAL '1' SEconD +InTeRVAl '1' dAY,InTerval '1 24' daY TO hoUR,INTERvAL '1 1:60' dAY tO MINUtE,INtERvAl '106751991 4:0:54.775808' daY To seCond,InterVAL '1' HoUR,INteRval '1:60' hOUR tO MiNUTe,inteRvAl '1:60:1' hOuR TO SEcOnD,INtERVal '1' MInuTe,inteRvaL '1:1' MINuTe to sECoNd,iNtERvaL '1' seCond +INTerVal '1' daY,INtERVal '1 24' DaY to HouR,IntERVAL '1 1:60' DAy tO mInUTe,iNtERVal '-106751991 4:0:54.775809' dAY TO SeCond,inTerVAL '1' hOur,iNTeRvAl '1:60' HOuR tO MINute,iNTervAl '1:60:1' hour to SeCoNd,intERvAL '1' miNUTE,IntErVal '1:1' mInUte To SecoNd,InteRval '1' SeCOnd +,,,,,,,,,interval '9223372036855' second +,,,,,,,,,interval -'9223372036855' second +INTerval '1 dAY,inTErvAL '1 1 dAY tO HOUR,interval '1 1:1 dAY TO MInUTe,iNtErVaL '1 1:1:1 dAy To seconD,iNTerVaL '1 hoUr,InterVAl '1:1 HouR to mInUtE,iNtERVAl '1:1:1 HOUR to SECoND,InTerVAL '1 minUtE,INterVaL '1:1 mInUTe TO SeconD,INTeRVal '1 SECond