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 @@ -18,8 +18,6 @@
package org.apache.spark.unsafe.types;

import java.io.Serializable;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

/**
* The internal representation of interval type.
Expand All @@ -32,209 +30,6 @@ 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;

private static Pattern yearMonthPattern = Pattern.compile(
"^([+|-])?(\\d+)-(\\d+)$");

private static Pattern dayTimePattern = Pattern.compile(
"^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$");

public static long toLongWithRange(String fieldName,
String s, long minValue, long maxValue) throws IllegalArgumentException {
long result = 0;
if (s != null) {
result = Long.parseLong(s);
if (result < minValue || result > maxValue) {
throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]",
fieldName, result, minValue, maxValue));
}
}
return result;
}

/**
* Parse YearMonth string in form: [-]YYYY-MM
*
* adapted from HiveIntervalYearMonth.valueOf
*/
public static CalendarInterval fromYearMonthString(String s) throws IllegalArgumentException {
CalendarInterval result = null;
if (s == null) {
throw new IllegalArgumentException("Interval year-month string was null");
}
s = s.trim();
Matcher m = yearMonthPattern.matcher(s);
if (!m.matches()) {
throw new IllegalArgumentException(
"Interval string does not match year-month format of 'y-m': " + s);
} else {
try {
int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1;
int years = (int) toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE);
int months = (int) toLongWithRange("month", m.group(3), 0, 11);
result = new CalendarInterval(sign * (years * 12 + months), 0);
} catch (Exception e) {
throw new IllegalArgumentException(
"Error parsing interval year-month string: " + e.getMessage(), e);
}
}
return result;
}

/**
* Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn
*
* adapted from HiveIntervalDayTime.valueOf
*/
public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException {
return fromDayTimeString(s, "day", "second");
}

/**
* Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn
*
* adapted from HiveIntervalDayTime.valueOf.
* Below interval conversion patterns are supported:
* - DAY TO (HOUR|MINUTE|SECOND)
* - HOUR TO (MINUTE|SECOND)
* - MINUTE TO SECOND
*/
public static CalendarInterval fromDayTimeString(String s, String from, String to)
throws IllegalArgumentException {
CalendarInterval result = null;
if (s == null) {
throw new IllegalArgumentException("Interval day-time string was null");
}
s = s.trim();
Matcher m = dayTimePattern.matcher(s);
if (!m.matches()) {
throw new IllegalArgumentException(
"Interval string does not match day-time format of 'd h:m:s.n': " + s);
} else {
try {
int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1;
long days = m.group(2) == null ? 0 : toLongWithRange("day", m.group(3),
0, Integer.MAX_VALUE);
long hours = 0;
long minutes;
long seconds = 0;
if (m.group(5) != null || from.equals("minute")) { // 'HH:mm:ss' or 'mm:ss minute'
hours = toLongWithRange("hour", m.group(5), 0, 23);
minutes = toLongWithRange("minute", m.group(6), 0, 59);
seconds = toLongWithRange("second", m.group(7), 0, 59);
} else if (m.group(8) != null){ // 'mm:ss.nn'
minutes = toLongWithRange("minute", m.group(6), 0, 59);
seconds = toLongWithRange("second", m.group(7), 0, 59);
} else { // 'HH:mm'
hours = toLongWithRange("hour", m.group(6), 0, 23);
minutes = toLongWithRange("second", m.group(7), 0, 59);
}
// Hive allow nanosecond precision interval
String nanoStr = m.group(9) == null ? null : (m.group(9) + "000000000").substring(0, 9);
long nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L);
switch (to) {
case "hour":
minutes = 0;
seconds = 0;
nanos = 0;
break;
case "minute":
seconds = 0;
nanos = 0;
break;
case "second":
// No-op
break;
default:
throw new IllegalArgumentException(
String.format("Cannot support (interval '%s' %s to %s) expression", s, from, to));
}
result = new CalendarInterval(0, sign * (
days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE +
seconds * MICROS_PER_SECOND + nanos / 1000L));
} catch (Exception e) {
throw new IllegalArgumentException(
"Error parsing interval day-time string: " + e.getMessage(), e);
}
}
return result;
}

public static CalendarInterval fromUnitStrings(String[] units, String[] values)
throws IllegalArgumentException {
assert units.length == values.length;
int months = 0;
long microseconds = 0;

for (int i = 0; i < units.length; i++) {
try {
switch (units[i]) {
case "year":
months = Math.addExact(months, Math.multiplyExact(Integer.parseInt(values[i]), 12));
break;
case "month":
months = Math.addExact(months, Integer.parseInt(values[i]));
break;
case "week":
microseconds = Math.addExact(
microseconds,
Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_WEEK));
break;
case "day":
microseconds = Math.addExact(
microseconds,
Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_DAY));
break;
case "hour":
microseconds = Math.addExact(
microseconds,
Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_HOUR));
break;
case "minute":
microseconds = Math.addExact(
microseconds,
Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MINUTE));
break;
case "second": {
microseconds = Math.addExact(microseconds, parseSecondNano(values[i]));
break;
}
case "millisecond":
microseconds = Math.addExact(
microseconds,
Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MILLI));
break;
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 new CalendarInterval(months, microseconds);
}

