Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -218,12 +218,7 @@ object IntervalUtils {
minutes = toLongWithRange("second", m.group(7), 0, 59)
}
// Hive allow nanosecond precision interval
val nanoStr = if (m.group(9) == null) {
null
} else {
(m.group(9) + "000000000").substring(0, 9)
}
var nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L)
var nanos = parseNanos(m.group(9), seconds < 0)
to match {
case "hour" =>
minutes = 0
Expand Down Expand Up @@ -292,6 +287,13 @@ object IntervalUtils {
new CalendarInterval(months, microseconds)
}

private def parseNanos(nanosStr: String, isNegative: Boolean): Long = {
val alignedStr = if (nanosStr == null) nanosStr else (nanosStr + "000000000").substring(0, 9)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shall we return 0 immediately if nanosStr is null? It's easier to read than relying on toLongWithRange to return 0.

val nanos = toLongWithRange("nanosecond", alignedStr, 0L, 999999999L)
val micros = nanos / DateTimeUtils.NANOS_PER_MICROS
if (isNegative) -micros else micros
}

/**
* Parse second_nano string in ss.nnnnnnnnn format to microseconds
*/
Expand All @@ -303,15 +305,13 @@ object IntervalUtils {
Long.MinValue / DateTimeUtils.MICROS_PER_SECOND,
Long.MaxValue / DateTimeUtils.MICROS_PER_SECOND) * DateTimeUtils.MICROS_PER_SECOND
}
def parseNanos(nanosStr: String): Long = {
toLongWithRange("nanosecond", nanosStr, 0L, 999999999L) / DateTimeUtils.NANOS_PER_MICROS
}

secondNano.split("\\.") match {
case Array(secondsStr) => parseSeconds(secondsStr)
case Array("", nanosStr) => parseNanos(nanosStr)
case Array("", nanosStr) => parseNanos(nanosStr, false)
case Array(secondsStr, nanosStr) =>
Math.addExact(parseSeconds(secondsStr), parseNanos(nanosStr))
val seconds = parseSeconds(secondsStr)
Math.addExact(seconds, parseNanos(nanosStr, seconds < 0))
case _ =>
throw new IllegalArgumentException(
"Interval string does not match second-nano format of ss.nnnnnnnnn")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils}
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, IntervalUtils}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval
Expand Down Expand Up @@ -628,7 +628,17 @@ class ExpressionParserSuite extends AnalysisTest {

// Hive nanosecond notation.
checkIntervals("13.123456789 seconds", intervalLiteral("second", "13.123456789"))
checkIntervals("-13.123456789 second", intervalLiteral("second", "-13.123456789"))
checkIntervals(
"-13.123456789 second",
Literal(new CalendarInterval(
0,
-13 * DateTimeUtils.MICROS_PER_SECOND - 123 * DateTimeUtils.MICROS_PER_MILLIS - 456)))
checkIntervals(
"13.123456 second",
Literal(new CalendarInterval(
0,
13 * DateTimeUtils.MICROS_PER_SECOND + 123 * DateTimeUtils.MICROS_PER_MILLIS + 456)))
checkIntervals("1.001 second", Literal(IntervalUtils.fromString("1 second 1 millisecond")))

// Non Existing unit
intercept("interval 10 nanoseconds",
Expand Down
28 changes: 14 additions & 14 deletions sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
Original file line number Diff line number Diff line change
Expand Up @@ -61,9 +61,9 @@ select
interval '13' second,
interval '13.123456789' second
-- !query 4 schema
struct<interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds:interval,interval 1 weeks 3 days:interval,interval 11 hours:interval,interval 12 minutes:interval,interval 13 seconds:interval,interval 13 seconds 123 milliseconds 456 microseconds:interval>
struct<interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 microseconds:interval,interval 1 weeks 3 days:interval,interval 11 hours:interval,interval 12 minutes:interval,interval 13 seconds:interval,interval 13 seconds 123 milliseconds 456 microseconds:interval>
-- !query 4 output
interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds
interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds


-- !query 5
Expand All @@ -75,9 +75,9 @@ select
'13' second,
'13.123456789' second
-- !query 5 schema
struct<interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds:interval,interval 1 weeks 3 days:interval,interval 11 hours:interval,interval 12 minutes:interval,interval 13 seconds:interval,interval 13 seconds 123 milliseconds 456 microseconds:interval>
struct<interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 microseconds:interval,interval 1 weeks 3 days:interval,interval 11 hours:interval,interval 12 minutes:interval,interval 13 seconds:interval,interval 13 seconds 123 milliseconds 456 microseconds:interval>
-- !query 5 output
interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds
interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds


-- !query 6
Expand Down Expand Up @@ -204,7 +204,7 @@ select
interval '99 11:22:33.123456789' day to second + dateval
from interval_arithmetic
-- !query 15 schema
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds) AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS DATE):date>
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds) AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS DATE):date>
-- !query 15 output
2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09

