diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 28fb64f7cd0e0..184ddac9a71a6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -32,94 +32,11 @@ public final class CalendarInterval implements Serializable { public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24; public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; - /** - * A function to generate regex which matches interval string's unit part like "3 years". - * - * First, we can leave out some units in interval string, and we only care about the value of - * unit, so here we use non-capturing group to wrap the actual regex. - * At the beginning of the actual regex, we should match spaces before the unit part. - * Next is the number part, starts with an optional "-" to represent negative value. We use - * capturing group to wrap this part as we need the value later. - * Finally is the unit name, ends with an optional "s". - */ - private static String unitRegex(String unit) { - return "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?"; - } - - private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + - unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + - unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond"), - Pattern.CASE_INSENSITIVE); - - private static Pattern yearMonthPattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); + private static Pattern yearMonthPattern = Pattern.compile( + "^([+|-])?(\\d+)-(\\d+)$"); private static Pattern dayTimePattern = Pattern.compile( - "^(?:['|\"])?([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); - - private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); - - private static long toLong(String s) { - if (s == null) { - return 0; - } else { - return Long.parseLong(s); - } - } - - /** - * Convert a string to CalendarInterval. Return null if the input string is not a valid interval. - * This method is case-insensitive. - */ - public static CalendarInterval fromString(String s) { - try { - return fromCaseInsensitiveString(s); - } catch (IllegalArgumentException e) { - return null; - } - } - - /** - * Convert a string to CalendarInterval. This method can handle - * strings without the `interval` prefix and throws IllegalArgumentException - * when the input string is not a valid interval. - * - * @throws IllegalArgumentException if the string is not a valid internal. - */ - public static CalendarInterval fromCaseInsensitiveString(String s) { - if (s == null) { - throw new IllegalArgumentException("Interval cannot be null"); - } - String trimmed = s.trim(); - if (trimmed.isEmpty()) { - throw new IllegalArgumentException("Interval cannot be blank"); - } - String prefix = "interval"; - String intervalStr = trimmed; - // Checks the given interval string does not start with the `interval` prefix - if (!intervalStr.regionMatches(true, 0, prefix, 0, prefix.length())) { - // Prepend `interval` if it does not present because - // the regular expression strictly require it. - intervalStr = prefix + " " + trimmed; - } else if (intervalStr.length() == prefix.length()) { - throw new IllegalArgumentException("Interval string must have time units"); - } - - Matcher m = p.matcher(intervalStr); - if (!m.matches()) { - throw new IllegalArgumentException("Invalid interval: " + s); - } - - long months = toLong(m.group(1)) * 12 + toLong(m.group(2)); - long microseconds = toLong(m.group(3)) * MICROS_PER_WEEK; - microseconds += toLong(m.group(4)) * MICROS_PER_DAY; - microseconds += toLong(m.group(5)) * MICROS_PER_HOUR; - microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE; - microseconds += toLong(m.group(7)) * MICROS_PER_SECOND; - microseconds += toLong(m.group(8)) * MICROS_PER_MILLI; - microseconds += toLong(m.group(9)); - return new CalendarInterval((int) months, microseconds); - } + "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$"); public static long toLongWithRange(String fieldName, String s, long minValue, long maxValue) throws IllegalArgumentException { @@ -242,72 +159,59 @@ public static CalendarInterval fromDayTimeString(String s, String from, String t return result; } - public static CalendarInterval fromSingleUnitString(String unit, String s) + public static CalendarInterval fromUnitStrings(String[] units, String[] values) throws IllegalArgumentException { + assert units.length == values.length; + int months = 0; + long microseconds = 0; - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException(String.format("Interval %s string was null", unit)); - } - s = s.trim(); - Matcher m = quoteTrimPattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); - } else { + for (int i = 0; i < units.length; i++) { try { - switch (unit) { + switch (units[i]) { case "year": - int year = (int) toLongWithRange("year", m.group(1), - Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); - result = new CalendarInterval(year * 12, 0L); + months = Math.addExact(months, Math.multiplyExact(Integer.parseInt(values[i]), 12)); break; case "month": - int month = (int) toLongWithRange("month", m.group(1), - Integer.MIN_VALUE, Integer.MAX_VALUE); - result = new CalendarInterval(month, 0L); + months = Math.addExact(months, Integer.parseInt(values[i])); break; case "week": - long week = toLongWithRange("week", m.group(1), - Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); - result = new CalendarInterval(0, week * MICROS_PER_WEEK); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_WEEK)); break; case "day": - long day = toLongWithRange("day", m.group(1), - Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); - result = new CalendarInterval(0, day * MICROS_PER_DAY); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_DAY)); break; case "hour": - long hour = toLongWithRange("hour", m.group(1), - Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); - result = new CalendarInterval(0, hour * MICROS_PER_HOUR); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_HOUR)); break; case "minute": - long minute = toLongWithRange("minute", m.group(1), - Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); - result = new CalendarInterval(0, minute * MICROS_PER_MINUTE); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MINUTE)); break; case "second": { - long micros = parseSecondNano(m.group(1)); - result = new CalendarInterval(0, micros); + microseconds = Math.addExact(microseconds, parseSecondNano(values[i])); break; } case "millisecond": - long millisecond = toLongWithRange("millisecond", m.group(1), - Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); - result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MILLI)); break; - case "microsecond": { - long micros = Long.parseLong(m.group(1)); - result = new CalendarInterval(0, micros); + case "microsecond": + microseconds = Math.addExact(microseconds, Long.parseLong(values[i])); break; - } } } catch (Exception e) { throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e); } } - return result; + return new CalendarInterval(months, microseconds); } /** diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 587071332ce47..9f3262bf2aaa4 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -19,8 +19,6 @@ import org.junit.Test; -import java.util.Arrays; - import static org.junit.Assert.*; import static org.apache.spark.unsafe.types.CalendarInterval.*; @@ -62,72 +60,6 @@ public void toStringTest() { assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); } - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - Arrays.asList( - "interval -5 years 23 month", - " -5 years 23 month", - "interval -5 years 23 month ", - " -5 years 23 month ", - " interval -5 years 23 month ").forEach(input -> - assertEquals(fromString(input), result) - ); - - // Error cases - Arrays.asList( - "interval 3month 1 hour", - "3month 1 hour", - "interval 3 moth 1 hour", - "3 moth 1 hour", - "interval", - "int", - "", - null).forEach(input -> assertNull(fromString(input))); - } - - @Test - public void fromCaseInsensitiveStringTest() { - for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); - } - - for (String input : new String[]{null, "", " "}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - String msg = e.getMessage(); - if (input == null) assertTrue(msg.contains("cannot be null")); - else assertTrue(msg.contains("cannot be blank")); - } - } - - for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - String msg = e.getMessage(); - if (input.trim().equalsIgnoreCase("interval")) { - assertTrue(msg.contains("Interval string must have time units")); - } else { - assertTrue(msg.contains("Invalid interval:")); - } - } - } - } - @Test public void fromYearMonthStringTest() { String input; @@ -194,107 +126,25 @@ public void fromDayTimeStringTest() { } } - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - } - @Test public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); } @Test public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } - - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - Arrays.asList("interval ", "").forEach(prefix -> { - String input1 = prefix + number + " " + unit; - String input2 = prefix + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); - }); - } - - @Test - public void fromStringCaseSensitivityTest() { - testSingleUnit("YEAR", 3, 36, 0); - testSingleUnit("Month", 3, 3, 0); - testSingleUnit("Week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("DAY", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("HouR", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("MiNuTe", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("Second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("MilliSecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("MicroSecond", 3, 0, 3); - - String input; - - input = "INTERVAL -5 YEARS 23 MONTHS"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); + CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - assertNull(fromString("INTERVAL")); - assertNull(fromString(" Interval ")); + input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); } } diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 970d244071e0d..9ee3fade3b96f 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -79,6 +79,10 @@ singleTableSchema : colTypeList EOF ; +singleInterval + : INTERVAL? (intervalValue intervalUnit)+ EOF + ; + statement : query #statementDefault | ctes? dmlStatementNoWith #dmlStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index d71f300dd26dd..862b2bb515a19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -23,7 +23,7 @@ import java.util.Locale import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{InternalRow, WalkedTypePath} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper} object Cast { @@ -466,7 +466,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // IntervalConverter private[this] def castToInterval(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => CalendarInterval.fromString(s.toString)) + buildCast[UTF8String](_, s => IntervalUtils.safeFromString(s.toString)) } // LongConverter @@ -1213,8 +1213,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def castToIntervalCode(from: DataType): CastFunction = from match { case StringType => + val util = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") (c, evPrim, evNull) => - code"""$evPrim = CalendarInterval.fromString($c.toString()); + code"""$evPrim = $util.safeFromString($c.toString()); if(${evPrim} == null) { ${evNull} = true; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 9aae678deb4bc..b9ec933f31493 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( timeColumn: Expression, @@ -102,7 +102,7 @@ object TimeWindow { * precision. */ private def getIntervalInMicroSeconds(interval: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) + val cal = IntervalUtils.fromString(interval) if (cal.months > 0) { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 940dfd0fc333d..b4cbd3f6492fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -100,6 +101,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList))) } + override def visitSingleInterval(ctx: SingleIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val units = ctx.intervalUnit().asScala.map { + u => normalizeInternalUnit(u.getText.toLowerCase(Locale.ROOT)) + }.toArray + val values = ctx.intervalValue().asScala.map(getIntervalValue).toArray + try { + CalendarInterval.fromUnitStrings(units, values) + } catch { + case i: IllegalArgumentException => + val e = new ParseException(i.getMessage, ctx) + e.setStackTrace(i.getStackTrace) + throw e + } + } + } + /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -1770,7 +1788,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => val interval = try { - CalendarInterval.fromCaseInsensitiveString(value) + IntervalUtils.fromString(value) } catch { case e: IllegalArgumentException => val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx) @@ -1930,15 +1948,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) { import ctx._ - val s = value.getText + val s = getIntervalValue(value) try { val unitText = unit.getText.toLowerCase(Locale.ROOT) val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match { - case (u, None) if u.endsWith("s") => - // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... - CalendarInterval.fromSingleUnitString(u.substring(0, u.length - 1), s) case (u, None) => - CalendarInterval.fromSingleUnitString(u, s) + CalendarInterval.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s)) case ("year", Some("month")) => CalendarInterval.fromYearMonthString(s) case ("day", Some("hour")) => @@ -1967,6 +1982,19 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + private def getIntervalValue(value: IntervalValueContext): String = { + if (value.STRING() != null) { + string(value.STRING()) + } else { + value.getText + } + } + + // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... + private def normalizeInternalUnit(s: String): String = { + if (s.endsWith("s")) s.substring(0, s.length - 1) else s + } + /* ******************************************************************************************** * DataType parsing * ******************************************************************************************** */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index a84d29b71ac42..b66cae7979416 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -29,12 +29,21 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.unsafe.types.CalendarInterval /** * Base SQL parsing infrastructure. */ 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()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 78d188f81f628..14fd153e15f58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,21 +17,24 @@ package org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { - val MONTHS_PER_YEAR: Int = 12 - val MONTHS_PER_QUARTER: Byte = 3 - val YEARS_PER_MILLENNIUM: Int = 1000 - val YEARS_PER_CENTURY: Int = 100 - val YEARS_PER_DECADE: Int = 10 - val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS - val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS - val DAYS_PER_MONTH: Byte = 30 - val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + final val MONTHS_PER_YEAR: Int = 12 + final val MONTHS_PER_QUARTER: Byte = 3 + final val YEARS_PER_MILLENNIUM: Int = 1000 + final val YEARS_PER_CENTURY: Int = 100 + final val YEARS_PER_DECADE: Int = 10 + final val MICROS_PER_HOUR: Long = + DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + final val MICROS_PER_MINUTE: Long = + DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + final val DAYS_PER_MONTH: Byte = 30 + final val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY /* 365.25 days per year assumes leap year every four years */ - val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 + final val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -88,4 +91,32 @@ object IntervalUtils { result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) 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 = { + try { + fromString(str) + } catch { + case _: IllegalArgumentException => null + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 603073b40d7aa..e10aa60d52cf8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH @@ -720,7 +720,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours"))), + Literal(IntervalUtils.fromString("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -729,7 +729,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:01")), - Literal(CalendarInterval.fromString("interval 12 hours"))), + Literal(IntervalUtils.fromString("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -738,7 +738,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(IntervalUtils.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -747,7 +747,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(IntervalUtils.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -756,7 +756,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -765,7 +765,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(IntervalUtils.fromString("interval 1 month").negate())), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -774,7 +774,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 day").negate())), + Literal(IntervalUtils.fromString("interval 1 month 1 day").negate())), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -783,7 +783,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-31 00:00:00")), Literal(Timestamp.valueOf("2018-04-30 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-31 00:00:00"), Timestamp.valueOf("2018-02-28 00:00:00"), @@ -793,7 +793,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 second"))), + Literal(IntervalUtils.fromString("interval 1 month 1 second"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:01"))) @@ -801,7 +801,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:04:06")), - Literal(CalendarInterval.fromString("interval 1 month 2 minutes 3 seconds"))), + Literal(IntervalUtils.fromString("interval 1 month 2 minutes 3 seconds"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:02:03"), @@ -839,7 +839,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-25 01:30:00")), Literal(Timestamp.valueOf("2018-03-25 03:30:00")), - Literal(CalendarInterval.fromString("interval 30 minutes"))), + Literal(IntervalUtils.fromString("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-03-25 01:30:00"), Timestamp.valueOf("2018-03-25 03:00:00"), @@ -849,7 +849,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-10-28 01:30:00")), Literal(Timestamp.valueOf("2018-10-28 03:30:00")), - Literal(CalendarInterval.fromString("interval 30 minutes"))), + Literal(IntervalUtils.fromString("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-10-28 01:30:00"), noDST(Timestamp.valueOf("2018-10-28 02:00:00")), @@ -866,7 +866,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-01-05")), - Literal(CalendarInterval.fromString("interval 2 days"))), + Literal(IntervalUtils.fromString("interval 2 days"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-01-03"), @@ -875,7 +875,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-03-01")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-02-01"), @@ -884,7 +884,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-31")), Literal(Date.valueOf("2018-04-30")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Date.valueOf("2018-01-31"), Date.valueOf("2018-02-28"), @@ -905,14 +905,14 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-02")), Literal(Date.valueOf("1970-01-01")), - Literal(CalendarInterval.fromString("interval 1 day"))), + Literal(IntervalUtils.fromString("interval 1 day"))), EmptyRow, "sequence boundaries: 1 to 0 by 1") checkExceptionInExpression[IllegalArgumentException]( new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(IntervalUtils.fromString("interval 1 month").negate())), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * CalendarInterval.MICROS_PER_DAY}") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index e893e863b3675..6abadd77bd41a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.internal.SQLConf @@ -1075,16 +1075,16 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), new CalendarInterval(0, 0)) checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), - CalendarInterval.fromString("interval 18173 days " + + IntervalUtils.fromString("interval 18173 days " + "11 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), - CalendarInterval.fromString("interval -18173 days " + + IntervalUtils.fromString("interval -18173 days " + "-11 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) checkEvaluation( SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), - CalendarInterval.fromString("interval 521722 weeks 4 days " + + IntervalUtils.fromString("interval 521722 weeks 4 days " + "23 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) } @@ -1093,18 +1093,18 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractDates(Literal(end), Literal(end)), new CalendarInterval(0, 0)) checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), - CalendarInterval.fromString("interval 1 days")) + IntervalUtils.fromString("interval 1 days")) checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), - CalendarInterval.fromString("interval -1 days")) + IntervalUtils.fromString("interval -1 days")) val epochDate = Literal(LocalDate.ofEpochDay(0)) checkEvaluation(SubtractDates(Literal(end), epochDate), - CalendarInterval.fromString("interval 49 years 9 months 4 days")) + IntervalUtils.fromString("interval 49 years 9 months 4 days")) checkEvaluation(SubtractDates(epochDate, Literal(end)), - CalendarInterval.fromString("interval -49 years -9 months -4 days")) + IntervalUtils.fromString("interval -49 years -9 months -4 days")) checkEvaluation( SubtractDates( Literal(LocalDate.of(10000, 1, 1)), Literal(LocalDate.of(1, 1, 1))), - CalendarInterval.fromString("interval 9999 years")) + IntervalUtils.fromString("interval 9999 years")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index f90c98be0b3fd..4b2da73abe562 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -30,9 +30,9 @@ import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.types.{ArrayType, StructType, _} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val random = new scala.util.Random @@ -252,7 +252,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for CalendarInterval type") { def checkHiveHashForIntervalType(interval: String, expected: Long): Unit = { - checkHiveHash(CalendarInterval.fromString(interval), CalendarIntervalType, expected) + checkHiveHash(IntervalUtils.fromString(interval), CalendarIntervalType, expected) } // ----- MICROSEC ----- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 078ec88800215..818ee239dbbf8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import scala.language.implicitConversions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.CalendarInterval class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { implicit def interval(s: String): Literal = { - Literal(CalendarInterval.fromString("interval " + s)) + Literal(IntervalUtils.fromString("interval " + s)) } test("millenniums") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 0d594eb10962e..23ba9c6ec7388 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -56,7 +56,7 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { testBothCodegenAndInterpreted("variable-length types") { val proj = createMutableProjection(variableLengthTypes) - val scalaValues = Seq("abc", BigDecimal(10), CalendarInterval.fromString("interval 1 day"), + val scalaValues = Seq("abc", BigDecimal(10), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), new java.lang.Integer(5)) val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index a171885471a36..4ccd4f7ce798d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -35,8 +35,7 @@ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -486,7 +485,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { ("abcd".getBytes, BinaryType), ("abcd", StringType), (BigDecimal.valueOf(10), DecimalType.IntDecimal), - (CalendarInterval.fromString("interval 3 day"), CalendarIntervalType), + (IntervalUtils.fromString("interval 3 day"), CalendarIntervalType), (java.math.BigDecimal.valueOf(10), DecimalType.BigIntDecimal), (Array(3, 2, 1), ArrayType(IntegerType)) ).foreach { case (input, dt) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 323a3a901689f..20e77254ecdad 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -531,7 +531,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // Simple tests val inputRow = InternalRow.fromSeq(Seq( false, 3.toByte, 15.toShort, -83, 129L, 1.0f, 8.0, UTF8String.fromString("test"), - Decimal(255), CalendarInterval.fromString("interval 1 day"), Array[Byte](1, 2) + Decimal(255), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2) )) val fields1 = Array( BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index e6eabcc1f3022..86b3aa8190b45 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -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 +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -432,7 +432,7 @@ class ExpressionParserSuite extends AnalysisTest { intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value") // Interval. - val intervalLiteral = Literal(CalendarInterval.fromString("interval 3 month 1 hour")) + val intervalLiteral = Literal(IntervalUtils.fromString("interval 3 month 1 hour")) assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral) assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") @@ -597,7 +597,7 @@ class ExpressionParserSuite extends AnalysisTest { "microsecond") def intervalLiteral(u: String, s: String): Literal = { - Literal(CalendarInterval.fromSingleUnitString(u, s)) + Literal(CalendarInterval.fromUnitStrings(Array(u), Array(s))) } test("intervals") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala new file mode 100644 index 0000000000000..e48779af3c9aa --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.IntervalUtils.fromString +import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.CalendarInterval._ + +class IntervalUtilsSuite extends SparkFunSuite { + + test("fromString: basic") { + testSingleUnit("YEAR", 3, 36, 0) + testSingleUnit("Month", 3, 3, 0) + testSingleUnit("Week", 3, 0, 3 * MICROS_PER_WEEK) + testSingleUnit("DAY", 3, 0, 3 * MICROS_PER_DAY) + testSingleUnit("HouR", 3, 0, 3 * MICROS_PER_HOUR) + testSingleUnit("MiNuTe", 3, 0, 3 * MICROS_PER_MINUTE) + testSingleUnit("Second", 3, 0, 3 * MICROS_PER_SECOND) + testSingleUnit("MilliSecond", 3, 0, 3 * MICROS_PER_MILLI) + testSingleUnit("MicroSecond", 3, 0, 3) + + for (input <- Seq(null, "", " ")) { + try { + fromString(input) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + val msg = e.getMessage + if (input == null) { + assert(msg.contains("cannot be null")) + } + } + } + + for (input <- Seq("interval", "interval1 day", "foo", "foo 1 day")) { + try { + fromString(input) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + val msg = e.getMessage + assert(msg.contains("Invalid interval string")) + } + } + } + + test("fromString: random order field") { + val input = "1 day 1 year" + val result = new CalendarInterval(12, MICROS_PER_DAY) + assert(fromString(input) == result) + } + + test("fromString: duplicated fields") { + val input = "1 day 1 day" + val result = new CalendarInterval(0, 2 * MICROS_PER_DAY) + assert(fromString(input) == result) + } + + test("fromString: value with +/-") { + val input = "+1 year -1 day" + val result = new CalendarInterval(12, -MICROS_PER_DAY) + assert(fromString(input) == result) + } + + private def testSingleUnit(unit: String, number: Int, months: Int, microseconds: Long): Unit = { + for (prefix <- Seq("interval ", "")) { + val input1 = prefix + number + " " + unit + val input2 = prefix + number + " " + unit + "s" + val result = new CalendarInterval(months, microseconds) + assert(fromString(input1) == result) + assert(fromString(input2) == result) + } + } +} diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt index 6605bd2c13b47..221ac42022a15 100644 --- a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -2,24 +2,24 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -string w/ interval 663 758 99 1.5 663.2 1.0X -string w/o interval 563 582 19 1.8 563.2 1.2X -1 units w/ interval 891 953 97 1.1 891.2 0.7X -1 units w/o interval 894 905 15 1.1 893.6 0.7X -2 units w/ interval 1142 1169 24 0.9 1141.8 0.6X -2 units w/o interval 1195 1200 7 0.8 1194.7 0.6X -3 units w/ interval 1449 1452 3 0.7 1449.5 0.5X -3 units w/o interval 1489 1491 1 0.7 1489.3 0.4X -4 units w/ interval 1688 1690 1 0.6 1688.4 0.4X -4 units w/o interval 1711 1722 11 0.6 1710.9 0.4X -5 units w/ interval 1961 1983 23 0.5 1961.3 0.3X -5 units w/o interval 1983 1991 10 0.5 1983.4 0.3X -6 units w/ interval 2217 2228 11 0.5 2216.9 0.3X -6 units w/o interval 2240 2244 5 0.4 2239.6 0.3X -7 units w/ interval 2454 2468 16 0.4 2454.1 0.3X -7 units w/o interval 2480 2491 15 0.4 2479.5 0.3X -8 units w/ interval 2762 2792 26 0.4 2761.9 0.2X -8 units w/o interval 2763 2778 14 0.4 2762.9 0.2X -9 units w/ interval 3036 3060 21 0.3 3036.4 0.2X -9 units w/o interval 3095 3111 15 0.3 3094.8 0.2X +prepare string w/ interval 672 728 64 1.5 672.1 1.0X +prepare string w/o interval 580 602 19 1.7 580.4 1.2X +1 units w/ interval 9450 9575 138 0.1 9449.6 0.1X +1 units w/o interval 8948 8968 19 0.1 8948.3 0.1X +2 units w/ interval 10947 10966 19 0.1 10947.1 0.1X +2 units w/o interval 10470 10489 26 0.1 10469.5 0.1X +3 units w/ interval 12265 12333 72 0.1 12264.5 0.1X +3 units w/o interval 12001 12004 3 0.1 12000.6 0.1X +4 units w/ interval 13749 13828 69 0.1 13748.5 0.0X +4 units w/o interval 13467 13479 15 0.1 13467.3 0.0X +5 units w/ interval 15392 15446 51 0.1 15392.1 0.0X +5 units w/o interval 15090 15107 29 0.1 15089.7 0.0X +6 units w/ interval 16696 16714 20 0.1 16695.9 0.0X +6 units w/o interval 16361 16366 5 0.1 16361.4 0.0X +7 units w/ interval 18190 18270 71 0.1 18190.2 0.0X +7 units w/o interval 17757 17767 9 0.1 17756.7 0.0X +8 units w/ interval 19821 19870 43 0.1 19820.7 0.0X +8 units w/o interval 19479 19555 97 0.1 19479.5 0.0X +9 units w/ interval 21417 21481 56 0.0 21417.1 0.0X +9 units w/o interval 21058 21131 86 0.0 21058.2 0.0X diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index 40169826cc627..60e8e5198353c 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -2,24 +2,25 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -string w/ interval 600 641 37 1.7 600.3 1.0X -string w/o interval 536 544 12 1.9 536.4 1.1X -1 units w/ interval 1017 1027 9 1.0 1016.8 0.6X -1 units w/o interval 1055 1073 16 0.9 1054.7 0.6X -2 units w/ interval 1272 1292 29 0.8 1272.2 0.5X -2 units w/o interval 1309 1314 9 0.8 1309.0 0.5X -3 units w/ interval 1545 1566 20 0.6 1544.8 0.4X -3 units w/o interval 1606 1610 5 0.6 1605.8 0.4X -4 units w/ interval 1820 1826 6 0.5 1819.7 0.3X -4 units w/o interval 1882 1885 3 0.5 1881.5 0.3X -5 units w/ interval 2039 2043 7 0.5 2038.9 0.3X -5 units w/o interval 2131 2133 3 0.5 2130.6 0.3X -6 units w/ interval 2269 2272 4 0.4 2269.5 0.3X -6 units w/o interval 2327 2333 6 0.4 2327.2 0.3X -7 units w/ interval 2477 2485 10 0.4 2476.8 0.2X -7 units w/o interval 2536 2538 3 0.4 2536.0 0.2X -8 units w/ interval 2764 2781 27 0.4 2763.8 0.2X -8 units w/o interval 2843 2847 5 0.4 2842.9 0.2X -9 units w/ interval 2983 2997 12 0.3 2982.5 0.2X -9 units w/o interval 3071 3072 1 0.3 3071.1 0.2X +prepare string w/ interval 596 647 61 1.7 596.0 1.0X +prepare string w/o interval 530 554 22 1.9 530.2 1.1X +1 units w/ interval 9168 9243 66 0.1 9167.8 0.1X +1 units w/o interval 8740 8744 5 0.1 8740.2 0.1X +2 units w/ interval 10815 10874 52 0.1 10815.0 0.1X +2 units w/o interval 10413 10419 11 0.1 10412.8 0.1X +3 units w/ interval 12490 12530 37 0.1 12490.3 0.0X +3 units w/o interval 12173 12180 9 0.1 12172.8 0.0X +4 units w/ interval 13788 13834 43 0.1 13788.0 0.0X +4 units w/o interval 13445 13456 10 0.1 13445.5 0.0X +5 units w/ interval 15313 15330 15 0.1 15312.7 0.0X +5 units w/o interval 14928 14942 16 0.1 14928.0 0.0X +6 units w/ interval 16959 17003 42 0.1 16959.1 0.0X +6 units w/o interval 16623 16627 5 0.1 16623.3 0.0X +7 units w/ interval 18955 18972 21 0.1 18955.4 0.0X +7 units w/o interval 18454 18462 7 0.1 18454.1 0.0X +8 units w/ interval 20835 20843 8 0.0 20835.4 0.0X +8 units w/o interval 20446 20463 19 0.0 20445.7 0.0X +9 units w/ interval 22981 23031 43 0.0 22981.4 0.0X +9 units w/o interval 22581 22603 25 0.0 22581.1 0.0X + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 076270a9f1c6b..5f6e0a82be4ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -46,6 +46,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ @@ -724,7 +725,7 @@ class Dataset[T] private[sql]( def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { val parsedDelay = try { - CalendarInterval.fromCaseInsensitiveString(delayThreshold) + IntervalUtils.fromString(delayThreshold) } catch { case e: IllegalArgumentException => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index dda9d41f630e6..d191a79187f28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -21,6 +21,7 @@ import java.sql.Date import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.streaming.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout} import org.apache.spark.unsafe.types.CalendarInterval @@ -159,7 +160,7 @@ private[sql] class GroupStateImpl[S] private( def getTimeoutTimestamp: Long = timeoutTimestamp private def parseDuration(duration: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(duration) + val cal = IntervalUtils.fromString(duration) if (cal.milliseconds < 0 || cal.months < 0) { throw new IllegalArgumentException(s"Provided duration ($duration) is not positive") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 2bdb3402c14b1..daa70a12ba0e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -21,8 +21,8 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.Duration +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval private object Triggers { def validate(intervalMs: Long): Unit = { @@ -30,7 +30,7 @@ private object Triggers { } def convert(interval: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) + val cal = IntervalUtils.fromString(interval) if (cal.months > 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 816386c483209..0f95f85237828 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -85,6 +85,19 @@ select timestamp '2016-33-11 20:54:00.000'; -- interval select interval 13.123456789 seconds, interval -13.123456789 second; select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond; +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; +select interval '0 0:0:0.1' day to second; +select interval '10-9' year to month; +select interval '20 15:40:32.99899999' day to hour; +select interval '20 15:40:32.99899999' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40:32.99899999' hour to minute; +select interval '15:40.99899999' hour to second; +select interval '15:40' hour to second; +select interval '15:40:32.99899999' hour to second; +select interval '20 40:32.99899999' minute to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; -- ns is not supported select interval 10 nanoseconds; diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index aef23963da374..fd6e51b2385de 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 46 +-- Number of queries: 59 -- !query 0 @@ -337,10 +337,114 @@ interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseco -- !query 36 -select interval 10 nanoseconds +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 36 schema -struct<> +struct -- !query 36 output +interval 32 years 1 months -14 weeks -6 hours -35 minutes -110 milliseconds -13 microseconds + + +-- !query 37 +select interval '0 0:0:0.1' day to second +-- !query 37 schema +struct +-- !query 37 output +interval 100 milliseconds + + +-- !query 38 +select interval '10-9' year to month +-- !query 38 schema +struct +-- !query 38 output +interval 10 years 9 months + + +-- !query 39 +select interval '20 15:40:32.99899999' day to hour +-- !query 39 schema +struct +-- !query 39 output +interval 2 weeks 6 days 15 hours + + +-- !query 40 +select interval '20 15:40:32.99899999' day to minute +-- !query 40 schema +struct +-- !query 40 output +interval 2 weeks 6 days 15 hours 40 minutes + + +-- !query 41 +select interval '20 15:40:32.99899999' day to second +-- !query 41 schema +struct +-- !query 41 output +interval 2 weeks 6 days 15 hours 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 42 +select interval '15:40:32.99899999' hour to minute +-- !query 42 schema +struct +-- !query 42 output +interval 15 hours 40 minutes + + +-- !query 43 +select interval '15:40.99899999' hour to second +-- !query 43 schema +struct +-- !query 43 output +interval 15 minutes 40 seconds 998 milliseconds 999 microseconds + + +-- !query 44 +select interval '15:40' hour to second +-- !query 44 schema +struct +-- !query 44 output +interval 15 hours 40 minutes + + +-- !query 45 +select interval '15:40:32.99899999' hour to second +-- !query 45 schema +struct +-- !query 45 output +interval 15 hours 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 46 +select interval '20 40:32.99899999' minute to second +-- !query 46 schema +struct +-- !query 46 output +interval 2 weeks 6 days 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 47 +select interval '40:32.99899999' minute to second +-- !query 47 schema +struct +-- !query 47 output +interval 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 48 +select interval '40:32' minute to second +-- !query 48 schema +struct +-- !query 48 output +interval 40 minutes 32 seconds + + +-- !query 49 +select interval 10 nanoseconds +-- !query 49 schema +struct<> +-- !query 49 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'interval 10 nanoseconds'(line 1, pos 19) @@ -350,11 +454,11 @@ select interval 10 nanoseconds -------------------^^^ --- !query 37 +-- !query 50 select GEO '(10,-6)' --- !query 37 schema +-- !query 50 schema struct<> --- !query 37 output +-- !query 50 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'GEO' are currently not supported.(line 1, pos 7) @@ -364,19 +468,19 @@ select GEO '(10,-6)' -------^^^ --- !query 38 +-- !query 51 select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD --- !query 38 schema +-- !query 51 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> --- !query 38 output +-- !query 51 output 90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 --- !query 39 +-- !query 52 select 1.20E-38BD --- !query 39 schema +-- !query 52 schema struct<> --- !query 39 output +-- !query 52 output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38(line 1, pos 7) @@ -386,19 +490,19 @@ select 1.20E-38BD -------^^^ --- !query 40 +-- !query 53 select x'2379ACFe' --- !query 40 schema +-- !query 53 schema struct --- !query 40 output +-- !query 53 output #y�� --- !query 41 +-- !query 54 select X'XuZ' --- !query 41 schema +-- !query 54 schema struct<> --- !query 41 output +-- !query 54 output org.apache.spark.sql.catalyst.parser.ParseException contains illegal character for hexBinary: 0XuZ(line 1, pos 7) @@ -408,33 +512,33 @@ select X'XuZ' -------^^^ --- !query 42 +-- !query 55 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 --- !query 42 schema +-- !query 55 schema struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> --- !query 42 output +-- !query 55 output 3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 --- !query 43 +-- !query 56 select map(1, interval 1 day, 2, interval 3 week) --- !query 43 schema +-- !query 56 schema struct> --- !query 43 output +-- !query 56 output {1:interval 1 days,2:interval 3 weeks} --- !query 44 +-- !query 57 select interval 'interval 3 year 1 hour' --- !query 44 schema +-- !query 57 schema struct --- !query 44 output +-- !query 57 output interval 3 years 1 hours --- !query 45 +-- !query 58 select interval '3 year 1 hour' --- !query 45 schema +-- !query 58 schema struct --- !query 45 output +-- !query 58 output interval 3 years 1 hours diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala index 4c1c75b815a02..d75cb1040f31e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala @@ -88,8 +88,9 @@ object IntervalBenchmark extends SqlBasedBenchmark { val intervalToTest = ListBuffer[String]() val benchmark = new Benchmark("cast strings to intervals", N, output = output) - addCase(benchmark, N, "string w/ interval", buildString(true, timeUnits)) - addCase(benchmark, N, "string w/o interval", buildString(false, timeUnits)) + // The first 2 cases are used to show the overhead of preparing the interval string. + addCase(benchmark, N, "prepare string w/ interval", buildString(true, timeUnits)) + addCase(benchmark, N, "prepare string w/o interval", buildString(false, timeUnits)) addCase(benchmark, N, intervalToTest) // Only years for (unit <- timeUnits) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 116a62b2f6dc6..3e5834f33ea52 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -42,7 +42,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) @@ -1183,51 +1182,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } - test("Convert hive interval term into Literal of CalendarIntervalType") { - checkAnswer(sql("select interval '0 0:0:0.1' day to second"), - Row(CalendarInterval.fromString("interval 100 milliseconds"))) - checkAnswer(sql("select interval '10-9' year to month"), - Row(CalendarInterval.fromString("interval 10 years 9 months"))) - checkAnswer(sql("select interval '20 15:40:32.99899999' day to hour"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours"))) - checkAnswer(sql("select interval '20 15:40:32.99899999' day to minute"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes"))) - checkAnswer(sql("select interval '20 15:40:32.99899999' day to second"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes " + - "32 seconds 998 milliseconds 999 microseconds"))) - checkAnswer(sql("select interval '15:40:32.99899999' hour to minute"), - Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) - checkAnswer(sql("select interval '15:40.99899999' hour to second"), - Row(CalendarInterval.fromString("interval 15 minutes 40 seconds 998 milliseconds " + - "999 microseconds"))) - checkAnswer(sql("select interval '15:40' hour to second"), - Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) - checkAnswer(sql("select interval '15:40:32.99899999' hour to second"), - Row(CalendarInterval.fromString("interval 15 hours 40 minutes 32 seconds 998 milliseconds " + - "999 microseconds"))) - checkAnswer(sql("select interval '20 40:32.99899999' minute to second"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 40 minutes 32 seconds " + - "998 milliseconds 999 microseconds"))) - checkAnswer(sql("select interval '40:32.99899999' minute to second"), - Row(CalendarInterval.fromString("interval 40 minutes 32 seconds 998 milliseconds " + - "999 microseconds"))) - checkAnswer(sql("select interval '40:32' minute to second"), - Row(CalendarInterval.fromString("interval 40 minutes 32 seconds"))) - checkAnswer(sql("select interval '30' year"), - Row(CalendarInterval.fromString("interval 30 years"))) - checkAnswer(sql("select interval '25' month"), - Row(CalendarInterval.fromString("interval 25 months"))) - checkAnswer(sql("select interval '-100' day"), - Row(CalendarInterval.fromString("interval -14 weeks -2 days"))) - checkAnswer(sql("select interval '40' hour"), - Row(CalendarInterval.fromString("interval 1 days 16 hours"))) - checkAnswer(sql("select interval '80' minute"), - Row(CalendarInterval.fromString("interval 1 hour 20 minutes"))) - checkAnswer(sql("select interval '299.889987299' second"), - Row(CalendarInterval.fromString( - "interval 4 minutes 59 seconds 889 milliseconds 987 microseconds"))) - } - test("specifying database name for a temporary view is not allowed") { withTempPath { dir => withTempView("db.t") {