[SPARK-35704][SQL] Add fields to DayTimeIntervalType

### What changes were proposed in this pull request?
Extend DayTimeIntervalType to support interval fields. Valid interval field values:
- 0 (DAY)
- 1 (HOUR)
- 2 (MINUTE)
- 3 (SECOND)

After the changes, the following day-time interval types are supported:
1. `DayTimeIntervalType(0, 0)` or `DayTimeIntervalType(DAY, DAY)`
2. `DayTimeIntervalType(0, 1)` or `DayTimeIntervalType(DAY, HOUR)`
3. `DayTimeIntervalType(0, 2)` or `DayTimeIntervalType(DAY, MINUTE)`
4. `DayTimeIntervalType(0, 3)` or `DayTimeIntervalType(DAY, SECOND)`. **It is the default one**. The second fraction precision is microseconds.
5. `DayTimeIntervalType(1, 1)` or `DayTimeIntervalType(HOUR, HOUR)`
6. `DayTimeIntervalType(1, 2)` or `DayTimeIntervalType(HOUR, MINUTE)`
7. `DayTimeIntervalType(1, 3)` or `DayTimeIntervalType(HOUR, SECOND)`
8. `DayTimeIntervalType(2, 2)` or `DayTimeIntervalType(MINUTE, MINUTE)`
9. `DayTimeIntervalType(2, 3)` or `DayTimeIntervalType(MINUTE, SECOND)`
10. `DayTimeIntervalType(3, 3)` or `DayTimeIntervalType(SECOND, SECOND)`

### Why are the changes needed?
In the current implementation, Spark supports only `interval day to second` but the SQL standard allows to specify the start and end fields. The changes will allow to follow ANSI SQL standard more precisely.

### Does this PR introduce _any_ user-facing change?
Yes but `DayTimeIntervalType` has not been released yet.

### How was this patch tested?
By existing test suites.

Closes #32849 from MaxGekk/day-time-interval-type-units.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
This commit is contained in:
Max Gekk 2021-06-11 16:16:33 +03:00
parent 692dc66c4a
commit d53831ff5c
66 changed files with 359 additions and 211 deletions

View file

@ -76,7 +76,7 @@ public final class UnsafeRow extends InternalRow implements Externalizable, Kryo
*/
public static final Set<DataType> mutableFieldTypes;
// DecimalType is also mutable
// DecimalType and DayTimeIntervalType are also mutable
static {
mutableFieldTypes = Collections.unmodifiableSet(
new HashSet<>(
@ -91,8 +91,7 @@ public final class UnsafeRow extends InternalRow implements Externalizable, Kryo
DoubleType,
DateType,
TimestampType,
YearMonthIntervalType,
DayTimeIntervalType
YearMonthIntervalType
})));
}
@ -104,7 +103,7 @@ public final class UnsafeRow extends InternalRow implements Externalizable, Kryo
if (dt instanceof DecimalType) {
return ((DecimalType) dt).precision() <= Decimal.MAX_LONG_DIGITS();
} else {
return mutableFieldTypes.contains(dt);
return dt instanceof DayTimeIntervalType || mutableFieldTypes.contains(dt);
}
}
@ -114,7 +113,7 @@ public final class UnsafeRow extends InternalRow implements Externalizable, Kryo
}
return mutableFieldTypes.contains(dt) || dt instanceof DecimalType ||
dt instanceof CalendarIntervalType;
dt instanceof CalendarIntervalType || dt instanceof DayTimeIntervalType;
}
//////////////////////////////////////////////////////////////////////////////

View file

