diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md
index 0add62b10ed6..5c8d17d270fc 100644
--- a/docs/sql-ref-datatypes.md
+++ b/docs/sql-ref-datatypes.md
@@ -46,6 +46,9 @@ Spark SQL and DataFrames support the following data types:
absolute point in time.
- `DateType`: Represents values comprising values of fields year, month and day, without a
time-zone.
+* CalendarInterval type
+ - `CalendarIntervalType`: The data type representing calendar time intervals. The calendar time interval is stored
+ internally in two components: number of months and the number of microseconds.
* Complex types
- `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of
elements with the type of `elementType`. `containsNull` is used to indicate if
@@ -159,6 +162,13 @@ You can access them by doing
DateType
+
+ | CalendarIntervalType |
+ org.apache.spark.sql.types.CalendarInterval |
+
+ CalendarIntervalType
+ |
+
| ArrayType |
scala.collection.Seq |
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java
index eea7149d0259..57190d4a4aa8 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGetters.java
@@ -19,10 +19,10 @@
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.CalendarInterval;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.catalyst.util.MapData;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
public interface SpecializedGetters {
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
index db6401b18c0e..cd1f8fc17363 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java
@@ -36,7 +36,6 @@
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.bitset.BitSetMethods;
import org.apache.spark.unsafe.hash.Murmur3_x86_32;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET;
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 8fd6029e976e..965bf5ff2e2f 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
@@ -37,7 +37,6 @@
import org.apache.spark.unsafe.array.ByteArrayMethods;
import org.apache.spark.unsafe.bitset.BitSetMethods;
import org.apache.spark.unsafe.hash.Murmur3_x86_32;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
import static org.apache.spark.sql.types.DataTypes.*;
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java
index 95263a0da95a..3762919a63d8 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java
@@ -19,10 +19,10 @@
import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData;
import org.apache.spark.sql.catalyst.expressions.UnsafeMapData;
import org.apache.spark.sql.catalyst.expressions.UnsafeRow;
+import org.apache.spark.sql.types.CalendarInterval;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java
similarity index 99%
rename from common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
rename to sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java
index 908ff1983e6b..fcf63020547d 100644
--- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/CalendarInterval.java
@@ -15,16 +15,20 @@
* limitations under the License.
*/
-package org.apache.spark.unsafe.types;
+package org.apache.spark.sql.types;
import java.io.Serializable;
import java.util.Locale;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import org.apache.spark.annotation.Unstable;
+
/**
- * The internal representation of interval type.
+ * The representation class of interval type.
*/
+
+@Unstable
public final class CalendarInterval implements Serializable {
public static final long MICROS_PER_MILLI = 1000L;
public static final long MICROS_PER_SECOND = MICROS_PER_MILLI * 1000;
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java
index f18d00359c90..a6035bb0120c 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java
@@ -17,9 +17,9 @@
package org.apache.spark.sql.vectorized;
import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.types.CalendarInterval;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java
index 147dd24e16ce..007343f4c9b1 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java
@@ -22,7 +22,6 @@
import org.apache.spark.sql.catalyst.util.ArrayData;
import org.apache.spark.sql.catalyst.util.GenericArrayData;
import org.apache.spark.sql.types.*;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java
index a2feac869ece..8e5334924f3b 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java
@@ -22,7 +22,6 @@
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.types.*;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
index 4b9d3c5f5991..7d7b88213715 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java
@@ -20,7 +20,6 @@
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.types.*;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
index 494387ecfe0c..2177b3da2ce7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
@@ -22,7 +22,7 @@ import scala.util.hashing.MurmurHash3
import org.apache.spark.annotation.Stable
import org.apache.spark.sql.catalyst.expressions.GenericRow
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.types.{CalendarInterval, StructType}
/**
* @since 1.3.0
@@ -159,6 +159,7 @@ trait Row extends Serializable {
* ArrayType -> scala.collection.Seq (use getList for java.util.List)
* MapType -> scala.collection.Map (use getJavaMap for java.util.Map)
* StructType -> org.apache.spark.sql.Row
+ * CalendarIntervalType -> org.apache.spark.sql.types.CalendarInterval
* }}}
*/
def apply(i: Int): Any = get(i)
@@ -184,6 +185,7 @@ trait Row extends Serializable {
* ArrayType -> scala.collection.Seq (use getList for java.util.List)
* MapType -> scala.collection.Map (use getJavaMap for java.util.Map)
* StructType -> org.apache.spark.sql.Row
+ * CalendarIntervalType -> org.apache.spark.sql.types.CalendarInterval
* }}}
*/
def get(i: Int): Any
@@ -327,6 +329,14 @@ trait Row extends Serializable {
*/
def getStruct(i: Int): Row = getAs[Row](i)
+ /**
+ * Returns value at position i of calendar Interval type as a
+ * [[org.apache.spark.sql.types.CalendarInterval]] object.
+ *
+ * @throws ClassCastException when data type does not match.
+ */
+ def getCalendarInterval(i: Int): CalendarInterval = CalendarInterval.fromString(get(i).toString)
+
/**
* Returns the value at position i.
* For primitive types if value is null it returns 'zero value' specific for primitive
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 ceb315550038..293ff3b97052 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
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, _}
import org.apache.spark.sql.catalyst.expressions.objects._
import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
/**
@@ -671,6 +671,8 @@ object ScalaReflection extends ScalaReflection {
Schema(TimestampType, nullable = true)
case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => Schema(DateType, nullable = true)
case t if isSubtype(t, localTypeOf[java.sql.Date]) => Schema(DateType, nullable = true)
+ case t if isSubtype(t, localTypeOf[CalendarInterval]) =>
+ Schema(CalendarIntervalType, nullable = true)
case t if isSubtype(t, localTypeOf[BigDecimal]) =>
Schema(DecimalType.SYSTEM_DEFAULT, nullable = true)
case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
index c1d72f9b58a4..69eeb9f3e5b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
@@ -25,8 +25,6 @@ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
-
/**
* Helper object for stream joins. See [[StreamingSymmetricHashJoinExec]] in SQL for more details.
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 765018f07d87..49cc7f5bb9d1 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
@@ -108,6 +108,14 @@ object RowEncoder {
createSerializerForSqlDate(inputObject)
}
+ case CalendarIntervalType =>
+ StaticInvoke(
+ classOf[CalendarInterval],
+ CalendarIntervalType,
+ "fromString",
+ inputObject :: Nil,
+ returnNullable = false)
+
case d: DecimalType =>
CheckOverflow(StaticInvoke(
Decimal.getClass,
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 5001b4350d77..553da1ebb9ad 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
@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.UTF8StringBuilder
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper}
object Cast {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala
index 7770684a5b39..adf6ea6e6fd6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/JoinedRow.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
/**
* A mutable wrapper that makes two rows appear as a single concatenated row. Designed to
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
index 9aae678deb4b..b760abeb5003 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
@@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
case class TimeWindow(
timeColumn: Expression,
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 e4276e33acbd..f4f62e7c8e28 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
@@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
@ExpressionDescription(
usage = "_FUNC_(expr) - Returns the negated value of `expr`.",
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 1d010ff5abfc..c845f7442837 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
@@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
-import org.apache.spark.unsafe.types._
+import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.{ParentClassLoader, Utils}
/**
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 5314821ea3a5..4ed199df4f9b 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
@@ -35,7 +35,6 @@ import org.apache.spark.unsafe.UTF8StringBuilder
import org.apache.spark.unsafe.array.ByteArrayMethods
import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH
import org.apache.spark.unsafe.types.{ByteArray, UTF8String}
-import org.apache.spark.unsafe.types.CalendarInterval
import org.apache.spark.util.collection.OpenHashSet
/**
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 89a6d23b1d73..6d4cc8f2ab2d 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
@@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
/**
* Common base class for time zone aware expressions.
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 2aa1e6cc518c..8dfc9517bd40 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
@@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.hash.Murmur3_x86_32
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
////////////////////////////////////////////////////////////////////////////////////////////////////
// This file defines all the expressions for hashing.
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 9cef3ecadc54..8b642122efdb 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
@@ -45,7 +45,7 @@ import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types._
+import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.Utils
object Literal {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index 65539a2f00e6..edf4d60cae57 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
/**
* An extended version of [[InternalRow]] that implements all special getters, toString
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 c334d09b691e..04b8b3b148ba 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
@@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate
import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.random.RandomSampler
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
index 8441c2c481ec..6b15a834c536 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala
@@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.plans.logical
import java.util.concurrent.TimeUnit
import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.types.CalendarInterval
import org.apache.spark.sql.types.MetadataBuilder
-import org.apache.spark.unsafe.types.CalendarInterval
object EventTimeWatermark {
/** The [[org.apache.spark.sql.types.Metadata]] key used to hold the eventTime watermark delay. */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala
index 83ad08d8e175..3e7d49f1513a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala
@@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.util
import scala.collection.JavaConverters._
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.types.{DataType, Decimal}
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.sql.types.{CalendarInterval, DataType, Decimal}
+import org.apache.spark.unsafe.types.UTF8String
private object GenericArrayData {
diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java
similarity index 98%
rename from common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java
rename to sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java
index c307d74e0ba0..66a6f4c044ad 100644
--- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java
+++ b/sql/catalyst/src/test/java/org/apache/spark/sql/types/CalendarIntervalSuite.java
@@ -15,12 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.unsafe.types;
+package org.apache.spark.sql.types;
import org.junit.Test;
import static org.junit.Assert.*;
-import static org.apache.spark.unsafe.types.CalendarInterval.*;
+import static org.apache.spark.sql.types.CalendarInterval.*;
public class CalendarIntervalSuite {
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 d361e6248e2f..17fb6c88bcce 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
@@ -25,7 +25,6 @@ import scala.util.Random
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
/**
* Random data generators for Spark SQL DataTypes. These generators do not generate uniformly random
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
index f9cd9c3c398f..5442625dec9c 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
@@ -48,7 +48,8 @@ case class NullableData(
decimalField: java.math.BigDecimal,
dateField: Date,
timestampField: Timestamp,
- binaryField: Array[Byte])
+ binaryField: Array[Byte],
+ intervalField: CalendarInterval)
case class OptionalData(
intField: Option[Int],
@@ -58,7 +59,8 @@ case class OptionalData(
shortField: Option[Short],
byteField: Option[Byte],
booleanField: Option[Boolean],
- structField: Option[PrimitiveData])
+ structField: Option[PrimitiveData],
+ intervalField: Option[CalendarInterval])
case class ComplexData(
arrayField: Seq[Int],
@@ -200,7 +202,8 @@ class ScalaReflectionSuite extends SparkFunSuite {
StructField("decimalField", DecimalType.SYSTEM_DEFAULT, nullable = true),
StructField("dateField", DateType, nullable = true),
StructField("timestampField", TimestampType, nullable = true),
- StructField("binaryField", BinaryType, nullable = true))),
+ StructField("binaryField", BinaryType, nullable = true),
+ StructField("intervalField", CalendarIntervalType, nullable = true))),
nullable = true))
}
@@ -215,7 +218,9 @@ class ScalaReflectionSuite extends SparkFunSuite {
StructField("shortField", ShortType, nullable = true),
StructField("byteField", ByteType, nullable = true),
StructField("booleanField", BooleanType, nullable = true),
- StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true))),
+ StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true),
+ StructField("intervalField", CalendarIntervalType, nullable = true)
+ )),
nullable = true))
}
@@ -256,7 +261,8 @@ class ScalaReflectionSuite extends SparkFunSuite {
StructField("floatField", FloatType, nullable = false),
StructField("shortField", ShortType, nullable = false),
StructField("byteField", ByteType, nullable = false),
- StructField("booleanField", BooleanType, nullable = false))),
+ StructField("booleanField", BooleanType, nullable = false),
+ StructField("intervalField", CalendarIntervalType, nullable = false))),
nullable = true),
StructField(
"nestedArrayField",
@@ -287,7 +293,8 @@ class ScalaReflectionSuite extends SparkFunSuite {
test("convert PrimitiveData to catalyst") {
val data = PrimitiveData(1, 1, 1, 1, 1, 1, true)
- val convertedData = InternalRow(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true)
+ val convertedData = InternalRow(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true,
+ new CalendarInterval(2, 0))
val dataType = schemaFor[PrimitiveData].dataType
assert(CatalystTypeConverters.createToCatalystConverter(dataType)(data) === convertedData)
}
@@ -295,10 +302,10 @@ class ScalaReflectionSuite extends SparkFunSuite {
test("convert Option[Product] to catalyst") {
val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true)
val data = OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true),
- Some(primitiveData))
+ Some(primitiveData), Some(new CalendarInterval(2, 0)))
val dataType = schemaFor[OptionalData].dataType
val convertedData = InternalRow(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true,
- InternalRow(1, 1, 1, 1, 1, 1, true))
+ InternalRow(1, 1, 1, 1, 1, 1, true), new CalendarInterval(2, 0))
assert(CatalystTypeConverters.createToCatalystConverter(dataType)(data) === convertedData)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
index 50c38145ae21..eea96dfb50e6 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
@@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
class TypeCoercionSuite extends AnalysisTest {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
index 0fe646edb340..67b6c7af27ff 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
@@ -30,8 +30,7 @@ import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.{FlatMapGroupsWithState, _}
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
import org.apache.spark.sql.streaming.OutputMode
-import org.apache.spark.sql.types.{IntegerType, LongType, MetadataBuilder}
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.{CalendarInterval, IntegerType, LongType, MetadataBuilder}
/** A dummy command for testing unsupported operations. */
case class DummyCommand() extends Command
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
index d2acfb968670..f79dab384d67 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
@@ -207,9 +207,10 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes
productTest(
OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true),
- Some(PrimitiveData(1, 1, 1, 1, 1, 1, true))))
+ Some(PrimitiveData(1, 1, 1, 1, 1, 1, true)),
+ Some(new CalendarInterval(2, 0))))
- productTest(OptionalData(None, None, None, None, None, None, None, None))
+ productTest(OptionalData(None, None, None, None, None, None, None, None, None))
encodeDecodeTest(Seq(Some(1), None), "Option in array")
encodeDecodeTest(Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), "Option in map")
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 1a1cab823d4f..83e97f4b00aa 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
@@ -261,6 +261,16 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest {
assert(convertedBack.getSeq(2) == Seq(Seq(Seq(0L, null), null), null))
}
+ test("RowEncoder should support CalendarInterval as the external type for CalendarIntervalType") {
+ val schema = new StructType()
+ .add("calendarInterval", CalendarIntervalType)
+ val encoder = RowEncoder(schema).resolveAndBind()
+ val input = Row("interval '1' day")
+ val row = encoder.toRow(input)
+ val convertedBack = encoder.fromRow(row)
+ assert(convertedBack.getCalendarInterval(0) == CalendarInterval.fromString("interval '1' day"))
+ }
+
test("RowEncoder should throw RuntimeException if input row object is null") {
val schema = new StructType().add("int", IntegerType)
val encoder = RowEncoder(schema)
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 ffb14e283868..19edc1cc1368 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
@@ -807,8 +807,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("cast between string and interval") {
- import org.apache.spark.unsafe.types.CalendarInterval
-
checkEvaluation(Cast(Literal(""), CalendarIntervalType), null)
checkEvaluation(Cast(Literal("interval -3 month 7 hours"), CalendarIntervalType),
new CalendarInterval(-3, 7 * CalendarInterval.MICROS_PER_HOUR))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
index 603073b40d7a..03282cc6f24d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
@@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH
-import org.apache.spark.unsafe.types.CalendarInterval
class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
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 ae3549b4aaf5..6b43921f8417 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
@@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
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 b5cfaf8f4b0f..125cd6c7ce0b 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
@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData}
import org.apache.spark.sql.types.{ArrayType, StructType, _}
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val random = new scala.util.Random
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 269f1a09ac53..ae999c847c39 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
@@ -30,8 +30,6 @@ import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
-
class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
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 b111797c3588..6ac9931e09d5 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
@@ -25,7 +25,6 @@ import org.scalacheck.{Arbitrary, Gen}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
/**
* Property is a high-level specification of behavior that should hold for a range of data points.
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 0d594eb10962..26f998707d2c 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
@@ -22,7 +22,6 @@ import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala
index a171885471a3..2f2e4eba6a2e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, Generic
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class InvokeTargetClass extends Serializable {
def filterInt(e: Any): Any = e.asInstanceOf[Int] > 0
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
index 69523fa81bc6..2ef124dc9a64 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.types.{IntegerType, LongType, _}
import org.apache.spark.unsafe.array.ByteArrayMethods
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase
with ExpressionEvalHelper {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala
index 01aa3579aea9..4a60769d6271 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.BoundReference
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, MapData}
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class GenerateUnsafeProjectionSuite extends SparkFunSuite {
test("Test unsafe projection string access pattern") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index 3ec8d18bc871..9de9faf40397 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -25,8 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.types.{BooleanType, IntegerType}
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.{BooleanType, CalendarInterval, IntegerType}
class FilterPushdownSuite extends PlanTest {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala
index f6d1898dc64a..4247cc69270e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala
@@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.types.IntegerType
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.{CalendarInterval, IntegerType}
class LeftSemiPushdownSuite extends PlanTest {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
index 5da2bf059758..5765ab82e1ff 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
@@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
/**
* Test basic expression parsing.
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala
index 0b9e023b0b45..558953a05b60 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder}
import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.Platform
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class UnsafeArraySuite extends SparkFunSuite {
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java
index 829f3ce750fe..a436c8555b2a 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java
@@ -33,7 +33,7 @@
import org.apache.spark.sql.vectorized.ColumnarArray;
import org.apache.spark.sql.vectorized.ColumnarBatch;
import org.apache.spark.sql.vectorized.ColumnarMap;
-import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.sql.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java
index fca7e3685912..c7063ef127e6 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java
@@ -26,7 +26,6 @@
import org.apache.spark.sql.vectorized.ColumnarBatch;
import org.apache.spark.sql.vectorized.ColumnarMap;
import org.apache.spark.sql.vectorized.ColumnarRow;
-import org.apache.spark.unsafe.types.CalendarInterval;
import org.apache.spark.unsafe.types.UTF8String;
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 88fe7a3f380a..2b1108badb7e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -59,7 +59,6 @@ import org.apache.spark.sql.types._
import org.apache.spark.sql.util.SchemaUtils
import org.apache.spark.storage.StorageLevel
import org.apache.spark.unsafe.array.ByteArrayMethods
-import org.apache.spark.unsafe.types.CalendarInterval
import org.apache.spark.util.Utils
private[sql] object Dataset {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
index 6d1131e6939d..09f51dd25cfe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala
@@ -23,8 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection}
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
-import org.apache.spark.sql.types.MetadataBuilder
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.{CalendarInterval, MetadataBuilder}
import org.apache.spark.util.AccumulatorV2
/** Class for collecting event time stats with an accumulator */
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala
index dda9d41f630e..03fd3d197437 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout}
import org.apache.spark.sql.execution.streaming.GroupStateImpl._
import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout}
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.CalendarInterval
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala
index 2d4c3c10e644..c82aee1b779f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala
@@ -30,7 +30,6 @@ import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.streaming.WatermarkSupport.watermarkExpression
import org.apache.spark.sql.execution.streaming.state.{StateStoreCoordinatorRef, StateStoreProvider, StateStoreProviderId}
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
index 2bdb3402c14b..26a6c148d798 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala
@@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
import scala.concurrent.duration.Duration
import org.apache.spark.sql.streaming.Trigger
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.CalendarInterval
private object Triggers {
def validate(intervalMs: Long): Unit = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 84e0eaff2d42..f30136052fa8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -3128,14 +3128,14 @@ object functions {
* @param timeColumn The column or the expression to use as the timestamp for windowing by time.
* The time column must be of TimestampType.
* @param windowDuration A string specifying the width of the window, e.g. `10 minutes`,
- * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
+ * `1 second`. Check `org.apache.spark.sql.types.CalendarInterval` for
* valid duration identifiers. Note that the duration is a fixed length of
* time, and does not vary over time according to a calendar. For example,
* `1 day` always means 86,400,000 milliseconds, not a calendar day.
* @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`.
* A new window will be generated every `slideDuration`. Must be less than
* or equal to the `windowDuration`. Check
- * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration
+ * `org.apache.spark.sql.types.CalendarInterval` for valid duration
* identifiers. This duration is likewise absolute, and does not vary
* according to a calendar.
* @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start
@@ -3184,14 +3184,14 @@ object functions {
* @param timeColumn The column or the expression to use as the timestamp for windowing by time.
* The time column must be of TimestampType.
* @param windowDuration A string specifying the width of the window, e.g. `10 minutes`,
- * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
+ * `1 second`. Check `org.apache.spark.sql.types.CalendarInterval` for
* valid duration identifiers. Note that the duration is a fixed length of
* time, and does not vary over time according to a calendar. For example,
* `1 day` always means 86,400,000 milliseconds, not a calendar day.
* @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`.
* A new window will be generated every `slideDuration`. Must be less than
* or equal to the `windowDuration`. Check
- * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration
+ * `org.apache.spark.sql.types.CalendarInterval` for valid duration
* identifiers. This duration is likewise absolute, and does not vary
* according to a calendar.
*
@@ -3229,7 +3229,7 @@ object functions {
* @param timeColumn The column or the expression to use as the timestamp for windowing by time.
* The time column must be of TimestampType.
* @param windowDuration A string specifying the width of the window, e.g. `10 minutes`,
- * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for
+ * `1 second`. Check `org.apache.spark.sql.types.CalendarInterval` for
* valid duration identifiers.
*
* @group datetime_funcs
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
index 2fef05f97e57..831236b3e0f0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
@@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.CalendarInterval
+
class DateFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 80c1e24bfa56..c40d11167d74 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -1470,8 +1470,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession {
}
test("SPARK-8753: add interval type") {
- import org.apache.spark.unsafe.types.CalendarInterval
-
val df = sql("select interval 3 years -3 month 7 week 123 microseconds")
checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * 1000 * 1000 * 3600 * 24 * 7 + 123 )))
withTempPath(f => {
@@ -1495,8 +1493,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession {
}
test("SPARK-8945: add and subtract expressions for interval type") {
- import org.apache.spark.unsafe.types.CalendarInterval
- import org.apache.spark.unsafe.types.CalendarInterval.MICROS_PER_WEEK
+ import org.apache.spark.sql.types.CalendarInterval.MICROS_PER_WEEK
val df = sql("select interval 3 years -3 month 7 week 123 microseconds as i")
checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * MICROS_PER_WEEK + 123)))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala
index 758780c80b28..2cda197f8558 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.types._
import org.apache.spark.sql.util.ArrowUtils
import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch}
import org.apache.spark.unsafe.Platform
-import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+import org.apache.spark.unsafe.types.UTF8String
class ColumnarBatchSuite extends SparkFunSuite {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala
index ae701f266bf4..1c6b0f525f95 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.sql.types.CalendarInterval
class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton {
protected def checkSQL(e: Expression, expectedSQL: String): Unit = {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 1638f6cd9180..61b06d894a93 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -42,7 +42,6 @@ import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.CalendarInterval
case class Nested1(f1: Nested2)
case class Nested2(f2: Nested3)