Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -1877,7 +1877,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
IntervalUtils.fromString(value)
} catch {
case e: IllegalArgumentException =>
val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx)
val ex = new ParseException(s"Cannot parse the INTERVAL value: $value\n" +
e.getMessage, ctx)
ex.setStackTrace(e.getStackTrace)
throw ex
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,14 +36,6 @@ import org.apache.spark.unsafe.types.CalendarInterval
*/
abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Logging {

/**
* Creates [[CalendarInterval]] for a given SQL String. Throws [[ParseException]] if the SQL
* string is not a valid interval format.
*/
def parseInterval(sqlText: String): CalendarInterval = parse(sqlText) { parser =>
astBuilder.visitSingleInterval(parser.singleInterval())
}

/** Creates/Resolves DataType for a given SQL string. */
override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser =>
astBuilder.visitSingleDataType(parser.singleDataType())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,11 @@

package org.apache.spark.sql.catalyst.util

import java.util.Locale
import java.util.concurrent.TimeUnit

import scala.util.control.NonFatal

import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException}
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.types.Decimal
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
Expand Down Expand Up @@ -101,34 +101,6 @@ object IntervalUtils {
Decimal(result, 18, 6)
}

/**
* Converts a string to [[CalendarInterval]] case-insensitively.
*
* @throws IllegalArgumentException if the input string is not in valid interval format.
*/
def fromString(str: String): CalendarInterval = {
if (str == null) throw new IllegalArgumentException("Interval string cannot be null")
try {
CatalystSqlParser.parseInterval(str)
} catch {
case e: ParseException =>
val ex = new IllegalArgumentException(s"Invalid interval string: $str\n" + e.message)
ex.setStackTrace(e.getStackTrace)
throw ex
}
}

/**
* A safe version of `fromString`. It returns null for invalid input string.
*/
def safeFromString(str: String): CalendarInterval = {
Copy link
Member Author

@yaooqinn yaooqinn Nov 10, 2019

Choose a reason for hiding this comment

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

This method is removed because never used

try {
fromString(str)
} catch {
case _: IllegalArgumentException => null
}
}

private def toLongWithRange(
fieldName: IntervalUnit,
s: String,
Expand Down Expand Up @@ -250,6 +222,78 @@ object IntervalUtils {
}
}

private val isYear: String => Boolean =
y => """y((r)|(rs)|(ear)|(ears))?""".r.pattern.matcher(y).matches()
private val isMonth: String => Boolean =
mon => """mon((s)|(th)|(ths))?""".r.pattern.matcher(mon).matches()
private val isWeek: String => Boolean =
w => """w((eek)|(eeks))?""".r.pattern.matcher(w).matches()
private val isDay: String => Boolean =
d => """d((ay)|(ays))?""".r.pattern.matcher(d).matches()
private val isHour: String => Boolean =
h => """h((r)|(rs)|(our)|(ours))?""".r.pattern.matcher(h).matches()
private val isMinute: String => Boolean =
m => """m((in)|(ins)|(inute)|(inutes))?""".r.pattern.matcher(m).matches()
private val isSecond: String => Boolean =
s => """s((ec)|(ecs)|(econd)|(econds))?""".r.pattern.matcher(s).matches()
private val isMs: String => Boolean =
ms => """(ms((ec)|(ecs)|(econds))?|(millisecond)[s]?)""".r.pattern.matcher(ms).matches()
private val isUs: String => Boolean =
us => """(us((ec)|(ecs)|(econds))?|(microsecond)[s]?)""".r.pattern.matcher(us).matches()

/**
* Converts a string with multiple value unit pairs to [[CalendarInterval]] case-insensitively.
*
* @throws IllegalArgumentException if the input string is not in valid interval format.
*/
def fromString(str: String): CalendarInterval = {
if (str == null) throw new IllegalArgumentException("Interval multi unit string cannot be null")
var months: Int = 0
var days: Int = 0
var us: Long = 0L
val unitValuePart = str.trim.stripPrefix("interval ").trim
var array = "-\\s+".r.replaceAllIn(unitValuePart, "-").toLowerCase(Locale.ROOT)
.split("\\s+").filter(_ != "+").toList
require(array.nonEmpty && array.length % 2 == 0,
"Interval string should be value and unit pairs")

try {
while (array.nonEmpty) {
array match {
case valueStr :: unit :: tail =>
if (isYear(unit)) {
months = Math.addExact(months, Math.multiplyExact(valueStr.toInt, MONTHS_PER_YEAR))
} else if (isMonth(unit)) {
months = Math.addExact(months, valueStr.toInt)
} else if (isWeek(unit)) {
days = Math.addExact(days, Math.multiplyExact(valueStr.toInt, DAYS_PER_WEEK))
} else if (isDay(unit)) {
days = Math.addExact(days, valueStr.toInt)
} else if (isHour(unit)) {
us = Math.addExact(us, Math.multiplyExact(valueStr.toLong, MICROS_PER_HOUR))
} else if (isMinute(unit)) {
us = Math.addExact(us, Math.multiplyExact(valueStr.toLong, MICROS_PER_MINUTE))
} else if (isSecond(unit)) {
us = Math.addExact(us, parseSecondNano(valueStr))
} else if (isMs(unit)) {
us = Math.addExact(us, Math.multiplyExact(valueStr.toLong, MICROS_PER_MILLIS))
} else if (isUs(unit)) {
us = Math.addExact(us, valueStr.toLong)
} else {
throw new IllegalArgumentException(s"Error paring interval unit: $unit")
}
array = tail
case _ => // never reach
}
}
new CalendarInterval(months, days, us)
} catch {
case e: IllegalArgumentException => throw e
case e: Exception =>
throw new IllegalArgumentException(s"Invalid interval string: $str\n" + e.getMessage, e)
}
}