@ -99,11 +99,6 @@ public class DataTypes {
*/
public static final DataType NullType = NullType$.MODULE$;
/**
* Gets the DayTimeIntervalType object.
*/
public static final DataType DayTimeIntervalType = DayTimeIntervalType$.MODULE$;
/**
* Gets the YearMonthIntervalType object.
*/
@ -145,6 +140,20 @@ public class DataTypes {
return DecimalType$.MODULE$.USER_DEFAULT();
}
/**
* Creates a DayTimeIntervalType by specifying the start and end fields.
*/
public static DayTimeIntervalType createDayTimeIntervalType(byte startField, byte endField) {
return DayTimeIntervalType$.MODULE$.apply(startField, endField);
}
/**
* Creates a DayTimeIntervalType with default start and end fields: interval day to second.
*/
public static DayTimeIntervalType createDayTimeIntervalType() {
return DayTimeIntervalType$.MODULE$.DEFAULT();
}
/**
* Creates a MapType by specifying the data type of keys ({@code keyType}) and values
* ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}.

View file

@ -75,7 +75,8 @@ object CatalystTypeConverters {
case LongType => LongConverter
case FloatType => FloatConverter
case DoubleType => DoubleConverter
case DayTimeIntervalType => DurationConverter
// TODO(SPARK-35726): Truncate java.time.Duration by fields of day-time interval type
case _: DayTimeIntervalType => DurationConverter
case YearMonthIntervalType => PeriodConverter
case dataType: DataType => IdentityConverter(dataType)
}

View file

@ -134,7 +134,7 @@ object InternalRow {
case ShortType => (input, ordinal) => input.getShort(ordinal)
case IntegerType | DateType | YearMonthIntervalType =>
(input, ordinal) => input.getInt(ordinal)
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType =>
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType =>
(input, ordinal) => input.getLong(ordinal)
case FloatType => (input, ordinal) => input.getFloat(ordinal)
case DoubleType => (input, ordinal) => input.getDouble(ordinal)
@ -171,7 +171,7 @@ object InternalRow {
case ShortType => (input, v) => input.setShort(ordinal, v.asInstanceOf[Short])
case IntegerType | DateType | YearMonthIntervalType =>
(input, v) => input.setInt(ordinal, v.asInstanceOf[Int])
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType =>
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType =>
(input, v) => input.setLong(ordinal, v.asInstanceOf[Long])
case FloatType => (input, v) => input.setFloat(ordinal, v.asInstanceOf[Float])
case DoubleType => (input, v) => input.setDouble(ordinal, v.asInstanceOf[Double])

View file

@ -120,7 +120,7 @@ object JavaTypeInference {
case c: Class[_] if c == classOf[java.time.Instant] => (TimestampType, true)
case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true)
case c: Class[_] if c == classOf[java.time.LocalDateTime] => (TimestampWithoutTZType, true)
case c: Class[_] if c == classOf[java.time.Duration] => (DayTimeIntervalType, true)
case c: Class[_] if c == classOf[java.time.Duration] => (DayTimeIntervalType(), true)
case c: Class[_] if c == classOf[java.time.Period] => (YearMonthIntervalType, true)
case _ if typeToken.isArray =>

View file

@ -759,7 +759,7 @@ object ScalaReflection extends ScalaReflection {
case t if isSubtype(t, localTypeOf[CalendarInterval]) =>
Schema(CalendarIntervalType, nullable = true)
case t if isSubtype(t, localTypeOf[java.time.Duration]) =>
Schema(DayTimeIntervalType, nullable = true)
Schema(DayTimeIntervalType(), nullable = true)
case t if isSubtype(t, localTypeOf[java.time.Period]) =>
Schema(YearMonthIntervalType, nullable = true)
case t if isSubtype(t, localTypeOf[BigDecimal]) =>
@ -861,7 +861,6 @@ object ScalaReflection extends ScalaReflection {
TimestampWithoutTZType -> classOf[TimestampWithoutTZType.InternalType],
BinaryType -> classOf[BinaryType.InternalType],
CalendarIntervalType -> classOf[CalendarInterval],
DayTimeIntervalType -> classOf[DayTimeIntervalType.InternalType],
YearMonthIntervalType -> classOf[YearMonthIntervalType.InternalType]
)
@ -876,13 +875,13 @@ object ScalaReflection extends ScalaReflection {
DateType -> classOf[java.lang.Integer],
TimestampType -> classOf[java.lang.Long],
TimestampWithoutTZType -> classOf[java.lang.Long],
DayTimeIntervalType -> classOf[java.lang.Long],
YearMonthIntervalType -> classOf[java.lang.Integer]
)
def dataTypeJavaClass(dt: DataType): Class[_] = {
dt match {
case _: DecimalType => classOf[Decimal]
case it: DayTimeIntervalType => classOf[it.InternalType]
case _: StructType => classOf[InternalRow]
case _: ArrayType => classOf[ArrayData]
case _: MapType => classOf[MapData]
@ -893,6 +892,7 @@ object ScalaReflection extends ScalaReflection {
def javaBoxedType(dt: DataType): Class[_] = dt match {
case _: DecimalType => classOf[Decimal]
case _: DayTimeIntervalType => classOf[java.lang.Long]
case BinaryType => classOf[Array[Byte]]
case StringType => classOf[UTF8String]
case CalendarIntervalType => classOf[CalendarInterval]

View file

@ -116,7 +116,7 @@ object SerializerBuildHelper {
def createSerializerForJavaDuration(inputObject: Expression): Expression = {
StaticInvoke(
IntervalUtils.getClass,
DayTimeIntervalType,
DayTimeIntervalType(),
"durationToMicros",
inputObject :: Nil,
returnNullable = false)

View file

@ -349,34 +349,34 @@ class Analyzer(override val catalogManager: CatalogManager)
case p: LogicalPlan => p.transformExpressionsUpWithPruning(
_.containsPattern(BINARY_ARITHMETIC), ruleId) {
case a @ Add(l, r, f) if a.childrenResolved => (l.dataType, r.dataType) match {
case (DateType, DayTimeIntervalType) => TimeAdd(Cast(l, TimestampType), r)
case (DayTimeIntervalType, DateType) => TimeAdd(Cast(r, TimestampType), l)
case (DateType, _: DayTimeIntervalType) => TimeAdd(Cast(l, TimestampType), r)
case (_: DayTimeIntervalType, DateType) => TimeAdd(Cast(r, TimestampType), l)
case (DateType, YearMonthIntervalType) => DateAddYMInterval(l, r)
case (YearMonthIntervalType, DateType) => DateAddYMInterval(r, l)
case (TimestampType, YearMonthIntervalType) => TimestampAddYMInterval(l, r)
case (YearMonthIntervalType, TimestampType) => TimestampAddYMInterval(r, l)
case (CalendarIntervalType, CalendarIntervalType) |
(DayTimeIntervalType, DayTimeIntervalType) => a
(_: DayTimeIntervalType, _: DayTimeIntervalType) => a
case (DateType, CalendarIntervalType) => DateAddInterval(l, r, ansiEnabled = f)
case (_, CalendarIntervalType | DayTimeIntervalType) => Cast(TimeAdd(l, r), l.dataType)
case (_, CalendarIntervalType | _: DayTimeIntervalType) => Cast(TimeAdd(l, r), l.dataType)
case (CalendarIntervalType, DateType) => DateAddInterval(r, l, ansiEnabled = f)
case (CalendarIntervalType | DayTimeIntervalType, _) => Cast(TimeAdd(r, l), r.dataType)
case (CalendarIntervalType | _: DayTimeIntervalType, _) => Cast(TimeAdd(r, l), r.dataType)
case (DateType, dt) if dt != StringType => DateAdd(l, r)
case (dt, DateType) if dt != StringType => DateAdd(r, l)
case _ => a
}
case s @ Subtract(l, r, f) if s.childrenResolved => (l.dataType, r.dataType) match {
case (DateType, DayTimeIntervalType) =>
case (DateType, _: DayTimeIntervalType) =>
DatetimeSub(l, r, TimeAdd(Cast(l, TimestampType), UnaryMinus(r, f)))
case (DateType, YearMonthIntervalType) =>
DatetimeSub(l, r, DateAddYMInterval(l, UnaryMinus(r, f)))
case (TimestampType, YearMonthIntervalType) =>
DatetimeSub(l, r, TimestampAddYMInterval(l, UnaryMinus(r, f)))
case (CalendarIntervalType, CalendarIntervalType) |
(DayTimeIntervalType, DayTimeIntervalType) => s
(_: DayTimeIntervalType, _: DayTimeIntervalType) => s
case (DateType, CalendarIntervalType) =>
DatetimeSub(l, r, DateAddInterval(l, UnaryMinus(r, f), ansiEnabled = f))
case (_, CalendarIntervalType | DayTimeIntervalType) =>
case (_, CalendarIntervalType | _: DayTimeIntervalType) =>
Cast(DatetimeSub(l, r, TimeAdd(l, UnaryMinus(r, f))), l.dataType)
case (TimestampType, _) => SubtractTimestamps(l, r)
case (_, TimestampType) => SubtractTimestamps(l, r)
@ -389,14 +389,14 @@ class Analyzer(override val catalogManager: CatalogManager)
case (_, CalendarIntervalType) => MultiplyInterval(r, l, f)
case (YearMonthIntervalType, _) => MultiplyYMInterval(l, r)
case (_, YearMonthIntervalType) => MultiplyYMInterval(r, l)
case (DayTimeIntervalType, _) => MultiplyDTInterval(l, r)
case (_, DayTimeIntervalType) => MultiplyDTInterval(r, l)
case (_: DayTimeIntervalType, _) => MultiplyDTInterval(l, r)
case (_, _: DayTimeIntervalType) => MultiplyDTInterval(r, l)
case _ => m
}
case d @ Divide(l, r, f) if d.childrenResolved => (l.dataType, r.dataType) match {
case (CalendarIntervalType, _) => DivideInterval(l, r, f)
case (YearMonthIntervalType, _) => DivideYMInterval(l, r)
case (DayTimeIntervalType, _) => DivideDTInterval(l, r)
case (_: DayTimeIntervalType, _) => DivideDTInterval(l, r)
case _ => d
}
}

View file

@ -302,8 +302,11 @@ package object dsl {
AttributeReference(s, TimestampWithoutTZType, nullable = true)()
/** Creates a new AttributeReference of the day-time interval type */
def dayTimeInterval: AttributeReference = {
AttributeReference(s, DayTimeIntervalType, nullable = true)()
def dayTimeInterval(startField: Byte, endField: Byte): AttributeReference = {
AttributeReference(s, DayTimeIntervalType(startField, endField), nullable = true)()
}
def dayTimeInterval(): AttributeReference = {
AttributeReference(s, DayTimeIntervalType(), nullable = true)()
}
/** Creates a new AttributeReference of the year-month interval type */

View file

@ -114,7 +114,7 @@ object RowEncoder {
createSerializerForSqlDate(inputObject)
}
case DayTimeIntervalType => createSerializerForJavaDuration(inputObject)
case _: DayTimeIntervalType => createSerializerForJavaDuration(inputObject)
case YearMonthIntervalType => createSerializerForJavaPeriod(inputObject)
@ -238,7 +238,7 @@ object RowEncoder {
} else {
ObjectType(classOf[java.sql.Date])
}
case DayTimeIntervalType => ObjectType(classOf[java.time.Duration])
case _: DayTimeIntervalType => ObjectType(classOf[java.time.Duration])
case YearMonthIntervalType => ObjectType(classOf[java.time.Period])
case _: DecimalType => ObjectType(classOf[java.math.BigDecimal])
case StringType => ObjectType(classOf[java.lang.String])
@ -297,7 +297,7 @@ object RowEncoder {
createDeserializerForSqlDate(input)
}
case DayTimeIntervalType => createDeserializerForDuration(input)
case _: DayTimeIntervalType => createDeserializerForDuration(input)
case YearMonthIntervalType => createDeserializerForPeriod(input)

View file

@ -77,7 +77,7 @@ object Cast {
case (TimestampWithoutTZType, DateType) => true
case (StringType, CalendarIntervalType) => true
case (StringType, DayTimeIntervalType) => true
case (StringType, _: DayTimeIntervalType) => true
case (StringType, YearMonthIntervalType) => true
case (StringType, _: NumericType) => true
@ -424,9 +424,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
case YearMonthIntervalType =>
buildCast[Int](_, i => UTF8String.fromString(
IntervalUtils.toYearMonthIntervalString(i, ANSI_STYLE)))
case DayTimeIntervalType =>
case DayTimeIntervalType(startField, endField) =>
buildCast[Long](_, i => UTF8String.fromString(
IntervalUtils.toDayTimeIntervalString(i, ANSI_STYLE)))
IntervalUtils.toDayTimeIntervalString(i, ANSI_STYLE, startField, endField)))
case _ => buildCast[Any](_, o => UTF8String.fromString(o.toString))
}
@ -555,8 +555,11 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
buildCast[UTF8String](_, s => IntervalUtils.safeStringToInterval(s))
}
private[this] def castToDayTimeInterval(from: DataType): Any => Any = from match {
case StringType => buildCast[UTF8String](_, s => IntervalUtils.castStringToDTInterval(s))
private[this] def castToDayTimeInterval(
from: DataType,
it: DayTimeIntervalType): Any => Any = from match {
case StringType => buildCast[UTF8String](_, s =>
IntervalUtils.castStringToDTInterval(s, it.startField, it.endField))
}
private[this] def castToYearMonthInterval(from: DataType): Any => Any = from match {
@ -868,7 +871,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
case TimestampType => castToTimestamp(from)
case TimestampWithoutTZType => castToTimestampWithoutTZ(from)
case CalendarIntervalType => castToInterval(from)
case DayTimeIntervalType => castToDayTimeInterval(from)
case it: DayTimeIntervalType => castToDayTimeInterval(from, it)
case YearMonthIntervalType => castToYearMonthInterval(from)
case BooleanType => castToBoolean(from)
case ByteType => castToByte(from)
@ -929,7 +932,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
case TimestampType => castToTimestampCode(from, ctx)
case TimestampWithoutTZType => castToTimestampWithoutTZCode(from)
case CalendarIntervalType => castToIntervalCode(from)
case DayTimeIntervalType => castToDayTimeIntervalCode(from)
case it: DayTimeIntervalType => castToDayTimeIntervalCode(from, it)
case YearMonthIntervalType => castToYearMonthIntervalCode(from)
case BooleanType => castToBooleanCode(from)
case ByteType => castToByteCode(from, ctx)
@ -1169,13 +1172,23 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
(c, evPrim, evNull) => {
code"$evPrim = UTF8String.fromString($udtRef.deserialize($c).toString());"
}
case i @ (YearMonthIntervalType | DayTimeIntervalType) =>
case YearMonthIntervalType =>
val iu = IntervalUtils.getClass.getName.stripSuffix("$")
val iss = IntervalStringStyles.getClass.getName.stripSuffix("$")
val subType = if (i.isInstanceOf[YearMonthIntervalType]) "YearMonth" else "DayTime"
val f = s"to${subType}IntervalString"
val style = s"$iss$$.MODULE$$.ANSI_STYLE()"
(c, evPrim, _) => code"""$evPrim = UTF8String.fromString($iu.$f($c, $style));"""
(c, evPrim, _) =>
code"""
$evPrim = UTF8String.fromString($iu.toYearMonthIntervalString($c, $style));
"""
case i : DayTimeIntervalType =>
val iu = IntervalUtils.getClass.getName.stripSuffix("$")
val iss = IntervalStringStyles.getClass.getName.stripSuffix("$")
val style = s"$iss$$.MODULE$$.ANSI_STYLE()"
(c, evPrim, _) =>
code"""
$evPrim = UTF8String.fromString($iu.toDayTimeIntervalString($c, $style,
(byte)${i.startField}, (byte)${i.endField}));
"""
case _ =>
(c, evPrim, evNull) => code"$evPrim = UTF8String.fromString(String.valueOf($c));"
}
@ -1399,10 +1412,15 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
}
private[this] def castToDayTimeIntervalCode(from: DataType): CastFunction = from match {
private[this] def castToDayTimeIntervalCode(
from: DataType,
it: DayTimeIntervalType): CastFunction = from match {
case StringType =>
val util = IntervalUtils.getClass.getCanonicalName.stripSuffix("$")
(c, evPrim, _) => code"$evPrim = $util.castStringToDTInterval($c);"
(c, evPrim, _) =>
code"""
$evPrim = $util.castStringToDTInterval($c, (byte)${it.startField}, (byte)${it.endField});
"""
}
private[this] def castToYearMonthIntervalCode(from: DataType): CastFunction = from match {
@ -1974,7 +1992,7 @@ object AnsiCast {
case (DateType, TimestampWithoutTZType) => true
case (StringType, _: CalendarIntervalType) => true
case (StringType, DayTimeIntervalType) => true
case (StringType, _: DayTimeIntervalType) => true
case (StringType, YearMonthIntervalType) => true
case (StringType, DateType) => true

View file

@ -160,7 +160,7 @@ object InterpretedUnsafeProjection {
case IntegerType | DateType | YearMonthIntervalType =>
(v, i) => writer.write(i, v.getInt(i))
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType =>
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType =>
(v, i) => writer.write(i, v.getLong(i))
case FloatType =>

View file

@ -196,7 +196,8 @@ final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGen
// We use INT for DATE and YearMonthIntervalType internally
case IntegerType | DateType | YearMonthIntervalType => new MutableInt
// We use Long for Timestamp, Timestamp without time zone and DayTimeInterval internally
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType => new MutableLong
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType =>
new MutableLong
case FloatType => new MutableFloat
case DoubleType => new MutableDouble
case BooleanType => new MutableBoolean

View file

@ -58,14 +58,14 @@ case class Average(child: Expression) extends DeclarativeAggregate with Implicit
case DecimalType.Fixed(p, s) =>
DecimalType.bounded(p + 4, s + 4)
case _: YearMonthIntervalType => YearMonthIntervalType
case _: DayTimeIntervalType => DayTimeIntervalType
case _: DayTimeIntervalType => DayTimeIntervalType()
case _ => DoubleType
}
private lazy val sumDataType = child.dataType match {
case _ @ DecimalType.Fixed(p, s) => DecimalType.bounded(p + 10, s)
case _: YearMonthIntervalType => YearMonthIntervalType
case _: DayTimeIntervalType => DayTimeIntervalType
case _: DayTimeIntervalType => DayTimeIntervalType()
case _ => DoubleType
}
@ -95,7 +95,7 @@ case class Average(child: Expression) extends DeclarativeAggregate with Implicit
Literal(null, YearMonthIntervalType), DivideYMInterval(sum, count))
case _: DayTimeIntervalType =>
If(EqualTo(count, Literal(0L)),
Literal(null, DayTimeIntervalType), DivideDTInterval(sum, count))
Literal(null, DayTimeIntervalType()), DivideDTInterval(sum, count))
case _ =>
Divide(sum.cast(resultType), count.cast(resultType), failOnError = false)
}

