This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new d53831f  [SPARK-35704][SQL] Add fields to `DayTimeIntervalType`
d53831f is described below

commit d53831ff5cef618cfbc08dcea84ca9bb61ed9903
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Fri Jun 11 16:16:33 2021 +0300

    [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.g...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../spark/sql/catalyst/expressions/UnsafeRow.java  |  9 ++--
 .../java/org/apache/spark/sql/types/DataTypes.java | 19 ++++++--
 .../sql/catalyst/CatalystTypeConverters.scala      |  3 +-
 .../apache/spark/sql/catalyst/InternalRow.scala    |  4 +-
 .../spark/sql/catalyst/JavaTypeInference.scala     |  2 +-
 .../spark/sql/catalyst/ScalaReflection.scala       |  6 +--
 .../spark/sql/catalyst/SerializerBuildHelper.scala |  2 +-
 .../spark/sql/catalyst/analysis/Analyzer.scala     | 22 ++++-----
 .../apache/spark/sql/catalyst/dsl/package.scala    |  7 ++-
 .../spark/sql/catalyst/encoders/RowEncoder.scala   |  6 +--
 .../spark/sql/catalyst/expressions/Cast.scala      | 46 ++++++++++++------
 .../expressions/InterpretedUnsafeProjection.scala  |  2 +-
 .../catalyst/expressions/SpecificInternalRow.scala |  3 +-
 .../catalyst/expressions/aggregate/Average.scala   |  6 +--
 .../sql/catalyst/expressions/aggregate/Sum.scala   |  2 +-
 .../sql/catalyst/expressions/arithmetic.scala      | 10 ++--
 .../expressions/codegen/CodeGenerator.scala        |  4 +-
 .../catalyst/expressions/codegen/javaCode.scala    |  2 +-
 .../expressions/collectionOperations.scala         |  8 ++--
 .../catalyst/expressions/datetimeExpressions.scala | 14 +++---
 .../spark/sql/catalyst/expressions/hash.scala      |  2 +-
 .../catalyst/expressions/intervalExpressions.scala | 12 ++---
 .../spark/sql/catalyst/expressions/literals.scala  | 16 ++++---
 .../catalyst/expressions/windowExpressions.scala   |  2 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala     |  6 ++-
 .../spark/sql/catalyst/util/IntervalUtils.scala    | 13 +++++-
 .../apache/spark/sql/catalyst/util/TypeUtils.scala |  5 +-
 .../spark/sql/errors/QueryCompilationErrors.scala  | 11 +++++
 .../org/apache/spark/sql/types/DataType.scala      |  3 +-
 .../spark/sql/types/DayTimeIntervalType.scala      | 54 ++++++++++++++++++----
 .../org/apache/spark/sql/util/ArrowUtils.scala     |  4 +-
 .../org/apache/spark/sql/RandomDataGenerator.scala |  2 +-
 .../spark/sql/RandomDataGeneratorSuite.scala       |  3 +-
 .../sql/catalyst/CatalystTypeConvertersSuite.scala |  3 +-
 .../sql/catalyst/encoders/RowEncoderSuite.scala    | 17 ++++---
 .../expressions/ArithmeticExpressionSuite.scala    |  8 ++--
 .../spark/sql/catalyst/expressions/CastSuite.scala | 39 +++++++++-------
 .../expressions/DateExpressionsSuite.scala         | 12 +++--
 .../expressions/HashExpressionsSuite.scala         |  2 +-
 .../expressions/IntervalExpressionsSuite.scala     | 32 ++++++++-----
 .../expressions/LiteralExpressionSuite.scala       |  4 +-
 .../catalyst/expressions/LiteralGenerator.scala    |  4 +-
 .../expressions/MutableProjectionSuite.scala       |  9 ++--
 .../sql/catalyst/parser/DataTypeParserSuite.scala  |  2 +-
 .../sql/catalyst/util/IntervalUtilsSuite.scala     |  6 ++-
 .../org/apache/spark/sql/types/DataTypeSuite.scala |  5 +-
 .../apache/spark/sql/types/DataTypeTestUtils.scala | 23 ++++++---
 .../apache/spark/sql/util/ArrowUtilsSuite.scala    |  2 +-
 .../apache/spark/sql/execution/HiveResult.scala    |  4 +-
 .../sql/execution/aggregate/HashMapGenerator.scala |  2 +-
 .../spark/sql/execution/aggregate/udaf.scala       |  4 +-
 .../spark/sql/execution/arrow/ArrowWriter.scala    |  2 +-
 .../sql/execution/columnar/ColumnAccessor.scala    |  2 +-
 .../sql/execution/columnar/ColumnBuilder.scala     |  2 +-
 .../spark/sql/execution/columnar/ColumnType.scala  |  4 +-
 .../columnar/GenerateColumnAccessor.scala          |  3 +-
 .../sql/execution/window/WindowExecBase.scala      |  2 +-
 .../apache/spark/sql/DataFrameAggregateSuite.scala | 12 +++--
 .../test/scala/org/apache/spark/sql/UDFSuite.scala |  9 ++--
 .../sql/execution/arrow/ArrowWriterSuite.scala     | 11 +++--
 .../SparkExecuteStatementOperation.scala           |  4 +-
 .../thriftserver/SparkGetColumnsOperation.scala    |  4 +-
 .../thriftserver/SparkMetadataOperationSuite.scala |  4 +-
 .../org/apache/spark/sql/hive/HiveInspectors.scala | 10 ++--
 .../sql/hive/execution/AggregationQuerySuite.scala |  3 +-
 .../execution/HiveScriptTransformationSuite.scala  | 10 ++--
 66 files changed, 359 insertions(+), 211 deletions(-)

diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
index 0c6685d..572e901 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
@@ -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;
   }
 
   
