From 19002475f2860fe8b6af696f4a8c8f7867ea086c Mon Sep 17 00:00:00 2001 From: vinodkc Date: Sat, 22 Nov 2025 16:57:54 -0800 Subject: [PATCH] Add ORC serialization and deserialization support for TIME type --- .../datasources/orc/OrcDeserializer.scala | 4 +- .../datasources/orc/OrcFileFormat.scala | 1 - .../datasources/orc/OrcSerializer.scala | 2 +- .../execution/datasources/orc/OrcUtils.scala | 6 ++- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- .../datasources/orc/OrcQuerySuite.scala | 48 +++++++++++++++++++ 6 files changed, 57 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala index d5942f8ed7d2..861e271acd3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala @@ -126,8 +126,8 @@ class OrcDeserializer( case IntegerType | _: YearMonthIntervalType => (ordinal, value) => updater.setInt(ordinal, value.asInstanceOf[IntWritable].get) - case LongType | _: DayTimeIntervalType | _: TimestampNTZType => (ordinal, value) => - updater.setLong(ordinal, value.asInstanceOf[LongWritable].get) + case LongType | _: DayTimeIntervalType | _: TimestampNTZType | _: TimeType => + (ordinal, value) => updater.setLong(ordinal, value.asInstanceOf[LongWritable].get) case FloatType => (ordinal, value) => updater.setFloat(ordinal, value.asInstanceOf[FloatWritable].get) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 40d10d0e4403..34b1af6ca1e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -249,7 +249,6 @@ class OrcFileFormat override def supportDataType(dataType: DataType): Boolean = dataType match { case _: VariantType => false - case _: TimeType => false case _: AtomicType => true case st: StructType => st.forall { f => supportDataType(f.dataType) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala index 8b74405b74f7..bff49e844057 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala @@ -98,7 +98,7 @@ class OrcSerializer(dataSchema: StructType) { } - case LongType | _: DayTimeIntervalType | _: TimestampNTZType => + case LongType | _: DayTimeIntervalType | _: TimestampNTZType | _: TimeType => if (reuseObj) { val result = new LongWritable() (getter, ordinal) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 20c40f65b068..aaf5d6f3b79e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -282,7 +282,7 @@ object OrcUtils extends Logging { s"array<${getOrcSchemaString(a.elementType)}>" case m: MapType => s"map<${getOrcSchemaString(m.keyType)},${getOrcSchemaString(m.valueType)}>" - case _: DayTimeIntervalType | _: TimestampNTZType => LongType.catalogString + case _: DayTimeIntervalType | _: TimestampNTZType | _: TimeType => LongType.catalogString case _: YearMonthIntervalType => IntegerType.catalogString case _ => dt.catalogString } @@ -302,6 +302,10 @@ object OrcUtils extends Logging { val typeDesc = new TypeDescription(TypeDescription.Category.LONG) typeDesc.setAttribute(CATALYST_TYPE_ATTRIBUTE_NAME, n.typeName) Some(typeDesc) + case tm: TimeType => + val typeDesc = new TypeDescription(TypeDescription.Category.LONG) + typeDesc.setAttribute(CATALYST_TYPE_ATTRIBUTE_NAME, tm.typeName) + Some(typeDesc) case t: TimestampType => val typeDesc = new TypeDescription(TypeDescription.Category.TIMESTAMP) typeDesc.setAttribute(CATALYST_TYPE_ATTRIBUTE_NAME, t.typeName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 7eb2d89a79cd..5e0009d0fafb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -1247,7 +1247,7 @@ class FileBasedDataSourceSuite extends QueryTest } test("SPARK-51590: unsupported the TIME data types in data sources") { - val datasources = Seq("orc", "text") + val datasources = Seq("text") Seq(true, false).foreach { useV1 => val useV1List = if (useV1) { datasources.mkString(",") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 8d90a78c6e62..e24185c94d01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -899,6 +899,54 @@ abstract class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { } } } + + test("TIME type support for ORC format") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.sql(""" + SELECT + id, + TIME'09:30:00' as morning, + TIME'14:45:30.123456' as afternoon, + TIME'23:59:59.999999' as end_of_day, + TIME'00:00:00' as midnight, + CASE WHEN id % 2 = 0 THEN TIME'12:30:00' ELSE NULL END as nullable_time + FROM VALUES (1), (2), (3) AS t(id) + """) + + df.write.mode("overwrite").orc(path) + val result = spark.read.orc(path) + + Seq("morning", "afternoon", "end_of_day", "midnight", "nullable_time").foreach { col => + assert(result.schema(col).dataType == TimeType(6)) + } + checkAnswer(result, df) + } + } + + test("TIME type with different precisions in ORC") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.sql(""" + SELECT + CAST(TIME'12:34:56' AS TIME(0)) as time_p0, + CAST(TIME'12:34:56.1' AS TIME(1)) as time_p1, + CAST(TIME'12:34:56.12' AS TIME(2)) as time_p2, + CAST(TIME'12:34:56.123' AS TIME(3)) as time_p3, + CAST(TIME'12:34:56.1234' AS TIME(4)) as time_p4, + CAST(TIME'12:34:56.12345' AS TIME(5)) as time_p5, + CAST(TIME'12:34:56.123456' AS TIME(6)) as time_p6 + """) + + df.write.mode("overwrite").orc(path) + val result = spark.read.orc(path) + + (0 to 6).foreach { p => + assert(result.schema(s"time_p$p").dataType == TimeType(p)) + } + checkAnswer(result, df) + } + } } class OrcV1QuerySuite extends OrcQuerySuite {