/**
* Parse second_nano string in ss.nnnnnnnnn format to microseconds
*/
public static long parseSecondNano(String secondNano) throws IllegalArgumentException {
String[] parts = secondNano.split("\\.");
if (parts.length == 1) {
return toLongWithRange("second", parts[0], Long.MIN_VALUE / MICROS_PER_SECOND,
Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND;

} else if (parts.length == 2) {
long seconds = parts[0].equals("") ? 0L : toLongWithRange("second", parts[0],
Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND);
long nanos = toLongWithRange("nanosecond", parts[1], 0L, 999999999L);
return seconds * MICROS_PER_SECOND + nanos / 1000L;

} else {
throw new IllegalArgumentException(
"Interval string does not match second-nano format of ss.nnnnnnnnn");
}
}

public final int months;
public final long microseconds;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,72 +60,6 @@ public void toStringTest() {
assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString());
}

@Test
public void fromYearMonthStringTest() {
String input;
CalendarInterval i;

input = "99-10";
i = new CalendarInterval(99 * 12 + 10, 0L);
assertEquals(fromYearMonthString(input), i);

input = "-8-10";
i = new CalendarInterval(-8 * 12 - 10, 0L);
assertEquals(fromYearMonthString(input), i);

try {
input = "99-15";
fromYearMonthString(input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("month 15 outside range"));
}
}

@Test
public void fromDayTimeStringTest() {
String input;
CalendarInterval i;

input = "5 12:40:30.999999999";
i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR +
40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L);
assertEquals(fromDayTimeString(input), i);

input = "10 0:12:0.888";
i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE +
888 * MICROS_PER_MILLI);
assertEquals(fromDayTimeString(input), i);

input = "-3 0:0:0";
i = new CalendarInterval(0, -3 * MICROS_PER_DAY);
assertEquals(fromDayTimeString(input), i);

try {
input = "5 30:12:20";
fromDayTimeString(input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("hour 30 outside range"));
}

try {
input = "5 30-12";
fromDayTimeString(input);
fail("Expected to throw an exception for the invalid input");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("not match day-time format"));
}

try {
input = "5 1:12:20";
fromDayTimeString(input, "hour", "microsecond");
fail("Expected to throw an exception for the invalid convention type");
} catch (IllegalArgumentException e) {
assertTrue(e.getMessage().contains("Cannot support (interval"));
}
}

@Test
public void addTest() {
CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}.toArray
val values = ctx.intervalValue().asScala.map(getIntervalValue).toArray
try {
CalendarInterval.fromUnitStrings(units, values)
IntervalUtils.fromUnitStrings(units, values)
} catch {
case i: IllegalArgumentException =>
val e = new ParseException(i.getMessage, ctx)
Expand Down Expand Up @@ -1953,21 +1953,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
val unitText = unit.getText.toLowerCase(Locale.ROOT)
val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match {
case (u, None) =>
CalendarInterval.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s))
IntervalUtils.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s))
case ("year", Some("month")) =>
CalendarInterval.fromYearMonthString(s)
IntervalUtils.fromYearMonthString(s)
case ("day", Some("hour")) =>
CalendarInterval.fromDayTimeString(s, "day", "hour")
IntervalUtils.fromDayTimeString(s, "day", "hour")
case ("day", Some("minute")) =>
CalendarInterval.fromDayTimeString(s, "day", "minute")
IntervalUtils.fromDayTimeString(s, "day", "minute")
case ("day", Some("second")) =>
CalendarInterval.fromDayTimeString(s, "day", "second")
IntervalUtils.fromDayTimeString(s, "day", "second")
case ("hour", Some("minute")) =>
CalendarInterval.fromDayTimeString(s, "hour", "minute")
IntervalUtils.fromDayTimeString(s, "hour", "minute")
case ("hour", Some("second")) =>
CalendarInterval.fromDayTimeString(s, "hour", "second")
IntervalUtils.fromDayTimeString(s, "hour", "second")
case ("minute", Some("second")) =>
CalendarInterval.fromDayTimeString(s, "minute", "second")
IntervalUtils.fromDayTimeString(s, "minute", "second")
case (from, Some(t)) =>
throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx)
}
Expand Down
Loading