[SPARK-29783][SQL] Support SQL Standard/ISO_8601 output style for interval type

### What changes were proposed in this pull request?

Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table.

Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval
-- | -- | -- | -- | --
Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds
SQL STANDARD  | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06
ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S

### Why are the changes needed?

for ANSI SQL support
### Does this PR introduce any user-facing change?

yes,interval out now has 3 output styles
### How was this patch tested?

add new unit tests

cc cloud-fan maropu MaxGekk HyukjinKwon thanks.

Closes #26418 from yaooqinn/SPARK-29783.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
Kent Yao 2019-11-18 15:42:22 +08:00 committed by Wenchen Fan
parent 73912379d0
commit 5cebe587c7
19 changed files with 361 additions and 77 deletions

View file

@ -18,7 +18,6 @@
package org.apache.spark.unsafe.types;
import java.io.Serializable;
import java.math.BigDecimal;
import java.time.Duration;
import java.time.Period;
import java.time.temporal.ChronoUnit;
@ -80,39 +79,8 @@ public final class CalendarInterval implements Serializable, Comparable<Calendar
@Override
public String toString() {
if (months == 0 && days == 0 && microseconds == 0) {
return "0 seconds";
}
StringBuilder sb = new StringBuilder();
if (months != 0) {
appendUnit(sb, months / 12, "years");
appendUnit(sb, months % 12, "months");
}
appendUnit(sb, days, "days");
if (microseconds != 0) {
long rest = microseconds;
appendUnit(sb, rest / MICROS_PER_HOUR, "hours");
rest %= MICROS_PER_HOUR;
appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes");
rest %= MICROS_PER_MINUTE;
if (rest != 0) {
String s = BigDecimal.valueOf(rest, 6).stripTrailingZeros().toPlainString();
sb.append(s).append(" seconds ");
}
}
sb.setLength(sb.length() - 1);
return sb.toString();
}
private void appendUnit(StringBuilder sb, long value, String unit) {
if (value != 0) {
sb.append(value).append(' ').append(unit).append(' ');
}
return "CalendarInterval(months= " + months + ", days = " + days + ", microsecond = " +
microseconds + ")";
}
/**

View file

@ -46,36 +46,6 @@ public class CalendarIntervalSuite {
assertEquals(i1, i6);
}
@Test
public void toStringTest() {
CalendarInterval i;
i = new CalendarInterval(0, 0, 0);
assertEquals("0 seconds", i.toString());
i = new CalendarInterval(34, 0, 0);
assertEquals("2 years 10 months", i.toString());
i = new CalendarInterval(-34, 0, 0);
assertEquals("-2 years -10 months", i.toString());
i = new CalendarInterval(0, 31, 0);
assertEquals("31 days", i.toString());
i = new CalendarInterval(0, -31, 0);
assertEquals("-31 days", i.toString());
i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123);
assertEquals("3 hours 13 minutes 0.000123 seconds", i.toString());
i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123);
assertEquals("-3 hours -13 minutes -0.000123 seconds", i.toString());
i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123);
assertEquals("2 years 10 months 31 days 3 hours 13 minutes 0.000123 seconds",
i.toString());
}
@Test
public void periodAndDurationTest() {
CalendarInterval interval = new CalendarInterval(120, -40, 123456);

View file

@ -30,7 +30,9 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.catalyst.util.IntervalUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.IntervalStyle._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.UTF8StringBuilder
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
@ -281,6 +283,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
// UDFToString
private[this] def castToString(from: DataType): Any => Any = from match {
case CalendarIntervalType => SQLConf.get.intervalOutputStyle match {
case SQL_STANDARD =>
buildCast[CalendarInterval](_, i => UTF8String.fromString(toSqlStandardString(i)))
case ISO_8601 =>
buildCast[CalendarInterval](_, i => UTF8String.fromString(toIso8601String(i)))
case MULTI_UNITS =>
buildCast[CalendarInterval](_, i => UTF8String.fromString(toMultiUnitsString(i)))
}
case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d)))
case TimestampType => buildCast[Long](_,
@ -985,6 +995,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
timestampFormatter.getClass)
(c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString(
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));"""
case CalendarIntervalType =>
val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$")
val funcName = SQLConf.get.intervalOutputStyle match {
case SQL_STANDARD => "toSqlStandardString"
case ISO_8601 => "toIso8601String"
case MULTI_UNITS => "toMultiUnitsString"
}
(c, evPrim, _) => code"""$evPrim = UTF8String.fromString($iu.$funcName($c));"""
case ArrayType(et, _) =>
(c, evPrim, evNull) => {
val buffer = ctx.freshVariable("buffer", classOf[UTF8StringBuilder])

View file

@ -409,6 +409,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone))
s"TIMESTAMP('${formatter.format(v)}')"
case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'"
case (v: CalendarInterval, CalendarIntervalType) => IntervalUtils.toMultiUnitsString(v)
case _ => value.toString
}
}

View file

@ -119,6 +119,10 @@ private[sql] class JacksonGenerator(
(row: SpecializedGetters, ordinal: Int) =>
gen.writeNumber(row.getDouble(ordinal))
case CalendarIntervalType =>
(row: SpecializedGetters, ordinal: Int) =>
gen.writeString(IntervalUtils.toMultiUnitsString(row.getInterval(ordinal)))
case StringType =>
(row: SpecializedGetters, ordinal: Int) =>
gen.writeString(row.getUTF8String(ordinal).toString)
@ -214,10 +218,15 @@ private[sql] class JacksonGenerator(
private def writeMapData(
map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = {
val keyArray = map.keyArray()
val keyString = mapType.keyType match {
case CalendarIntervalType =>
(i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i))
case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString
}
val valueArray = map.valueArray()
var i = 0
while (i < map.numElements()) {
gen.writeFieldName(keyArray.get(i, mapType.keyType).toString)
gen.writeFieldName(keyString(i))
if (!valueArray.isNullAt(i)) {
fieldWriter.apply(valueArray, i)
} else {

View file

@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.util
import java.math.BigDecimal
import java.util.concurrent.TimeUnit
import scala.util.control.NonFatal
@ -424,6 +425,111 @@ object IntervalUtils {
fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num)
}
def toMultiUnitsString(interval: CalendarInterval): String = {
if (interval.months == 0 && interval.days == 0 && interval.microseconds == 0) {
return "0 seconds"
}
val sb = new StringBuilder
if (interval.months != 0) {
appendUnit(sb, interval.months / 12, "years")
appendUnit(sb, interval.months % 12, "months")
}
appendUnit(sb, interval.days, "days")
if (interval.microseconds != 0) {
var rest = interval.microseconds
appendUnit(sb, rest / MICROS_PER_HOUR, "hours")
rest %= MICROS_PER_HOUR
appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes")
rest %= MICROS_PER_MINUTE
if (rest != 0) {
val s = BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString
sb.append(s).append(" seconds ")
}
}
sb.setLength(sb.length - 1)
sb.toString
}
private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = {
if (value != 0) sb.append(value).append(' ').append(unit).append(' ')
}
def toSqlStandardString(interval: CalendarInterval): String = {
val yearMonthPart = if (interval.months < 0) {
val ma = math.abs(interval.months)
"-" + ma / 12 + "-" + ma % 12
} else if (interval.months > 0) {
"+" + interval.months / 12 + "-" + interval.months % 12
} else {
""
}
val dayPart = if (interval.days < 0) {
interval.days.toString
} else if (interval.days > 0) {
"+" + interval.days
} else {
""
}
val timePart = if (interval.microseconds != 0) {
val sign = if (interval.microseconds > 0) "+" else "-"
val sb = new StringBuilder(sign)
var rest = math.abs(interval.microseconds)
sb.append(rest / MICROS_PER_HOUR)
sb.append(':')
rest %= MICROS_PER_HOUR
val minutes = rest / MICROS_PER_MINUTE;
if (minutes < 10) {
sb.append(0)
}
sb.append(minutes)
sb.append(':')
rest %= MICROS_PER_MINUTE
val bd = BigDecimal.valueOf(rest, 6)
if (bd.compareTo(new BigDecimal(10)) < 0) {
sb.append(0)
}
val s = bd.stripTrailingZeros().toPlainString
sb.append(s)
sb.toString()
} else {
""
}
val intervalList = Seq(yearMonthPart, dayPart, timePart).filter(_.nonEmpty)
if (intervalList.nonEmpty) intervalList.mkString(" ") else "0"
}
def toIso8601String(interval: CalendarInterval): String = {
val sb = new StringBuilder("P")
val year = interval.months / 12
if (year != 0) sb.append(year + "Y")
val month = interval.months % 12
if (month != 0) sb.append(month + "M")
if (interval.days != 0) sb.append(interval.days + "D")
if (interval.microseconds != 0) {
sb.append('T')
var rest = interval.microseconds
val hour = rest / MICROS_PER_HOUR
if (hour != 0) sb.append(hour + "H")
rest %= MICROS_PER_HOUR
val minute = rest / MICROS_PER_MINUTE
if (minute != 0) sb.append(minute + "M")
rest %= MICROS_PER_MINUTE
if (rest != 0) {
val bd = BigDecimal.valueOf(rest, 6)
sb.append(bd.stripTrailingZeros().toPlainString + "S")
}
} else if (interval.days == 0 && interval.months == 0) {
sb.append("T0S")
}
sb.toString()
}
private object ParseState extends Enumeration {
type ParseState = Value

View file

@ -1782,6 +1782,23 @@ object SQLConf {
.checkValues(StoreAssignmentPolicy.values.map(_.toString))
.createWithDefault(StoreAssignmentPolicy.ANSI.toString)
object IntervalStyle extends Enumeration {
type IntervalStyle = Value
val SQL_STANDARD, ISO_8601, MULTI_UNITS = Value
}
val INTERVAL_STYLE = buildConf("spark.sql.intervalOutputStyle")
.doc("When converting interval values to strings (i.e. for display), this config decides the" +
" interval string format. The value SQL_STANDARD will produce output matching SQL standard" +
" interval literals (i.e. '+3-2 +10 -00:00:01'). The value ISO_8601 will produce output" +
" matching the ISO 8601 standard (i.e. 'P3Y2M10DT-1S'). The value MULTI_UNITS (which is the" +
" default) will produce output in form of value unit pairs, (i.e. '3 year 2 months 10 days" +
" -1 seconds'")
.stringConf
.transform(_.toUpperCase(Locale.ROOT))
.checkValues(IntervalStyle.values.map(_.toString))
.createWithDefault(IntervalStyle.MULTI_UNITS.toString)
val SORT_BEFORE_REPARTITION =
buildConf("spark.sql.execution.sortBeforeRepartition")
.internal()
@ -2496,6 +2513,8 @@ class SQLConf extends Serializable with Logging {
def storeAssignmentPolicy: StoreAssignmentPolicy.Value =
StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY))
def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE))
def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString
def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED)

View file

@ -298,4 +298,68 @@ class IntervalUtilsSuite extends SparkFunSuite {
new CalendarInterval(-2, 0, -1 * MICROS_PER_HOUR))
intercept[ArithmeticException](justifyInterval(new CalendarInterval(2, 0, Long.MaxValue)))
}
test("to ansi sql standard string") {
val i1 = new CalendarInterval(0, 0, 0)
assert(IntervalUtils.toSqlStandardString(i1) === "0")
val i2 = new CalendarInterval(34, 0, 0)
assert(IntervalUtils.toSqlStandardString(i2) === "+2-10")
val i3 = new CalendarInterval(-34, 0, 0)
assert(IntervalUtils.toSqlStandardString(i3) === "-2-10")
val i4 = new CalendarInterval(0, 31, 0)
assert(IntervalUtils.toSqlStandardString(i4) === "+31")
val i5 = new CalendarInterval(0, -31, 0)
assert(IntervalUtils.toSqlStandardString(i5) === "-31")
val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123)
assert(IntervalUtils.toSqlStandardString(i6) === "+3:13:00.000123")
val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123)
assert(IntervalUtils.toSqlStandardString(i7) === "-3:13:00.000123")
val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123)
assert(IntervalUtils.toSqlStandardString(i8) === "-2-10 +31 +3:13:00.000123")
val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR)
assert(IntervalUtils.toSqlStandardString(i9) === "-3000:00:00")
}
test("to iso 8601 string") {
val i1 = new CalendarInterval(0, 0, 0)
assert(IntervalUtils.toIso8601String(i1) === "PT0S")
val i2 = new CalendarInterval(34, 0, 0)
assert(IntervalUtils.toIso8601String(i2) === "P2Y10M")
val i3 = new CalendarInterval(-34, 0, 0)
assert(IntervalUtils.toIso8601String(i3) === "P-2Y-10M")
val i4 = new CalendarInterval(0, 31, 0)
assert(IntervalUtils.toIso8601String(i4) === "P31D")
val i5 = new CalendarInterval(0, -31, 0)
assert(IntervalUtils.toIso8601String(i5) === "P-31D")
val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123)
assert(IntervalUtils.toIso8601String(i6) === "PT3H13M0.000123S")
val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123)
assert(IntervalUtils.toIso8601String(i7) === "PT-3H-13M-0.000123S")
val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123)
assert(IntervalUtils.toIso8601String(i8) === "P-2Y-10M31DT3H13M0.000123S")
val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR)
assert(IntervalUtils.toIso8601String(i9) === "PT-3000H")
}
test("to multi units string") {
val i1 = new CalendarInterval(0, 0, 0)
assert(IntervalUtils.toMultiUnitsString(i1) === "0 seconds")
val i2 = new CalendarInterval(34, 0, 0)
assert(IntervalUtils.toMultiUnitsString(i2) === "2 years 10 months")
val i3 = new CalendarInterval(-34, 0, 0)
assert(IntervalUtils.toMultiUnitsString(i3) === "-2 years -10 months")
val i4 = new CalendarInterval(0, 31, 0)
assert(IntervalUtils.toMultiUnitsString(i4) === "31 days")
val i5 = new CalendarInterval(0, -31, 0)
assert(IntervalUtils.toMultiUnitsString(i5) === "-31 days")
val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123)
assert(IntervalUtils.toMultiUnitsString(i6) === "3 hours 13 minutes 0.000123 seconds")
val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123)
assert(IntervalUtils.toMultiUnitsString(i7) === "-3 hours -13 minutes -0.000123 seconds")
val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123)
assert(IntervalUtils.toMultiUnitsString(i8) ===
"-2 years -10 months 31 days 3 hours 13 minutes 0.000123 seconds")
val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR)
assert(IntervalUtils.toMultiUnitsString(i9) === "-3000 hours")
}
}

View file

@ -22,9 +22,12 @@ import java.sql.{Date, Timestamp}
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.IntervalUtils._
import org.apache.spark.sql.execution.command.{DescribeCommandBase, ExecutedCommandExec, ShowTablesCommand}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.IntervalStyle._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval
/**
* Runs a query returning the result in Hive compatible form.
@ -97,7 +100,12 @@ object HiveResult {
case (null, _) => "null"
case (s: String, StringType) => "\"" + s + "\""
case (decimal, DecimalType()) => decimal.toString
case (interval, CalendarIntervalType) => interval.toString
case (interval: CalendarInterval, CalendarIntervalType) =>
SQLConf.get.intervalOutputStyle match {
case SQL_STANDARD => toSqlStandardString(interval)
case ISO_8601 => toIso8601String(interval)
case MULTI_UNITS => toMultiUnitsString(interval)
}
case (other, tpe) if primitiveTypes contains tpe => other.toString
}
@ -120,6 +128,12 @@ object HiveResult {
DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t))
case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8)
case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal)
case (interval: CalendarInterval, CalendarIntervalType) =>
SQLConf.get.intervalOutputStyle match {
case SQL_STANDARD => toSqlStandardString(interval)
case ISO_8601 => toIso8601String(interval)
case MULTI_UNITS => toMultiUnitsString(interval)
}
case (interval, CalendarIntervalType) => interval.toString
case (other, _ : UserDefinedType[_]) => other.toString
case (other, tpe) if primitiveTypes.contains(tpe) => other.toString

View file

@ -0,0 +1,3 @@
-- tests for interval output style with iso_8601 format
--SET spark.sql.intervalOutputStyle = ISO_8601
--import interval-display.sql

View file

@ -0,0 +1,3 @@
-- tests for interval output style with sql standard format
--SET spark.sql.intervalOutputStyle = SQL_STANDARD
--import interval-display.sql

View file

@ -0,0 +1,14 @@
-- tests for interval output style
SELECT
cast(null as interval), -- null
interval '0 day', -- 0
interval '1 year', -- year only
interval '1 month', -- month only
interval '1 year 2 month', -- year month only
interval '1 day -1 hours',
interval '-1 day -1 hours',
interval '-1 day 1 hours',
interval '-1 days +1 hours',
interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds';

View file

@ -270,10 +270,12 @@ SELECT interval '1 2:03:04' minute to second;
-- test output of couple non-standard interval values in the sql style
-- [SPARK-29406] Interval output styles
-- SET IntervalStyle TO sql_standard;
-- SELECT interval '1 day -1 hours',
-- interval '-1 days +1 hours',
-- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
-- - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds';
set spark.sql.intervalOutputStyle=SQL_STANDARD;
SELECT interval '1 day -1 hours',
interval '-1 days +1 hours',
interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds';
set spark.sql.intervalOutputStyle=MULTI_UNITS;
-- test outputting iso8601 intervals
-- [SPARK-29406] Interval output styles

View file

@ -0,0 +1,21 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 1
-- !query 0
SELECT
cast(null as interval), -- null
interval '0 day', -- 0
interval '1 year', -- year only
interval '1 month', -- month only
interval '1 year 2 month', -- year month only
interval '1 day -1 hours',
interval '-1 day -1 hours',
interval '-1 day 1 hours',
interval '-1 days +1 hours',
interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'
-- !query 0 schema
struct<CAST(NULL AS INTERVAL):interval,0 seconds:interval,1 years:interval,1 months:interval,1 years 2 months:interval,1 days -1 hours:interval,-1 days -1 hours:interval,-1 days 1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval>
-- !query 0 output
NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S

View file

@ -0,0 +1,21 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 1
-- !query 0
SELECT
cast(null as interval), -- null
interval '0 day', -- 0
interval '1 year', -- year only
interval '1 month', -- month only
interval '1 year 2 month', -- year month only
interval '1 day -1 hours',
interval '-1 day -1 hours',
interval '-1 day 1 hours',
interval '-1 days +1 hours',
interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'
-- !query 0 schema
struct<CAST(NULL AS INTERVAL):interval,0 seconds:interval,1 years:interval,1 months:interval,1 years 2 months:interval,1 days -1 hours:interval,-1 days -1 hours:interval,-1 days 1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval>
-- !query 0 output
NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789

View file

@ -0,0 +1,21 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 1
-- !query 0
SELECT
cast(null as interval), -- null
interval '0 day', -- 0
interval '1 year', -- year only
interval '1 month', -- month only
interval '1 year 2 month', -- year month only
interval '1 day -1 hours',
interval '-1 day -1 hours',
interval '-1 day 1 hours',
interval '-1 days +1 hours',
interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'
-- !query 0 schema
struct<CAST(NULL AS INTERVAL):interval,0 seconds:interval,1 years:interval,1 months:interval,1 years 2 months:interval,1 days -1 hours:interval,-1 days -1 hours:interval,-1 days 1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval>
-- !query 0 output
NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds

View file

@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 27
-- Number of queries: 30
-- !query 0
@ -216,3 +216,30 @@ SELECT interval '1 2:03:04' minute to second
struct<1 days 2 hours 3 minutes 4 seconds:interval>
-- !query 26 output
1 days 2 hours 3 minutes 4 seconds
-- !query 27
set spark.sql.intervalOutputStyle=SQL_STANDARD
-- !query 27 schema
struct<key:string,value:string>
-- !query 27 output
spark.sql.intervalOutputStyle SQL_STANDARD
-- !query 28
SELECT interval '1 day -1 hours',
interval '-1 days +1 hours',
interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds',
- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'
-- !query 28 schema
struct<1 days -1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval>
-- !query 28 output
+1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789
-- !query 29
set spark.sql.intervalOutputStyle=MULTI_UNITS
-- !query 29 schema
struct<key:string,value:string>
-- !query 29 output
spark.sql.intervalOutputStyle MULTI_UNITS

View file

@ -23,7 +23,7 @@ import java.time.Instant
import java.util.Locale
import java.util.concurrent.TimeUnit
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
@ -293,10 +293,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
val i = new CalendarInterval(2, 2, 2000000L)
val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d")
checkAnswer(
df.selectExpr(s"d + INTERVAL'$i'"),
df.selectExpr(s"d + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"),
Seq(Row(Date.valueOf("2015-10-02")), Row(Date.valueOf("2016-03-02"))))
checkAnswer(
df.selectExpr(s"t + INTERVAL'$i'"),
df.selectExpr(s"t + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"),
Seq(Row(Timestamp.valueOf("2015-10-03 00:00:01")),
Row(Timestamp.valueOf("2016-03-02 00:00:02"))))
}
@ -309,10 +309,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession {
val i = new CalendarInterval(2, 2, 2000000L)
val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d")
checkAnswer(
df.selectExpr(s"d - INTERVAL'$i'"),
df.selectExpr(s"d - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"),
Seq(Row(Date.valueOf("2015-07-27")), Row(Date.valueOf("2015-12-26"))))
checkAnswer(
df.selectExpr(s"t - INTERVAL'$i'"),
df.selectExpr(s"t - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"),
Seq(Row(Timestamp.valueOf("2015-07-29 23:59:59")),
Row(Timestamp.valueOf("2015-12-27 00:00:00"))))
}

View file

@ -93,7 +93,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite {
"subquery/in-subquery/in-group-by.sql",
"subquery/in-subquery/simple-in.sql",
"subquery/in-subquery/in-order-by.sql",
"subquery/in-subquery/in-set-operations.sql"
"subquery/in-subquery/in-set-operations.sql",
// SPARK-29783: need to set conf
"interval-display-iso_8601.sql",
"interval-display-sql_standard.sql"
)
override def runQueries(