View file

@ -60,7 +60,7 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast
DecimalType.bounded(precision + 10, scale)
case _: IntegralType => LongType
case _: YearMonthIntervalType => YearMonthIntervalType
case _: DayTimeIntervalType => DayTimeIntervalType
case it: DayTimeIntervalType => it
case _ => DoubleType
}

View file

@ -85,7 +85,7 @@ case class UnaryMinus(
val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$")
val method = if (failOnError) "negateExact" else "negate"
defineCodeGen(ctx, ev, c => s"$iu.$method($c)")
case DayTimeIntervalType | YearMonthIntervalType =>
case _: DayTimeIntervalType | YearMonthIntervalType =>
nullSafeCodeGen(ctx, ev, eval => {
val mathClass = classOf[Math].getName
s"${ev.value} = $mathClass.negateExact($eval);"
@ -96,7 +96,7 @@ case class UnaryMinus(
case CalendarIntervalType if failOnError =>
IntervalUtils.negateExact(input.asInstanceOf[CalendarInterval])
case CalendarIntervalType => IntervalUtils.negate(input.asInstanceOf[CalendarInterval])
case DayTimeIntervalType => Math.negateExact(input.asInstanceOf[Long])
case _: DayTimeIntervalType => Math.negateExact(input.asInstanceOf[Long])
case YearMonthIntervalType => Math.negateExact(input.asInstanceOf[Int])
case _ => numeric.negate(input)
}
@ -229,7 +229,7 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant {
case CalendarIntervalType =>
val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$")
defineCodeGen(ctx, ev, (eval1, eval2) => s"$iu.$calendarIntervalMethod($eval1, $eval2)")
case DayTimeIntervalType | YearMonthIntervalType =>
case _: DayTimeIntervalType | YearMonthIntervalType =>
assert(exactMathMethod.isDefined,
s"The expression '$nodeName' must override the exactMathMethod() method " +
"if it is supposed to operate over interval types.")
@ -317,7 +317,7 @@ case class Add(
case CalendarIntervalType =>
IntervalUtils.add(
input1.asInstanceOf[CalendarInterval], input2.asInstanceOf[CalendarInterval])
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
Math.addExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long])
case YearMonthIntervalType =>
Math.addExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int])
@ -363,7 +363,7 @@ case class Subtract(
case CalendarIntervalType =>
IntervalUtils.subtract(
input1.asInstanceOf[CalendarInterval], input2.asInstanceOf[CalendarInterval])
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
Math.subtractExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long])
case YearMonthIntervalType =>
Math.subtractExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int])

View file