def fromUnitStrings(units: Array[IntervalUnit], values: Array[String]): CalendarInterval = {
assert(units.length == values.length)
var months: Int = 0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,9 +68,11 @@ class IntervalUtilsSuite extends SparkFunSuite {

checkFromInvalidString(null, "cannot be null")

for (input <- Seq("", " ", "interval", "interval1 day", "foo", "foo 1 day")) {
checkFromInvalidString(input, "Invalid interval string")
for (input <- Seq("", " ", "interval", "foo", "foo 1 day")) {
checkFromInvalidString(input, "Interval string should be value and unit pairs")
}

checkFromInvalidString("interval1 day", "For input string")
}


Expand All @@ -79,7 +81,7 @@ class IntervalUtilsSuite extends SparkFunSuite {
"-1 MONTH 1 day -1 microseconds" -> new CalendarInterval(-1, 1, -1),
" 123 MONTHS 123 DAYS 123 Microsecond " -> new CalendarInterval(123, 123, 123),
"interval -1 day +3 Microseconds" -> new CalendarInterval(0, -1, 3),
" interval 8 years -11 months 123 weeks -1 day " +
"interval 8 years -11 months 123 weeks -1 day " +
"23 hours -22 minutes 1 second -123 millisecond 567 microseconds " ->
new CalendarInterval(85, 860, 81480877567L)).foreach { case (input, expected) =>
checkFromString(input, expected)
Expand All @@ -92,8 +94,8 @@ class IntervalUtilsSuite extends SparkFunSuite {
// Allow duplicated units and summarize their values
checkFromString("1 day 10 day", new CalendarInterval(0, 11, 0))
// Only the seconds units can have the fractional part
checkFromInvalidString("1.5 days", "Error parsing interval string")
checkFromInvalidString("1. hour", "Error parsing interval string")
checkFromInvalidString("1.5 days", "1.5")
checkFromInvalidString("1. hour", "1.")
}

test("string to interval: seconds with fractional part") {
Expand All @@ -105,7 +107,7 @@ class IntervalUtilsSuite extends SparkFunSuite {
checkFromString("-1.5 seconds", new CalendarInterval(0, 0, -1500000))
// truncate nanoseconds to microseconds
checkFromString("0.999999999 seconds", new CalendarInterval(0, 0, 999999))
checkFromInvalidString("0.123456789123 seconds", "Error parsing interval string")
checkFromInvalidString("0.123456789123 seconds", "outside range [0, 999999999]")
}

test("from year-month string") {
Expand Down
20 changes: 20 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/interval.sql
Original file line number Diff line number Diff line change
Expand Up @@ -65,3 +65,23 @@ select make_interval(1, 2, 3, 4);
select make_interval(1, 2, 3, 4, 5);
select make_interval(1, 2, 3, 4, 5, 6);
select make_interval(1, 2, 3, 4, 5, 6, 7.008009);

-- abbreviations support for interval multi unit string
select interval '1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseconds 9 microseconds';
select interval '1 year 2 month 3 week 4 day 5 hour 6 minute 7 second 8 millisecond 9 microsecond';
select interval '1 y 2 mon 3 w 4 d 5 h 6 m 7 s 8 ms 9 us';
select interval '1 y 2 yr -1 yrs 3 years -2 mons - 3 w + 4 d +5 hrs 6 mins 7 secs 8 msecs 9 usecs';
select interval 'day 1 y';
select interval '1 1 y';
select interval '1 1 1 y';
select interval '1 daY';
select interval '';
select interval ' ';
select interval '-';
select interval '- year';
select interval '-1 yea';

-- abbreviations not cover these which require units as keywords in sql parser
select interval 1 y 2 mon 3 week;
select interval '30' y '25' mon;
select interval '-1 yrs' year to month;
Loading