Expand All @@ -220,7 +220,7 @@ select
'99 11:22:33.123456789' day to second + dateval
from interval_arithmetic
-- !query 16 schema
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds) AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS DATE):date>
struct<dateval:date,CAST(CAST(dateval AS TIMESTAMP) - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds) AS DATE):date,CAST(CAST(dateval AS TIMESTAMP) + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS DATE):date>
-- !query 16 output
2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09

Expand All @@ -236,9 +236,9 @@ select
interval '99 11:22:33.123456789' day to second + tsval
from interval_arithmetic
-- !query 17 schema
struct<tsval:timestamp,CAST(tsval - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS TIMESTAMP):timestamp,CAST(tsval - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS TIMESTAMP):timestamp,CAST(tsval + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds) AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS TIMESTAMP):timestamp>
struct<tsval:timestamp,CAST(tsval - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS TIMESTAMP):timestamp,CAST(tsval - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS TIMESTAMP):timestamp,CAST(tsval + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds) AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS TIMESTAMP):timestamp>
-- !query 17 output
2012-01-01 00:00:00 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 2012-04-09 12:22:33.123456 2011-09-23 13:37:26.876544 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456
2012-01-01 00:00:00 2011-09-23 13:37:26.999877 2012-04-09 12:22:33.000123 2012-04-09 12:22:33.000123 2011-09-23 13:37:26.999877 2011-09-23 13:37:26.999877 2012-04-09 12:22:33.000123


-- !query 18
Expand All @@ -252,9 +252,9 @@ select
'99 11:22:33.123456789' day to second + tsval
from interval_arithmetic
-- !query 18 schema
struct<tsval:timestamp,CAST(tsval - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS TIMESTAMP):timestamp,CAST(tsval - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 milliseconds -456 microseconds AS TIMESTAMP):timestamp,CAST(tsval + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds) AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds AS TIMESTAMP):timestamp>
struct<tsval:timestamp,CAST(tsval - interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS TIMESTAMP):timestamp,CAST(tsval - interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS TIMESTAMP):timestamp,CAST(tsval + interval -14 weeks -1 days -11 hours -22 minutes -33 seconds -123 microseconds AS TIMESTAMP):timestamp,CAST(tsval + (- interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds) AS TIMESTAMP):timestamp,CAST(tsval + interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds AS TIMESTAMP):timestamp>
-- !query 18 output
2012-01-01 00:00:00 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 2012-04-09 12:22:33.123456 2011-09-23 13:37:26.876544 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456
2012-01-01 00:00:00 2011-09-23 13:37:26.999877 2012-04-09 12:22:33.000123 2012-04-09 12:22:33.000123 2011-09-23 13:37:26.999877 2011-09-23 13:37:26.999877 2012-04-09 12:22:33.000123


-- !query 19
Expand All @@ -263,9 +263,9 @@ select
interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
from interval_arithmetic
-- !query 19 schema
struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval>
struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 microseconds):interval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 microseconds):interval>
-- !query 19 output
interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds
interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds


-- !query 20
Expand All @@ -274,9 +274,9 @@ select
'99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second
from interval_arithmetic
-- !query 20 schema
struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval>
struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 microseconds):interval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 microseconds):interval>
-- !query 20 output
interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds
interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds


-- !query 21
Expand Down