@ -1817,7 +1817,7 @@ object CodeGenerator extends Logging {
case ByteType => JAVA_BYTE
case ShortType => JAVA_SHORT
case IntegerType | DateType | YearMonthIntervalType => JAVA_INT
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType => JAVA_LONG
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType => JAVA_LONG
case FloatType => JAVA_FLOAT
case DoubleType => JAVA_DOUBLE
case _: DecimalType => "Decimal"
@ -1838,7 +1838,7 @@ object CodeGenerator extends Logging {
case ByteType => java.lang.Byte.TYPE
case ShortType => java.lang.Short.TYPE
case IntegerType | DateType | YearMonthIntervalType => java.lang.Integer.TYPE
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType =>
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType =>
java.lang.Long.TYPE
case FloatType => java.lang.Float.TYPE
case DoubleType => java.lang.Double.TYPE

View file

@ -236,7 +236,7 @@ object Block {
} else {
args.foreach {
case _: ExprValue | _: Inline | _: Block =>
case _: Boolean | _: Int | _: Long | _: Float | _: Double | _: String =>
case _: Boolean | _: Byte | _: Int | _: Long | _: Float | _: Double | _: String =>
case other => throw QueryExecutionErrors.cannotInterpolateClassIntoCodeBlockError(other)
}

View file

@ -2564,14 +2564,14 @@ case class Sequence(
|1. The start and stop expressions must resolve to the same type.
|2. If start and stop expressions resolve to the 'date' or 'timestamp' type
|then the step expression must resolve to the 'interval' or
|'${YearMonthIntervalType.typeName}' or '${DayTimeIntervalType.typeName}' type,
|'${YearMonthIntervalType.typeName}' or '${DayTimeIntervalType.simpleString}' type,
|otherwise to the same type as the start and stop expressions.
""".stripMargin)
}
}
private def isNotIntervalType(expr: Expression) = expr.dataType match {
case CalendarIntervalType | YearMonthIntervalType | DayTimeIntervalType => false
case CalendarIntervalType | YearMonthIntervalType | _: DayTimeIntervalType => false
case _ => true
}
@ -2774,10 +2774,10 @@ object Sequence {
override val defaultStep: DefaultStep = new DefaultStep(
(dt.ordering.lteq _).asInstanceOf[LessThanOrEqualFn],
DayTimeIntervalType,
DayTimeIntervalType(),
Duration.ofDays(1))
val intervalType: DataType = DayTimeIntervalType
val intervalType: DataType = DayTimeIntervalType()
def splitStep(input: Any): (Int, Int, Long) = {
(0, 0, input.asInstanceOf[Long])

View file

@ -1362,7 +1362,7 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
copy(timeZoneId = Option(timeZoneId))
override def nullSafeEval(start: Any, interval: Any): Any = right.dataType match {
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
timestampAddDayTime(start.asInstanceOf[Long], interval.asInstanceOf[Long], zoneId)
case CalendarIntervalType =>
val i = interval.asInstanceOf[CalendarInterval]
@ -1373,7 +1373,7 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
interval.dataType match {
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
defineCodeGen(ctx, ev, (sd, dt) => s"""$dtu.timestampAddDayTime($sd, $dt, $zid)""")
case CalendarIntervalType =>
defineCodeGen(ctx, ev, (sd, i) => {
@ -2396,7 +2396,7 @@ object DatePart {
throw QueryCompilationErrors.literalTypeUnsupportedForSourceTypeError(fieldStr, source)
source.dataType match {
case YearMonthIntervalType | DayTimeIntervalType | CalendarIntervalType =>
case YearMonthIntervalType | _: DayTimeIntervalType | CalendarIntervalType =>
ExtractIntervalPart.parseExtractField(fieldStr, source, analysisException)
case _ =>
DatePart.parseExtractField(fieldStr, source, analysisException)
@ -2550,7 +2550,7 @@ case class SubtractTimestamps(
override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, TimestampType)
override def dataType: DataType =
if (legacyInterval) CalendarIntervalType else DayTimeIntervalType
if (legacyInterval) CalendarIntervalType else DayTimeIntervalType()
override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
copy(timeZoneId = Option(timeZoneId))
@ -2609,8 +2609,10 @@ case class SubtractDates(
this(left, right, SQLConf.get.legacyIntervalEnabled)
override def inputTypes: Seq[AbstractDataType] = Seq(DateType, DateType)
override def dataType: DataType =
if (legacyInterval) CalendarIntervalType else DayTimeIntervalType
override def dataType: DataType = {
// TODO(SPARK-35727): Return INTERVAL DAY from dates subtraction
if (legacyInterval) CalendarIntervalType else DayTimeIntervalType()
}
@transient
private lazy val evalFunc: (Int, Int) => Any = legacyInterval match {

View file

@ -495,7 +495,7 @@ abstract class HashExpression[E] extends Expression {
case DoubleType => genHashDouble(input, result)
case d: DecimalType => genHashDecimal(ctx, d, input, result)
case CalendarIntervalType => genHashCalendarInterval(input, result)
case DayTimeIntervalType => genHashLong(input, result)
case _: DayTimeIntervalType => genHashLong(input, result)
case YearMonthIntervalType => genHashInt(input, result)
case BinaryType => genHashBytes(input, result)
case StringType => genHashString(input, result)

View file

@ -132,19 +132,19 @@ object ExtractIntervalPart {
ExtractANSIIntervalMonths(source)
case ("MONTH" | "MON" | "MONS" | "MONTHS", CalendarIntervalType) =>
ExtractIntervalMonths(source)
case ("DAY" | "D" | "DAYS", DayTimeIntervalType) =>
case ("DAY" | "D" | "DAYS", _: DayTimeIntervalType) =>
ExtractANSIIntervalDays(source)
case ("DAY" | "D" | "DAYS", CalendarIntervalType) =>
ExtractIntervalDays(source)
case ("HOUR" | "H" | "HOURS" | "HR" | "HRS", DayTimeIntervalType) =>
case ("HOUR" | "H" | "HOURS" | "HR" | "HRS", _: DayTimeIntervalType) =>
ExtractANSIIntervalHours(source)
case ("HOUR" | "H" | "HOURS" | "HR" | "HRS", CalendarIntervalType) =>
ExtractIntervalHours(source)
case ("MINUTE" | "M" | "MIN" | "MINS" | "MINUTES", DayTimeIntervalType) =>
case ("MINUTE" | "M" | "MIN" | "MINS" | "MINUTES", _: DayTimeIntervalType) =>
ExtractANSIIntervalMinutes(source)
case ("MINUTE" | "M" | "MIN" | "MINS" | "MINUTES", CalendarIntervalType) =>
ExtractIntervalMinutes(source)
case ("SECOND" | "S" | "SEC" | "SECONDS" | "SECS", DayTimeIntervalType) =>
case ("SECOND" | "S" | "SEC" | "SECONDS" | "SECS", _: DayTimeIntervalType) =>
ExtractANSIIntervalSeconds(source)
case ("SECOND" | "S" | "SEC" | "SECONDS" | "SECS", CalendarIntervalType) =>
ExtractIntervalSeconds(source)
@ -406,7 +406,7 @@ case class MultiplyDTInterval(
override def right: Expression = num
override def inputTypes: Seq[AbstractDataType] = Seq(DayTimeIntervalType, NumericType)
override def dataType: DataType = DayTimeIntervalType
override def dataType: DataType = DayTimeIntervalType()
@transient
private lazy val evalFunc: (Long, Any) => Any = right.dataType match {
@ -535,7 +535,7 @@ case class DivideDTInterval(
override def right: Expression = num
override def inputTypes: Seq[AbstractDataType] = Seq(DayTimeIntervalType, NumericType)
override def dataType: DataType = DayTimeIntervalType
override def dataType: DataType = DayTimeIntervalType()
@transient
private lazy val evalFunc: (Long, Any) => Any = right.dataType match {

View file

@ -83,7 +83,7 @@ object Literal {
case l: LocalDateTime => Literal(DateTimeUtils.localDateTimeToMicros(l), TimestampWithoutTZType)
case ld: LocalDate => Literal(ld.toEpochDay.toInt, DateType)
case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType)
case d: Duration => Literal(durationToMicros(d), DayTimeIntervalType)
case d: Duration => Literal(durationToMicros(d), DayTimeIntervalType())
case p: Period => Literal(periodToMonths(p), YearMonthIntervalType)
case a: Array[Byte] => Literal(a, BinaryType)
case a: collection.mutable.WrappedArray[_] => apply(a.array)
@ -121,7 +121,7 @@ object Literal {
case _ if clz == classOf[Instant] => TimestampType
case _ if clz == classOf[Timestamp] => TimestampType
case _ if clz == classOf[LocalDateTime] => TimestampWithoutTZType
case _ if clz == classOf[Duration] => DayTimeIntervalType
case _ if clz == classOf[Duration] => DayTimeIntervalType()
case _ if clz == classOf[Period] => YearMonthIntervalType
case _ if clz == classOf[JavaBigDecimal] => DecimalType.SYSTEM_DEFAULT
case _ if clz == classOf[Array[Byte]] => BinaryType
@ -180,7 +180,7 @@ object Literal {
case DateType => create(0, DateType)
case TimestampType => create(0L, TimestampType)
case TimestampWithoutTZType => create(0L, TimestampWithoutTZType)
case DayTimeIntervalType => create(0L, DayTimeIntervalType)
case it: DayTimeIntervalType => create(0L, it)
case YearMonthIntervalType => create(0, YearMonthIntervalType)
case StringType => Literal("")
case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8))
@ -201,7 +201,7 @@ object Literal {
case ByteType => v.isInstanceOf[Byte]
case ShortType => v.isInstanceOf[Short]
case IntegerType | DateType | YearMonthIntervalType => v.isInstanceOf[Int]
case LongType | TimestampType | TimestampWithoutTZType | DayTimeIntervalType =>
case LongType | TimestampType | TimestampWithoutTZType | _: DayTimeIntervalType =>
v.isInstanceOf[Long]
case FloatType => v.isInstanceOf[Float]
case DoubleType => v.isInstanceOf[Double]
@ -346,7 +346,8 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
DateFormatter().format(value.asInstanceOf[Int])
case TimestampType =>
TimestampFormatter.getFractionFormatter(timeZoneId).format(value.asInstanceOf[Long])
case DayTimeIntervalType => toDayTimeIntervalString(value.asInstanceOf[Long], ANSI_STYLE)
case DayTimeIntervalType(startField, endField) =>
toDayTimeIntervalString(value.asInstanceOf[Long], ANSI_STYLE, startField, endField)
case YearMonthIntervalType => toYearMonthIntervalString(value.asInstanceOf[Int], ANSI_STYLE)
case _ =>
other.toString
@ -426,7 +427,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
}
case ByteType | ShortType =>
ExprCode.forNonNullValue(JavaCode.expression(s"($javaType)$value", dataType))
case TimestampType | TimestampWithoutTZType | LongType | DayTimeIntervalType =>
case TimestampType | TimestampWithoutTZType | LongType | _: DayTimeIntervalType =>
toExprCode(s"${value}L")
case _ =>
val constRef = ctx.addReferenceObj("literal", value, javaType)
@ -468,7 +469,8 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
case (i: CalendarInterval, CalendarIntervalType) =>
s"INTERVAL '${i.toString}'"
case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'"
case (i: Long, DayTimeIntervalType) => toDayTimeIntervalString(i, ANSI_STYLE)
case (i: Long, DayTimeIntervalType(startField, endField)) =>
toDayTimeIntervalString(i, ANSI_STYLE, startField, endField)
case (i: Int, YearMonthIntervalType) => toYearMonthIntervalString(i, ANSI_STYLE)
case _ => value.toString
}

View file

@ -104,7 +104,7 @@ case class WindowSpecDefinition(
case (DateType, YearMonthIntervalType) => true
case (TimestampType, CalendarIntervalType) => true
case (TimestampType, YearMonthIntervalType) => true
case (TimestampType, DayTimeIntervalType) => true
case (TimestampType, _: DayTimeIntervalType) => true
case (a, b) => a == b
}
}

View file

@ -2358,7 +2358,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
} else {
assert(calendarInterval.months == 0)
val micros = IntervalUtils.getDuration(calendarInterval, TimeUnit.MICROSECONDS)
Literal(micros, DayTimeIntervalType)
// TODO(SPARK-35737): Parse day-time interval literals to tightest types
Literal(micros, DayTimeIntervalType())
}
} else {
Literal(calendarInterval, CalendarIntervalType)
@ -2513,7 +2514,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
}
override def visitDayTimeIntervalDataType(ctx: DayTimeIntervalDataTypeContext): DataType = {
DayTimeIntervalType
// TODO(SPARK-35736): Parse any day-time interval types in SQL
DayTimeIntervalType()
}
/**

View file

@ -158,7 +158,11 @@ object IntervalUtils {
private val daySecondLiteralRegex =
(s"(?i)^INTERVAL\\s+([+|-])?\\'$daySecondPatternString\\'\\s+DAY\\s+TO\\s+SECOND$$").r
def castStringToDTInterval(input: UTF8String): Long = {
def castStringToDTInterval(
input: UTF8String,
// TODO(SPARK-35735): Take into account day-time interval fields in cast
startField: Byte,
endField: Byte): Long = {
def secondAndMicro(second: String, micro: String): String = {
if (micro != null) {
s"$second$micro"
@ -953,7 +957,12 @@ object IntervalUtils {
* @param style The style of textual representation of the interval
* @return Day-time interval string
*/
def toDayTimeIntervalString(micros: Long, style: IntervalStyle): String = {
def toDayTimeIntervalString(
micros: Long,
style: IntervalStyle,
// TODO(SPARK-35734): Format day-time intervals using type fields
startField: Byte,
endField: Byte): String = {
var sign = ""
var rest = micros
if (micros < 0) {

View file

@ -63,7 +63,8 @@ object TypeUtils {
def checkForAnsiIntervalOrNumericType(
dt: DataType, funcName: String): TypeCheckResult = dt match {
case YearMonthIntervalType | DayTimeIntervalType | NullType => TypeCheckResult.TypeCheckSuccess
case YearMonthIntervalType | _: DayTimeIntervalType | NullType =>
TypeCheckResult.TypeCheckSuccess
case dt if dt.isInstanceOf[NumericType] => TypeCheckResult.TypeCheckSuccess
case other => TypeCheckResult.TypeCheckFailure(
s"function $funcName requires numeric or interval types, not ${other.catalogString}")
@ -116,7 +117,7 @@ object TypeUtils {
def invokeOnceForInterval(dataType: DataType)(f: => Unit): Unit = {
def isInterval(dataType: DataType): Boolean = dataType match {
case CalendarIntervalType | DayTimeIntervalType | YearMonthIntervalType => true
case CalendarIntervalType | _: DayTimeIntervalType | YearMonthIntervalType => true
case _ => false
}
if (dataType.existsRecursively(isInterval)) f

View file

@ -1625,4 +1625,15 @@ private[spark] object QueryCompilationErrors {
def unknownHiveResourceTypeError(resourceType: String): Throwable = {
new AnalysisException(s"Unknown resource type: $resourceType")
}
def invalidDayTimeField(field: Byte): Throwable = {
val supportedIds = DayTimeIntervalType.dayTimeFields
.map(i => s"$i (${DayTimeIntervalType.fieldToString(i)})")
new AnalysisException(s"Invalid field id '$field' in day-time interval. " +
s"Supported interval fields: ${supportedIds.mkString(", ")}.")
}
def invalidDayTimeIntervalType(startFieldName: String, endFieldName: String): Throwable = {
new AnalysisException(s"'interval $startFieldName to $endFieldName' is invalid.")
}
}

View file

@ -171,7 +171,8 @@ object DataType {
private val otherTypes = {
Seq(NullType, DateType, TimestampType, BinaryType, IntegerType, BooleanType, LongType,
DoubleType, FloatType, ShortType, ByteType, StringType, CalendarIntervalType,
DayTimeIntervalType, YearMonthIntervalType, TimestampWithoutTZType)
// TODO(SPARK-35732): Parse DayTimeIntervalType from JSON
DayTimeIntervalType(), YearMonthIntervalType, TimestampWithoutTZType)
.map(t => t.typeName -> t).toMap
}

View file

@ -21,6 +21,8 @@ import scala.math.Ordering
import scala.reflect.runtime.universe.typeTag
import org.apache.spark.annotation.Unstable
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.types.DayTimeIntervalType.fieldToString
/**
* The type represents day-time intervals of the SQL standard. A day-time interval is made up
@ -32,12 +34,15 @@ import org.apache.spark.annotation.Unstable
*
* `DayTimeIntervalType` represents positive as well as negative day-time intervals.
*
* Please use the singleton `DataTypes.DayTimeIntervalType` to refer the type.
* @param startField The leftmost field which the type comprises of. Valid values:
* 0 (DAY), 1 (HOUR), 2 (MINUTE), 3 (SECOND).
* @param endField The rightmost field which the type comprises of. Valid values:
* 0 (DAY), 1 (HOUR), 2 (MINUTE), 3 (SECOND).
*
* @since 3.2.0
*/
@Unstable
class DayTimeIntervalType private() extends AtomicType {
case class DayTimeIntervalType(startField: Byte, endField: Byte) extends AtomicType {
/**
* Internally, values of day-time intervals are stored in `Long` values as amount of time in terms
* of microseconds that are calculated by the formula:
@ -57,16 +62,49 @@ class DayTimeIntervalType private() extends AtomicType {
private[spark] override def asNullable: DayTimeIntervalType = this
override def typeName: String = "interval day to second"
override val typeName: String = {
val startFieldName = fieldToString(startField)
val endFieldName = fieldToString(endField)
if (startFieldName == endFieldName) {
s"interval $startFieldName"
} else if (startField < endField) {
s"interval $startFieldName to $endFieldName"
} else {
throw QueryCompilationErrors.invalidDayTimeIntervalType(startFieldName, endFieldName)
}
}
}
/**
* The companion case object and its class is separated so the companion object also subclasses
* the DayTimeIntervalType class. Otherwise, the companion object would be of type
* "DayTimeIntervalType$" in byte code. Defined with a private constructor so the companion object
* is the only possible instantiation.
* Extra factory methods and pattern matchers for DayTimeIntervalType.
*
* @since 3.2.0
*/
@Unstable
case object DayTimeIntervalType extends DayTimeIntervalType
case object DayTimeIntervalType extends AbstractDataType {
val DAY: Byte = 0
val HOUR: Byte = 1
val MINUTE: Byte = 2
val SECOND: Byte = 3
val dayTimeFields = Seq(DAY, HOUR, MINUTE, SECOND)
def fieldToString(field: Byte): String = field match {
case DAY => "day"
case HOUR => "hour"
case MINUTE => "minute"
case SECOND => "second"
case invalid => throw QueryCompilationErrors.invalidDayTimeField(invalid)
}
val DEFAULT = DayTimeIntervalType(DAY, SECOND)
def apply(): DayTimeIntervalType = DEFAULT
override private[sql] def defaultConcreteType: DataType = DEFAULT
override private[sql] def acceptsType(other: DataType): Boolean = {
other.isInstanceOf[DayTimeIntervalType]
}
override private[sql] def simpleString: String = defaultConcreteType.simpleString
}

View file

@ -55,7 +55,7 @@ private[sql] object ArrowUtils {
}
case NullType => ArrowType.Null.INSTANCE
case YearMonthIntervalType => new ArrowType.Interval(IntervalUnit.YEAR_MONTH)
case DayTimeIntervalType => new ArrowType.Interval(IntervalUnit.DAY_TIME)
case _: DayTimeIntervalType => new ArrowType.Interval(IntervalUnit.DAY_TIME)
case _ =>
throw new UnsupportedOperationException(s"Unsupported data type: ${dt.catalogString}")
}
@ -77,7 +77,7 @@ private[sql] object ArrowUtils {
case ts: ArrowType.Timestamp if ts.getUnit == TimeUnit.MICROSECOND => TimestampType
case ArrowType.Null.INSTANCE => NullType
case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => YearMonthIntervalType
case di: ArrowType.Interval if di.getUnit == IntervalUnit.DAY_TIME => DayTimeIntervalType
case di: ArrowType.Interval if di.getUnit == IntervalUnit.DAY_TIME => DayTimeIntervalType()
case _ => throw new UnsupportedOperationException(s"Unsupported data type: $dt")
}

View file

@ -283,7 +283,7 @@ object RandomDataGenerator {
val ns = rand.nextLong()
new CalendarInterval(months, days, ns)
})
case DayTimeIntervalType => Some(() => Duration.of(rand.nextLong(), ChronoUnit.MICROS))
case _: DayTimeIntervalType => Some(() => Duration.of(rand.nextLong(), ChronoUnit.MICROS))
case YearMonthIntervalType => Some(() => Period.ofMonths(rand.nextInt()).normalized())
case DecimalType.Fixed(precision, scale) => Some(
() => BigDecimal.apply(

View file

@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
/**
* Tests of [[RandomDataGenerator]].
@ -145,7 +146,7 @@ class RandomDataGeneratorSuite extends SparkFunSuite with SQLHelper {
}
test("SPARK-35116: The generated data fits the precision of DayTimeIntervalType in spark") {
Seq(DayTimeIntervalType, YearMonthIntervalType).foreach { dt =>
(dayTimeIntervalTypes :+ YearMonthIntervalType).foreach { dt =>
for (seed <- 1 to 1000) {
val generator = RandomDataGenerator.forType(dt, false, new Random(seed)).get
val toCatalyst = CatalystTypeConverters.createToCatalystConverter(dt)

View file

@ -283,7 +283,8 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper {
Seq(1L, -1L).foreach { sign =>
val us = sign * input
val duration = IntervalUtils.microsToDuration(us)
assert(CatalystTypeConverters.createToScalaConverter(DayTimeIntervalType)(us) === duration)
assert(CatalystTypeConverters.createToScalaConverter(DayTimeIntervalType())(us)
=== duration)
}
}
}

View file

@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest
import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData, IntervalUtils}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
@SQLUserDefinedType(udt = classOf[ExamplePointUDT])
class ExamplePoint(val x: Double, val y: Double) extends Serializable {
@ -353,13 +354,15 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest {
}
test("SPARK-34605: encoding/decoding DayTimeIntervalType to/from java.time.Duration") {
val schema = new StructType().add("d", DayTimeIntervalType)
val encoder = RowEncoder(schema).resolveAndBind()
val duration = java.time.Duration.ofDays(1)
val row = toRow(encoder, Row(duration))
assert(row.getLong(0) === IntervalUtils.durationToMicros(duration))
val readback = fromRow(encoder, row)
assert(readback.get(0).equals(duration))
dayTimeIntervalTypes.foreach { dayTimeIntervalType =>
val schema = new StructType().add("d", dayTimeIntervalType)
val encoder = RowEncoder(schema).resolveAndBind()
val duration = java.time.Duration.ofDays(1)
val row = toRow(encoder, Row(duration))
assert(row.getLong(0) === IntervalUtils.durationToMicros(duration))
val readback = fromRow(encoder, row)
assert(readback.get(0).equals(duration))
}
}
test("SPARK-34615: encoding/decoding YearMonthIntervalType to/from java.time.Period") {

View file

@ -634,15 +634,15 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
checkExceptionInExpression[ArithmeticException](
Subtract(
Literal.create(Duration.ofDays(-106751991), DayTimeIntervalType),
Literal.create(Duration.ofDays(10), DayTimeIntervalType),
Literal.create(Duration.ofDays(-106751991), DayTimeIntervalType()),
Literal.create(Duration.ofDays(10), DayTimeIntervalType()),
failOnError
),
"overflow")
checkExceptionInExpression[ArithmeticException](
Add(
Literal.create(Duration.ofDays(106751991), DayTimeIntervalType),
Literal.create(Duration.ofDays(10), DayTimeIntervalType),
Literal.create(Duration.ofDays(106751991), DayTimeIntervalType()),
Literal.create(Duration.ofDays(10), DayTimeIntervalType()),
failOnError
),
"overflow")

View file

@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.catalyst.util.IntervalUtils.microsToDuration
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
import org.apache.spark.unsafe.types.UTF8String
abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
@ -841,8 +842,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper {
s"INTERVAL '$intervalPayload' DAY TO SECOND")
}
checkConsistencyBetweenInterpretedAndCodegen(
(child: Expression) => Cast(child, StringType), DayTimeIntervalType)
dayTimeIntervalTypes.foreach { it =>
checkConsistencyBetweenInterpretedAndCodegen((child: Expression) =>
Cast(child, StringType), it)
}
}
}
@ -1813,44 +1816,46 @@ class CastSuite extends CastSuiteBase {
}
test("SPARK-35112: Cast string to day-time interval") {
checkEvaluation(cast(Literal.create("0 0:0:0"), DayTimeIntervalType), 0L)
checkEvaluation(cast(Literal.create("0 0:0:0"), DayTimeIntervalType()), 0L)
checkEvaluation(cast(Literal.create(" interval '0 0:0:0' Day TO second "),
DayTimeIntervalType), 0L)
DayTimeIntervalType()), 0L)
checkEvaluation(cast(Literal.create("INTERVAL '1 2:03:04' DAY TO SECOND"),
DayTimeIntervalType), 93784000000L)
DayTimeIntervalType()), 93784000000L)
checkEvaluation(cast(Literal.create("INTERVAL '1 03:04:00' DAY TO SECOND"),
DayTimeIntervalType), 97440000000L)
DayTimeIntervalType()), 97440000000L)
checkEvaluation(cast(Literal.create("INTERVAL '1 03:04:00.0000' DAY TO SECOND"),
DayTimeIntervalType), 97440000000L)
checkEvaluation(cast(Literal.create("1 2:03:04"), DayTimeIntervalType), 93784000000L)
DayTimeIntervalType()), 97440000000L)
checkEvaluation(cast(Literal.create("1 2:03:04"), DayTimeIntervalType()), 93784000000L)
checkEvaluation(cast(Literal.create("INTERVAL '-10 2:03:04' DAY TO SECOND"),
DayTimeIntervalType), -871384000000L)
checkEvaluation(cast(Literal.create("-10 2:03:04"), DayTimeIntervalType), -871384000000L)
checkEvaluation(cast(Literal.create("-106751991 04:00:54.775808"), DayTimeIntervalType),
DayTimeIntervalType()), -871384000000L)
checkEvaluation(cast(Literal.create("-10 2:03:04"), DayTimeIntervalType()), -871384000000L)
checkEvaluation(cast(Literal.create("-106751991 04:00:54.775808"), DayTimeIntervalType()),
Long.MinValue)
checkEvaluation(cast(Literal.create("106751991 04:00:54.775807"), DayTimeIntervalType),
checkEvaluation(cast(Literal.create("106751991 04:00:54.775807"), DayTimeIntervalType()),
Long.MaxValue)
Seq("-106751991 04:00:54.775808", "106751991 04:00:54.775807").foreach { interval =>
val ansiInterval = s"INTERVAL '$interval' DAY TO SECOND"
checkEvaluation(
cast(cast(Literal.create(interval), DayTimeIntervalType), StringType), ansiInterval)
cast(cast(Literal.create(interval), DayTimeIntervalType()), StringType), ansiInterval)
checkEvaluation(cast(cast(Literal.create(ansiInterval),
DayTimeIntervalType), StringType), ansiInterval)
DayTimeIntervalType()), StringType), ansiInterval)
}
Seq("INTERVAL '-106751991 04:00:54.775809' YEAR TO MONTH",
"INTERVAL '106751991 04:00:54.775808' YEAR TO MONTH").foreach { interval =>
val e = intercept[IllegalArgumentException] {
cast(Literal.create(interval), DayTimeIntervalType).eval()
cast(Literal.create(interval), DayTimeIntervalType()).eval()
}.getMessage
assert(e.contains("Interval string must match day-time format of"))
}
Seq(Byte.MaxValue, Short.MaxValue, Int.MaxValue, Long.MaxValue, Long.MinValue + 1,
Long.MinValue).foreach { duration =>
val interval = Literal.create(Duration.of(duration, ChronoUnit.MICROS), DayTimeIntervalType)
checkEvaluation(cast(cast(interval, StringType), DayTimeIntervalType), duration)
val interval = Literal.create(
Duration.of(duration, ChronoUnit.MICROS),
DayTimeIntervalType())
checkEvaluation(cast(cast(interval, StringType), DayTimeIntervalType()), duration)
}
}

View file

@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, TimeZoneUTC}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
@ -1650,18 +1651,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(
TimeAdd(
Literal(new Timestamp(sdf.parse("2021-01-01 00:00:00.123").getTime)),
Literal.create(null, DayTimeIntervalType),
Literal.create(null, DayTimeIntervalType()),
timeZoneId),
null)
checkEvaluation(
TimeAdd(
Literal.create(null, TimestampType),
Literal.create(null, DayTimeIntervalType),
Literal.create(null, DayTimeIntervalType()),
timeZoneId),
null)
checkConsistencyBetweenInterpretedAndCodegen(
(ts: Expression, interval: Expression) => TimeAdd(ts, interval, timeZoneId),
TimestampType, DayTimeIntervalType)
dayTimeIntervalTypes.foreach { it =>
checkConsistencyBetweenInterpretedAndCodegen((ts: Expression, interval: Expression) =>
TimeAdd(ts, interval, timeZoneId), TimestampType, it)
}
}
}
}

View file

@ -698,7 +698,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("SPARK-35113: HashExpression support DayTimeIntervalType/YearMonthIntervalType") {
val dayTime = Literal.create(Duration.ofSeconds(1237123123), DayTimeIntervalType)
val dayTime = Literal.create(Duration.ofSeconds(1237123123), DayTimeIntervalType())
val yearMonth = Literal.create(Period.ofMonths(1234), YearMonthIntervalType)
checkEvaluation(Murmur3Hash(Seq(dayTime), 10), -428664612)
checkEvaluation(Murmur3Hash(Seq(yearMonth), 10), -686520021)

View file

@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._
import org.apache.spark.sql.catalyst.util.IntervalUtils.{safeStringToInterval, stringToInterval}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DayTimeIntervalType, Decimal, DecimalType, YearMonthIntervalType}
import org.apache.spark.sql.types.DataTypeTestUtils.numericTypes
import org.apache.spark.sql.types.DataTypeTestUtils.{dayTimeIntervalTypes, numericTypes}
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
@ -312,6 +312,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
}
// TODO(SPARK-35728): Check multiply/divide of day-time intervals of any fields by numeric
test("SPARK-34850: multiply day-time interval by numeric") {
Seq(
(Duration.ofHours(-123), Literal(null, DecimalType.USER_DEFAULT)) -> null,
@ -339,9 +340,11 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
numericTypes.foreach { numType =>
checkConsistencyBetweenInterpretedAndCodegenAllowingException(
(interval: Expression, num: Expression) => MultiplyDTInterval(interval, num),
DayTimeIntervalType, numType)
dayTimeIntervalTypes.foreach { it =>
checkConsistencyBetweenInterpretedAndCodegenAllowingException(
(interval: Expression, num: Expression) => MultiplyDTInterval(interval, num),
it, numType)
}
}
}
@ -378,6 +381,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
}
// TODO(SPARK-35728): Check multiply/divide of day-time intervals of any fields by numeric
test("SPARK-34875: divide day-time interval by numeric") {
Seq(
(Duration.ofDays(-123), Literal(null, DecimalType.USER_DEFAULT)) -> null,
@ -405,9 +409,11 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
numericTypes.foreach { numType =>
checkConsistencyBetweenInterpretedAndCodegenAllowingException(
(interval: Expression, num: Expression) => DivideDTInterval(interval, num),
DayTimeIntervalType, numType)
dayTimeIntervalTypes.foreach { it =>
checkConsistencyBetweenInterpretedAndCodegenAllowingException(
(interval: Expression, num: Expression) => DivideDTInterval(interval, num),
it, numType)
}
}
}
@ -441,9 +447,13 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(ExtractANSIIntervalSeconds(Literal(d)),
IntervalUtils.getSeconds(IntervalUtils.durationToMicros(d)))
}
checkEvaluation(ExtractANSIIntervalDays(Literal(null, DayTimeIntervalType)), null)
checkEvaluation(ExtractANSIIntervalHours(Literal(null, DayTimeIntervalType)), null)
checkEvaluation(ExtractANSIIntervalMinutes(Literal(null, DayTimeIntervalType)), null)
checkEvaluation(ExtractANSIIntervalSeconds(Literal(null, DayTimeIntervalType)), null)
checkEvaluation(ExtractANSIIntervalDays(
Literal(null, DayTimeIntervalType())), null)
checkEvaluation(ExtractANSIIntervalHours(
Literal(null, DayTimeIntervalType())), null)
checkEvaluation(ExtractANSIIntervalMinutes(
Literal(null, DayTimeIntervalType())), null)
checkEvaluation(ExtractANSIIntervalSeconds(
Literal(null, DayTimeIntervalType())), null)
}
}

View file

@ -51,7 +51,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(Literal.create(null, TimestampType), null)
checkEvaluation(Literal.create(null, CalendarIntervalType), null)
checkEvaluation(Literal.create(null, YearMonthIntervalType), null)
checkEvaluation(Literal.create(null, DayTimeIntervalType), null)
checkEvaluation(Literal.create(null, DayTimeIntervalType()), null)
checkEvaluation(Literal.create(null, ArrayType(ByteType, true)), null)
checkEvaluation(Literal.create(null, ArrayType(StringType, true)), null)
checkEvaluation(Literal.create(null, MapType(StringType, IntegerType)), null)
@ -80,7 +80,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
}
checkEvaluation(Literal.default(CalendarIntervalType), new CalendarInterval(0, 0, 0L))
checkEvaluation(Literal.default(YearMonthIntervalType), 0)
checkEvaluation(Literal.default(DayTimeIntervalType), 0L)
checkEvaluation(Literal.default(DayTimeIntervalType()), 0L)
checkEvaluation(Literal.default(ArrayType(StringType)), Array())
checkEvaluation(Literal.default(MapType(IntegerType, StringType)), Map())
checkEvaluation(Literal.default(StructType(StructField("a", StringType) :: Nil)), Row(""))

View file

@ -178,7 +178,7 @@ object LiteralGenerator {
calendarIntervalLiterGen.map { calendarIntervalLiteral =>
Literal.create(
calendarIntervalLiteral.value.asInstanceOf[CalendarInterval].extractAsDuration(),
DayTimeIntervalType)
DayTimeIntervalType())
}
}
@ -203,7 +203,7 @@ object LiteralGenerator {
case BinaryType => binaryLiteralGen
case CalendarIntervalType => calendarIntervalLiterGen
case DecimalType.Fixed(precision, scale) => decimalLiteralGen(precision, scale)
case DayTimeIntervalType => dayTimeIntervalLiteralGen
case _: DayTimeIntervalType => dayTimeIntervalLiteralGen
case YearMonthIntervalType => yearMonthIntervalLiteralGen
case dt => throw new IllegalArgumentException(s"not supported type $dt")
}

View file

@ -23,6 +23,7 @@ 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.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.types.UTF8String
@ -30,7 +31,7 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper {
val fixedLengthTypes = Array[DataType](
BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType,
DateType, TimestampType, YearMonthIntervalType, DayTimeIntervalType)
DateType, TimestampType, YearMonthIntervalType) ++ dayTimeIntervalTypes
val variableLengthTypes = Array(
StringType, DecimalType.defaultConcreteType, CalendarIntervalType, BinaryType,
@ -43,14 +44,16 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper {
testBothCodegenAndInterpreted("fixed-length types") {
val inputRow = InternalRow.fromSeq(Seq(
true, 3.toByte, 15.toShort, -83, 129L, 1.0f, 5.0, 1, 2L, Int.MaxValue, Long.MinValue))
true, 3.toByte, 15.toShort, -83, 129L, 1.0f, 5.0, 1, 2L, Int.MaxValue) ++
Seq.tabulate(dayTimeIntervalTypes.length)(_ => Long.MaxValue))
val proj = createMutableProjection(fixedLengthTypes)
assert(proj(inputRow) === inputRow)
}
testBothCodegenAndInterpreted("unsafe buffer") {
val inputRow = InternalRow.fromSeq(Seq(
false, 1.toByte, 9.toShort, -18, 53L, 3.2f, 7.8, 4, 9L, Int.MinValue, Long.MaxValue))
false, 1.toByte, 9.toShort, -18, 53L, 3.2f, 7.8, 4, 9L, Int.MinValue) ++
Seq.tabulate(dayTimeIntervalTypes.length)(_ => Long.MaxValue))
val numFields = fixedLengthTypes.length
val numBytes = Platform.BYTE_ARRAY_OFFSET + UnsafeRow.calculateBitSetWidthInBytes(numFields) +
UnsafeRow.WORD_SIZE * numFields

View file

@ -64,7 +64,7 @@ class DataTypeParserSuite extends SparkFunSuite {
checkDataType("void", NullType)
checkDataType("interval", CalendarIntervalType)
checkDataType("INTERVAL YEAR TO MONTH", YearMonthIntervalType)
checkDataType("interval day to second", DayTimeIntervalType)
checkDataType("interval day to second", DayTimeIntervalType())
checkDataType("array<doublE>", ArrayType(DoubleType, true))
checkDataType("Array<map<int, tinYint>>", ArrayType(MapType(IntegerType, ByteType, true), true))

View file

@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, HIVE
import org.apache.spark.sql.catalyst.util.IntervalUtils._
import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DayTimeIntervalType
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
class IntervalUtilsSuite extends SparkFunSuite with SQLHelper {
@ -519,6 +520,7 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper {
}
test("SPARK-35016: format day-time intervals") {
import DayTimeIntervalType._
Seq(
0L -> ("0 00:00:00.000000000", "INTERVAL '0 00:00:00' DAY TO SECOND"),
-1L -> ("-0 00:00:00.000001000", "INTERVAL '-0 00:00:00.000001' DAY TO SECOND"),
@ -528,8 +530,8 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper {
Long.MinValue -> ("-106751991 04:00:54.775808000",
"INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND")
).foreach { case (micros, (hiveIntervalStr, ansiIntervalStr)) =>
assert(toDayTimeIntervalString(micros, ANSI_STYLE) === ansiIntervalStr)
assert(toDayTimeIntervalString(micros, HIVE_STYLE) === hiveIntervalStr)
assert(toDayTimeIntervalString(micros, ANSI_STYLE, DAY, SECOND) === ansiIntervalStr)
assert(toDayTimeIntervalString(micros, HIVE_STYLE, DAY, SECOND) === hiveIntervalStr)
}
}
}

View file

@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonParseException
import org.apache.spark.{SparkException, SparkFunSuite}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat
import org.apache.spark.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
class DataTypeSuite extends SparkFunSuite {
@ -256,7 +257,7 @@ class DataTypeSuite extends SparkFunSuite {
checkDataTypeFromDDL(VarcharType(11))
checkDataTypeFromDDL(YearMonthIntervalType)
checkDataTypeFromDDL(DayTimeIntervalType)
checkDataTypeFromDDL(DayTimeIntervalType())
val metadata = new MetadataBuilder()
.putString("name", "age")
@ -325,7 +326,7 @@ class DataTypeSuite extends SparkFunSuite {
checkDefaultSize(VarcharType(5), 5)
checkDefaultSize(VarcharType(10), 10)
checkDefaultSize(YearMonthIntervalType, 4)
checkDefaultSize(DayTimeIntervalType, 8)
dayTimeIntervalTypes.foreach(checkDefaultSize(_, 8))
def checkEqualsIgnoreCompatibleNullability(
from: DataType,

View file

@ -17,6 +17,8 @@
package org.apache.spark.sql.types
import org.apache.spark.sql.types.DayTimeIntervalType.{DAY, HOUR, MINUTE, SECOND}
/**
* Utility functions for working with DataTypes in tests.
*/
@ -49,13 +51,24 @@ object DataTypeTestUtils {
// TODO: remove this once we find out how to handle decimal properly in property check
val numericTypeWithoutDecimal: Set[DataType] = integralType ++ Set(DoubleType, FloatType)
val dayTimeIntervalTypes: Seq[DayTimeIntervalType] = Seq(
DayTimeIntervalType(DAY, DAY),
DayTimeIntervalType(DAY, HOUR),
DayTimeIntervalType(DAY, MINUTE),
DayTimeIntervalType(DAY, SECOND),
DayTimeIntervalType(HOUR, HOUR),
DayTimeIntervalType(HOUR, MINUTE),
DayTimeIntervalType(HOUR, SECOND),
DayTimeIntervalType(MINUTE, MINUTE),
DayTimeIntervalType(MINUTE, SECOND),
DayTimeIntervalType(SECOND, SECOND))
/**
* Instances of all [[NumericType]]s and [[CalendarIntervalType]]
*/
val numericAndInterval: Set[DataType] = numericTypeWithoutDecimal ++ Set(
CalendarIntervalType,
DayTimeIntervalType,
YearMonthIntervalType)
YearMonthIntervalType) ++ dayTimeIntervalTypes
/**
* All the types that support ordering
@ -67,8 +80,7 @@ object DataTypeTestUtils {
DateType,
StringType,
BinaryType,
DayTimeIntervalType,
YearMonthIntervalType)
YearMonthIntervalType) ++ dayTimeIntervalTypes
/**
* All the types that we can use in a property check
@ -85,9 +97,8 @@ object DataTypeTestUtils {
StringType,
TimestampType,
TimestampWithoutTZType,
DayTimeIntervalType,
YearMonthIntervalType
)
) ++ dayTimeIntervalTypes
/**
* Instances of [[ArrayType]] for all [[AtomicType]]s. Arrays of these types may contain null.

View file

@ -49,7 +49,7 @@ class ArrowUtilsSuite extends SparkFunSuite {
roundtrip(DecimalType.SYSTEM_DEFAULT)
roundtrip(DateType)
roundtrip(YearMonthIntervalType)
roundtrip(DayTimeIntervalType)
roundtrip(DayTimeIntervalType())
val tsExMsg = intercept[UnsupportedOperationException] {
roundtrip(TimestampType)
}

View file

@ -119,8 +119,8 @@ object HiveResult {
}.mkString("{", ",", "}")
case (period: Period, YearMonthIntervalType) =>
toYearMonthIntervalString(periodToMonths(period), HIVE_STYLE)
case (duration: Duration, DayTimeIntervalType) =>
toDayTimeIntervalString(durationToMicros(duration), HIVE_STYLE)
case (duration: Duration, DayTimeIntervalType(startField, endField)) =>
toDayTimeIntervalString(durationToMicros(duration), HIVE_STYLE, startField, endField)
case (other, _: UserDefinedType[_]) => other.toString
}
}

View file

@ -159,7 +159,7 @@ abstract class HashMapGenerator(
dataType match {
case BooleanType => hashInt(s"$input ? 1 : 0")
case ByteType | ShortType | IntegerType | DateType | YearMonthIntervalType => hashInt(input)
case LongType | TimestampType | DayTimeIntervalType => hashLong(input)
case LongType | TimestampType | _: DayTimeIntervalType => hashLong(input)
case FloatType => hashInt(s"Float.floatToIntBits($input)")
case DoubleType => hashLong(s"Double.doubleToLongBits($input)")
case d: DecimalType =>

View file

@ -91,7 +91,7 @@ sealed trait BufferSetterGetterUtils {
(row: InternalRow, ordinal: Int) =>
if (row.isNullAt(ordinal)) null else row.getInt(ordinal)
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
(row: InternalRow, ordinal: Int) =>
if (row.isNullAt(ordinal)) null else row.getLong(ordinal)
@ -203,7 +203,7 @@ sealed trait BufferSetterGetterUtils {
row.setNullAt(ordinal)
}
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
(row: InternalRow, ordinal: Int, value: Any) =>
if (value != null) {
row.setLong(ordinal, value.asInstanceOf[Long])

View file

@ -76,7 +76,7 @@ object ArrowWriter {
new StructWriter(vector, children.toArray)
case (NullType, vector: NullVector) => new NullWriter(vector)
case (YearMonthIntervalType, vector: IntervalYearVector) => new IntervalYearWriter(vector)
case (DayTimeIntervalType, vector: IntervalDayVector) => new IntervalDayWriter(vector)
case (_: DayTimeIntervalType, vector: IntervalDayVector) => new IntervalDayWriter(vector)
case (dt, _) =>
throw QueryExecutionErrors.unsupportedDataTypeError(dt)
}

View file

@ -140,7 +140,7 @@ private[sql] object ColumnAccessor {
case ByteType => new ByteColumnAccessor(buf)
case ShortType => new ShortColumnAccessor(buf)
case IntegerType | DateType | YearMonthIntervalType => new IntColumnAccessor(buf)
case LongType | TimestampType | DayTimeIntervalType => new LongColumnAccessor(buf)
case LongType | TimestampType | _: DayTimeIntervalType => new LongColumnAccessor(buf)
case FloatType => new FloatColumnAccessor(buf)
case DoubleType => new DoubleColumnAccessor(buf)
case StringType => new StringColumnAccessor(buf)

View file

@ -175,7 +175,7 @@ private[columnar] object ColumnBuilder {
case ByteType => new ByteColumnBuilder
case ShortType => new ShortColumnBuilder
case IntegerType | DateType | YearMonthIntervalType => new IntColumnBuilder
case LongType | TimestampType | DayTimeIntervalType => new LongColumnBuilder
case LongType | TimestampType | _: DayTimeIntervalType => new LongColumnBuilder
case FloatType => new FloatColumnBuilder
case DoubleType => new DoubleColumnBuilder
case StringType => new StringColumnBuilder

View file

@ -290,7 +290,7 @@ private[columnar] object YEAR_MONTH_INTERVAL extends NativeColumnType(YearMonthI
}
}
private[columnar] object DAY_TIME_INTERVAL extends NativeColumnType(DayTimeIntervalType, 8) {
private[columnar] object DAY_TIME_INTERVAL extends NativeColumnType(DayTimeIntervalType(), 8) {
override def append(v: Long, buffer: ByteBuffer): Unit = {
buffer.putLong(v)
}
@ -818,7 +818,7 @@ private[columnar] object ColumnType {
case ByteType => BYTE
case ShortType => SHORT
case IntegerType | DateType | YearMonthIntervalType => INT
case LongType | TimestampType | DayTimeIntervalType => LONG
case LongType | TimestampType | _: DayTimeIntervalType => LONG
case FloatType => FLOAT
case DoubleType => DOUBLE
case StringType => STRING

View file

@ -81,7 +81,8 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera
case ByteType => classOf[ByteColumnAccessor].getName
case ShortType => classOf[ShortColumnAccessor].getName
case IntegerType | DateType | YearMonthIntervalType => classOf[IntColumnAccessor].getName
case LongType | TimestampType | DayTimeIntervalType => classOf[LongColumnAccessor].getName
case LongType | TimestampType | _: DayTimeIntervalType =>
classOf[LongColumnAccessor].getName
case FloatType => classOf[FloatColumnAccessor].getName
case DoubleType => classOf[DoubleColumnAccessor].getName
case StringType => classOf[StringColumnAccessor].getName

View file

@ -99,7 +99,7 @@ trait WindowExecBase extends UnaryExecNode {
case (TimestampType, CalendarIntervalType) => TimeAdd(expr, boundOffset, Some(timeZone))
case (TimestampType, YearMonthIntervalType) =>
TimestampAddYMInterval(expr, boundOffset, Some(timeZone))
case (TimestampType, DayTimeIntervalType) => TimeAdd(expr, boundOffset, Some(timeZone))
case (TimestampType, _: DayTimeIntervalType) => TimeAdd(expr, boundOffset, Some(timeZone))
case (a, b) if a == b => Add(expr, boundOffset)
}
val bound = MutableProjection.create(boundExpr :: Nil, child.output)

View file

@ -1130,7 +1130,8 @@ class DataFrameAggregateSuite extends QueryTest
checkAnswer(sumDF, Row(Period.of(2, 5, 0), Duration.ofDays(0)))
assert(find(sumDF.queryExecution.executedPlan)(_.isInstanceOf[HashAggregateExec]).isDefined)
assert(sumDF.schema == StructType(Seq(StructField("sum(year-month)", YearMonthIntervalType),
StructField("sum(day-time)", DayTimeIntervalType))))
// TODO(SPARK-35729): Check all day-time interval types in aggregate expressions
StructField("sum(day-time)", DayTimeIntervalType()))))
val sumDF2 = df.groupBy($"class").agg(sum($"year-month"), sum($"day-time"))
checkAnswer(sumDF2, Row(1, Period.ofMonths(10), Duration.ofDays(10)) ::
@ -1139,7 +1140,8 @@ class DataFrameAggregateSuite extends QueryTest
assert(find(sumDF2.queryExecution.executedPlan)(_.isInstanceOf[HashAggregateExec]).isDefined)
assert(sumDF2.schema == StructType(Seq(StructField("class", IntegerType, false),
StructField("sum(year-month)", YearMonthIntervalType),
StructField("sum(day-time)", DayTimeIntervalType))))
// TODO(SPARK-35729): Check all day-time interval types in aggregate expressions
StructField("sum(day-time)", DayTimeIntervalType()))))
val error = intercept[SparkException] {
checkAnswer(df2.select(sum($"year-month")), Nil)
@ -1168,7 +1170,8 @@ class DataFrameAggregateSuite extends QueryTest
checkAnswer(avgDF, Row(Period.ofMonths(7), Duration.ofDays(0)))
assert(find(avgDF.queryExecution.executedPlan)(_.isInstanceOf[HashAggregateExec]).isDefined)
assert(avgDF.schema == StructType(Seq(StructField("avg(year-month)", YearMonthIntervalType),
StructField("avg(day-time)", DayTimeIntervalType))))
// TODO(SPARK-35729): Check all day-time interval types in aggregate expressions
StructField("avg(day-time)", DayTimeIntervalType()))))
val avgDF2 = df.groupBy($"class").agg(avg($"year-month"), avg($"day-time"))
checkAnswer(avgDF2, Row(1, Period.ofMonths(10), Duration.ofDays(10)) ::
@ -1177,7 +1180,8 @@ class DataFrameAggregateSuite extends QueryTest
assert(find(avgDF2.queryExecution.executedPlan)(_.isInstanceOf[HashAggregateExec]).isDefined)
assert(avgDF2.schema == StructType(Seq(StructField("class", IntegerType, false),
StructField("avg(year-month)", YearMonthIntervalType),
StructField("avg(day-time)", DayTimeIntervalType))))
// TODO(SPARK-35729): Check all day-time interval types in aggregate expressions
StructField("avg(day-time)", DayTimeIntervalType()))))
val error = intercept[SparkException] {
checkAnswer(df2.select(avg($"year-month")), Nil)

View file

@ -879,18 +879,21 @@ class UDFSuite extends QueryTest with SharedSparkSession {
val plusHour = udf((d: java.time.Duration) => d.plusHours(1))
val result = input.select(plusHour($"d").as("new_d"))
checkAnswer(result, Row(java.time.Duration.ofDays(1)) :: Nil)
assert(result.schema === new StructType().add("new_d", DayTimeIntervalType))
// TODO(SPARK-35730): Check all day-time interval types in UDF
assert(result.schema === new StructType().add("new_d", DayTimeIntervalType()))
// UDF produces `null`
val nullFunc = udf((_: java.time.Duration) => null.asInstanceOf[java.time.Duration])
val nullResult = input.select(nullFunc($"d").as("null_d"))
checkAnswer(nullResult, Row(null) :: Nil)
assert(nullResult.schema === new StructType().add("null_d", DayTimeIntervalType))
// TODO(SPARK-35730): Check all day-time interval types in UDF
assert(nullResult.schema === new StructType().add("null_d", DayTimeIntervalType()))
// Input parameter of UDF is null
val nullInput = Seq(null.asInstanceOf[java.time.Duration]).toDF("null_d")
val constDuration = udf((_: java.time.Duration) => java.time.Duration.ofMinutes(10))
val constResult = nullInput.select(constDuration($"null_d").as("10_min"))
checkAnswer(constResult, Row(java.time.Duration.ofMinutes(10)) :: Nil)
assert(constResult.schema === new StructType().add("10_min", DayTimeIntervalType))
// TODO(SPARK-35730): Check all day-time interval types in UDF
assert(constResult.schema === new StructType().add("10_min", DayTimeIntervalType()))
// Error in the conversion of UDF result to the internal representation of day-time interval
val overflowFunc = udf((d: java.time.Duration) => d.plusDays(Long.MaxValue))
val e = intercept[SparkException] {

View file

@ -57,7 +57,7 @@ class ArrowWriterSuite extends SparkFunSuite {
case DateType => reader.getInt(rowId)
case TimestampType => reader.getLong(rowId)
case YearMonthIntervalType => reader.getInt(rowId)
case DayTimeIntervalType => reader.getLong(rowId)
case _: DayTimeIntervalType => reader.getLong(rowId)
}
assert(value === datum)
}
@ -78,13 +78,13 @@ class ArrowWriterSuite extends SparkFunSuite {
check(TimestampType, Seq(0L, 3.6e9.toLong, null, 8.64e10.toLong), "America/Los_Angeles")
check(NullType, Seq(null, null, null))
check(YearMonthIntervalType, Seq(null, 0, 1, -1, Int.MaxValue, Int.MinValue))
check(DayTimeIntervalType, Seq(null, 0L, 1000L, -1000L, (Long.MaxValue - 807L),
check(DayTimeIntervalType(), Seq(null, 0L, 1000L, -1000L, (Long.MaxValue - 807L),
(Long.MinValue + 808L)))
}
test("long overflow for DayTimeIntervalType")
{
val schema = new StructType().add("value", DayTimeIntervalType, nullable = true)
val schema = new StructType().add("value", DayTimeIntervalType(), nullable = true)
val writer = ArrowWriter.create(schema, null)
val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0))
val valueVector = writer.root.getFieldVectors().get(0).asInstanceOf[IntervalDayVector]
@ -129,7 +129,7 @@ class ArrowWriterSuite extends SparkFunSuite {
case DateType => reader.getInts(0, data.size)
case TimestampType => reader.getLongs(0, data.size)
case YearMonthIntervalType => reader.getInts(0, data.size)
case DayTimeIntervalType => reader.getLongs(0, data.size)
case _: DayTimeIntervalType => reader.getLongs(0, data.size)
}
assert(values === data)
@ -145,7 +145,8 @@ class ArrowWriterSuite extends SparkFunSuite {
check(DateType, (0 until 10))
check(TimestampType, (0 until 10).map(_ * 4.32e10.toLong), "America/Los_Angeles")
check(YearMonthIntervalType, (0 until 10))
check(DayTimeIntervalType, (-10 until 10).map(_ * 1000.toLong))
// TODO(SPARK-35731): Check all day-time interval types in arrow
check(DayTimeIntervalType(), (-10 until 10).map(_ * 1000.toLong))
}
test("array") {

View file

@ -121,7 +121,7 @@ private[hive] class SparkExecuteStatementOperation(
false,
timeFormatters)
case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] |
YearMonthIntervalType | DayTimeIntervalType =>
YearMonthIntervalType | _: DayTimeIntervalType =>
to += toHiveString((from.get(ordinal), dataTypes(ordinal)), false, timeFormatters)
}
}
@ -378,7 +378,7 @@ object SparkExecuteStatementOperation {
case NullType => "void"
case CalendarIntervalType => StringType.catalogString
case YearMonthIntervalType => "interval_year_month"
case DayTimeIntervalType => "interval_day_time"
case _: DayTimeIntervalType => "interval_day_time"
case other => other.catalogString
}
new FieldSchema(field.name, attrTypeString, field.getComment.getOrElse(""))

View file

@ -131,7 +131,7 @@ private[hive] class SparkGetColumnsOperation(
*/
private def getColumnSize(typ: DataType): Option[Int] = typ match {
case dt @ (BooleanType | _: NumericType | DateType | TimestampType |
CalendarIntervalType | NullType | YearMonthIntervalType | DayTimeIntervalType) =>
CalendarIntervalType | NullType | YearMonthIntervalType | _: DayTimeIntervalType) =>
Some(dt.defaultSize)
case CharType(n) => Some(n)
case StructType(fields) =>
@ -186,7 +186,7 @@ private[hive] class SparkGetColumnsOperation(
case _: MapType => java.sql.Types.JAVA_OBJECT
case _: StructType => java.sql.Types.STRUCT
// Hive's year-month and day-time intervals are mapping to java.sql.Types.OTHER
case _: CalendarIntervalType | YearMonthIntervalType | DayTimeIntervalType =>
case _: CalendarIntervalType | YearMonthIntervalType | _: DayTimeIntervalType =>
java.sql.Types.OTHER
case _ => throw new IllegalArgumentException(s"Unrecognized type name: ${typ.sql}")
}

View file

@ -418,8 +418,8 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase {
assert(rowSet.getString("TABLE_NAME") === viewName2)
assert(rowSet.getString("COLUMN_NAME") === "i")
assert(rowSet.getInt("DATA_TYPE") === java.sql.Types.OTHER)
assert(rowSet.getString("TYPE_NAME").equalsIgnoreCase(DayTimeIntervalType.sql))
assert(rowSet.getInt("COLUMN_SIZE") === DayTimeIntervalType.defaultSize)
assert(rowSet.getString("TYPE_NAME").equalsIgnoreCase(DayTimeIntervalType().sql))
assert(rowSet.getInt("COLUMN_SIZE") === DayTimeIntervalType().defaultSize)
assert(rowSet.getInt("DECIMAL_DIGITS") === 0)
assert(rowSet.getInt("NUM_PREC_RADIX") === 0)
assert(rowSet.getInt("NULLABLE") === 0)

View file

@ -840,7 +840,7 @@ private[hive] trait HiveInspectors {
case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector
case DateType => PrimitiveObjectInspectorFactory.javaDateObjectInspector
case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector
case DayTimeIntervalType =>
case _: DayTimeIntervalType =>
PrimitiveObjectInspectorFactory.javaHiveIntervalDayTimeObjectInspector
case YearMonthIntervalType =>
PrimitiveObjectInspectorFactory.javaHiveIntervalYearMonthObjectInspector
@ -889,7 +889,7 @@ private[hive] trait HiveInspectors {
getDecimalWritableConstantObjectInspector(value)
case Literal(_, NullType) =>
getPrimitiveNullWritableConstantObjectInspector
case Literal(_, DayTimeIntervalType) =>
case Literal(_, _: DayTimeIntervalType) =>
getHiveIntervalDayTimeWritableConstantObjectInspector
case Literal(_, YearMonthIntervalType) =>
getHiveIntervalYearMonthWritableConstantObjectInspector
@ -969,8 +969,8 @@ private[hive] trait HiveInspectors {
case _: JavaDateObjectInspector => DateType
case _: WritableTimestampObjectInspector => TimestampType
case _: JavaTimestampObjectInspector => TimestampType
case _: WritableHiveIntervalDayTimeObjectInspector => DayTimeIntervalType
case _: JavaHiveIntervalDayTimeObjectInspector => DayTimeIntervalType
case _: WritableHiveIntervalDayTimeObjectInspector => DayTimeIntervalType()
case _: JavaHiveIntervalDayTimeObjectInspector => DayTimeIntervalType()
case _: WritableHiveIntervalYearMonthObjectInspector => YearMonthIntervalType
case _: JavaHiveIntervalYearMonthObjectInspector => YearMonthIntervalType
case _: WritableVoidObjectInspector => NullType
@ -1155,7 +1155,7 @@ private[hive] trait HiveInspectors {
case DateType => dateTypeInfo
case TimestampType => timestampTypeInfo
case NullType => voidTypeInfo
case DayTimeIntervalType => intervalDayTimeTypeInfo
case _: DayTimeIntervalType => intervalDayTimeTypeInfo
case YearMonthIntervalType => intervalYearMonthTypeInfo
case dt =>
throw new AnalysisException(

View file

@ -31,6 +31,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.DataTypeTestUtils.dayTimeIntervalTypes
import org.apache.spark.tags.SlowHiveTest
import org.apache.spark.unsafe.UnsafeAlignedOffset
@ -890,7 +891,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te
FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5),
DateType, TimestampType,
ArrayType(IntegerType), MapType(StringType, LongType), struct,
new TestUDT.MyDenseVectorUDT())
new TestUDT.MyDenseVectorUDT()) ++ dayTimeIntervalTypes
// Right now, we will use SortAggregate to handle UDAFs.
// UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortAggregate to use
// UnsafeRow as the aggregation buffer. While, dataTypes will trigger

View file

@ -542,9 +542,10 @@ class HiveScriptTransformationSuite extends BaseScriptTransformationSuite with T
(child: SparkPlan) => createScriptTransformationExec(
script = "cat",
output = Seq(
AttributeReference("a", DayTimeIntervalType)(),
AttributeReference("b", DayTimeIntervalType)(),
AttributeReference("c", DayTimeIntervalType)(),
// TODO(SPARK-35733): Check all day-time interval types in HiveInspectors tests
AttributeReference("a", DayTimeIntervalType())(),
AttributeReference("b", DayTimeIntervalType())(),
AttributeReference("c", DayTimeIntervalType())(),
AttributeReference("d", YearMonthIntervalType)()),
child = child,
ioschema = hiveIOSchema),
@ -563,7 +564,8 @@ class HiveScriptTransformationSuite extends BaseScriptTransformationSuite with T
df,
(child: SparkPlan) => createScriptTransformationExec(
script = "cat",
output = Seq(AttributeReference("a", DayTimeIntervalType)()),
// TODO(SPARK-35733): Check all day-time interval types in HiveInspectors tests
output = Seq(AttributeReference("a", DayTimeIntervalType())()),
child = child,
ioschema = hiveIOSchema),
df.select($"a").collect())