//////////////////////////////////////////////////////////////////////////////
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java 
b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java
index 00bf42b..6604837 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java
@@ -100,11 +100,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.
    */
   public static final DataType YearMonthIntervalType = 
YearMonthIntervalType$.MODULE$;
@@ -146,6 +141,20 @@ public class DataTypes {
   }
 
   /**
+   * 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}.
    */
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
index abd3bf4..2efdf37 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
@@ -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)
     }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
index 202c718..ab668d7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
@@ -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])
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
index 807eb8c..6a31d4a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala
@@ -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 =>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 7ecf32da..c6854e9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -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]
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
index 0624698..9045d53 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala
@@ -116,7 +116,7 @@ object SerializerBuildHelper {
   def createSerializerForJavaDuration(inputObject: Expression): Expression = {
     StaticInvoke(
       IntervalUtils.getClass,
-      DayTimeIntervalType,
+      DayTimeIntervalType(),
       "durationToMicros",
       inputObject :: Nil,
       returnNullable = false)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index d38ddaf..668a661 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -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
         }
       }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index 86998a7..211af78 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -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 */
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
index 83b9197..cf22b35 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala
@@ -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)
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 2f63159..cd99ee2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -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 style = s"$iss$$.MODULE$$.ANSI_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 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.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
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
index e072c9a..79dbce0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala
@@ -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 =>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
index 849870f..891ac82 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificInternalRow.scala
@@ -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
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
index 82ad2df..a64ca57 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
@@ -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)
   }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
index 16cd9d7..09d88c4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
@@ -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
   }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index ec220de..9527df9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -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])
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index eec04d1..db7a349 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -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
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
index c840cdf..dbe9a81 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala
@@ -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)
         }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index 3de87de..72041e0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -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])
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 33e1110..b43bc07 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -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 {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
index 65e7714..9b8b2b9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
@@ -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)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
index 94ca6cc..512cba2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
@@ -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 {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index 2725999..92a6e83 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -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
   }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index fe9c41e..2960ec2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -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
   }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 8c4f021..8d7ff5c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -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()
   }
 
   /**
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
index 2a11d5f..c18cca9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
@@ -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) {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
index f020161..ea1e227 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala
@@ -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
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index d06e950..e2822f7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -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.")
+  }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
index a861856..11d33f0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala
@@ -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
   }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala
index 0582f20..bce1bdc 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DayTimeIntervalType.scala
@@ -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
+}
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
index ce8acd1..67d9dfd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
@@ -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")
   }
 
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
index e9db14b..3b4978e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
@@ -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(
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
index 69dca2c..cd8fae5 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
@@ -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)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala
index c116dab..3a90e94 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala
@@ -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)
       }
     }
   }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
index b333f12..316845d 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
@@ -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") {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
index 5d07912..ec8ae7f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
@@ -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")
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
index a8b9a26..d3d693b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
@@ -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)
       }
   }
 
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 74865e3..77fdc3f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -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)
+      }
     }
   }
 }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
index 97c2797..585f03e 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
@@ -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)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
index cf2f505..833829c 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
@@ -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)
   }
 }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
index f256528..7baffbf 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
@@ -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(""))
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala
index 8af2391..ac04897 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala
@@ -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")
     }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala
index 3d11ff9..8804000 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala
@@ -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
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
index fd6df92..8e0a01d 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
@@ -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))
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
index 87d306a..8db63e7 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
@@ -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)
     }
   }
 }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
index e78a426..3c85eef 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala
@@ -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,
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeTestUtils.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeTestUtils.scala
index 3a24daf..12f3783 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeTestUtils.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeTestUtils.scala
@@ -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.
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
index 5ae74c5..7955d6b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
@@ -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)
     }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
index a13abdc..b49757f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala
@@ -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
   }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
index 9b7ef23..2487312 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
@@ -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 =>
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
index 33cff7f..83caca2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
@@ -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])
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala
index fcae7ac..6786c2c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala
@@ -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)
     }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
index 6283528..16c3191 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
@@ -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)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
index 261e5dd..e2a9f90 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
@@ -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
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
index 2f74fab..cd6b74a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
@@ -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
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
index 376aa53..f3ac428 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
@@ -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
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
index c3ae4d9..f9b2c92 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
@@ -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)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
index a01e7e4..ff8361a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
@@ -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)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index d7ac2cd..6327744 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -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] {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala
index 1684633..52fc021 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala
@@ -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") {
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index e6cf989..6f0c32b 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -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(""))
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
index a354050..2d8d103 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala
@@ -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}")
   }
diff --git 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
index bc2d413..bac8757 100644
--- 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
+++ 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala
@@ -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)
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 37a1fc0..3f83c2f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -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(
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index 70dcfb0..069bc73 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -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
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala
index 71a745b..b396ddc 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationSuite.scala
@@ -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())

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to