From 4e03050983eadfe5c3d37458be70fe40a340c5e8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 17:01:39 +0500 Subject: [PATCH 01/18] Add TimeType --- .../org/apache/spark/sql/types/TimeType.scala | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala new file mode 100644 index 000000000000..bfbefa769424 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.Stable + +/** + * The time type represents local time in microsecond precision. + * Valid range is [00:00:00.000000, 23:59:59.999999]. + * + * Please use the singleton `DataTypes.TimeType` to refer the type. + * @since 3.0.0 + */ +@Stable +class TimeType private () extends AtomicType { + + /** + * Internally, time is stored as the number of microseconds since 00:00:00.000000. + */ + private[sql] type InternalType = Long + + @transient private[sql] lazy val tag = typeTag[InternalType] + + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the TimeType is 8 bytes. + */ + override def defaultSize: Int = 8 + + private[spark] override def asNullable: TimeType = this +} + +/** + * The companion case object and its class is separated so the companion object also subclasses + * the TimeType class. Otherwise, the companion object would be of type "TimeType$" + * in byte code. Defined with a private constructor so the companion object is the only possible + * instantiation. + * + * @since 3.0.0 + */ +@Stable +case object TimeType extends TimeType From d3b5764844e65c1123a9ea1dfd86843760fcf403 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 18:27:42 +0500 Subject: [PATCH 02/18] Add the MakeTime expression --- .../expressions/datetimeExpressions.scala | 96 ++++++++++++++++++- 1 file changed, 95 insertions(+), 1 deletion(-) 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 1ce493ece18b..74081b1dc1fa 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 @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, ZoneId} +import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, LocalTime, ZoneId} +import java.time.temporal.ChronoField.MICRO_OF_DAY import java.time.temporal.IsoFields import java.util.{Locale, TimeZone} @@ -1893,6 +1894,99 @@ case class MakeTimestamp( override def prettyName: String = "make_timestamp" } +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(hour, min, sec) - Create time from hour, min, sec fields.", + arguments = """ + Arguments: + * hour - the hour-of-day to represent, from 0 to 23 + * min - the minute-of-hour to represent, from 0 to 59 + * sec - the second-of-minute and its micro-fraction to represent, from + 0 to 60. If the sec argument equals to 60, the seconds field is set + to 0 and 1 minute is added to the final timestamp. + """, + examples = """ + Examples: + > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887); + 2014-12-28 06:30:45.887 + > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET'); + 2014-12-28 10:30:45.887 + > SELECT _FUNC_(2019, 6, 30, 23, 59, 60) + 2019-07-01 00:00:00 + > SELECT _FUNC_(2019, 13, 1, 10, 11, 12, 13); + NULL + > SELECT _FUNC_(null, 7, 22, 15, 30, 0); + NULL + """, + since = "3.0.0") +// scalastyle:on line.size.limit +case class MakeTime(hour: Expression, min: Expression, sec: Expression) + extends TernaryExpression with ImplicitCastInputTypes { + + override def children: Seq[Expression] = Seq(hour, min, sec) + // Accept `sec` as DecimalType to avoid loosing precision of microseconds while converting + // them to the fractional part of `sec`. + override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, IntegerType, DecimalType(8, 6)) + override def dataType: DataType = TimeType + override def nullable: Boolean = true + + override def nullSafeEval(hourOfDay: Any, minutes: Any, seconds: Any): Any = { + val hour = hourOfDay.asInstanceOf[Int] + val min = minutes.asInstanceOf[Int] + val secAndNanos = seconds.asInstanceOf[Decimal] + + try { + val secFloor = secAndNanos.floor + val nanosPerSec = Decimal(NANOS_PER_SECOND, 10, 0) + val nanos = ((secAndNanos - secFloor) * nanosPerSec).toInt + val seconds = secFloor.toInt + val localTime = if (seconds == 60) { + if (nanos == 0) { + // This case of sec = 60 and nanos = 0 is supported for compatibility with PostgreSQL + LocalTime.of(hour, min, 0, 0).plusMinutes(1) + } else { + throw new DateTimeException("The fraction of sec must be zero. Valid range is [0, 60].") + } + } else { + LocalTime.of(hour, min, seconds, nanos) + } + localTime.getLong(MICRO_OF_DAY) + } catch { + case _: DateTimeException => null + } + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val d = Decimal.getClass.getName.stripSuffix("$") + nullSafeCodeGen(ctx, ev, (hour, min, secAndNanos) => { + s""" + try { + org.apache.spark.sql.types.Decimal secFloor = $secAndNanos.floor(); + org.apache.spark.sql.types.Decimal nanosPerSec = $d$$.MODULE$$.apply(1000000000L, 10, 0); + int nanos = (($secAndNanos.$$minus(secFloor)).$$times(nanosPerSec)).toInt(); + int seconds = secFloor.toInt(); + java.time.LocalTime localTime; + if (seconds == 60) { + if (nanos == 0) { + localTime = java.time.LocalTime.of($hour, $min, 0, 0).plusMinutes(1); + } else { + throw new java.time.DateTimeException( + "The fraction of sec must be zero. Valid range is [0, 60]."); + } + } else { + localTime = java.time.LocalTime.of($hour, $min, seconds, nanos); + } + localTime.getLong(java.time.temporal.ChronoField.MICRO_OF_DAY); + } catch (java.time.DateTimeException e) { + ${ev.isNull} = true; + }""" + }) + } + + override def prettyName: String = "make_time" +} + case class Millennium(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) From 50a7b41a4ba5c88bd59a573ca40b184d3fef3da6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 19:43:28 +0500 Subject: [PATCH 03/18] Add tests to DateExpressionsSuite --- .../expressions/DateExpressionsSuite.scala | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) 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..8b1b8cde2b41 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{LocalDateTime, ZoneId, ZoneOffset} +import java.time.{LocalDateTime, LocalTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -973,6 +973,28 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-08-12 00:00:58.000001")) } + test("creating values of TimeType via make_time") { + var makeTimeExpr = MakeTime(Literal(8), Literal(15), Literal(Decimal(BigDecimal(23.5), 8, 6))) + val expected = LocalTime.parse("08:15:23.5") + checkEvaluation(makeTimeExpr, expected) + + checkEvaluation(makeTimeExpr.copy(hour = Literal.create(null, IntegerType)), null) + checkEvaluation(makeTimeExpr.copy(hour = Literal(25)), null) + + checkEvaluation(makeTimeExpr.copy(min = Literal.create(null, IntegerType)), null) + checkEvaluation(makeTimeExpr.copy(min = Literal(65)), null) + + checkEvaluation(makeTimeExpr.copy(sec = Literal.create(null, DecimalType(8, 6))), null) + checkEvaluation(makeTimeExpr.copy(sec = Literal(Decimal(BigDecimal(70.0), 8, 6))), null) + + makeTimeExpr = MakeTime(Literal(0), Literal(59), Literal(Decimal(BigDecimal(60.0), 8, 6))) + checkEvaluation(makeTimeExpr, LocalTime.parse("01:00:00")) + checkEvaluation(makeTimeExpr.copy(sec = Literal(Decimal(BigDecimal(60.5), 8, 6))), null) + + makeTimeExpr = MakeTime(Literal(0), Literal(0), Literal(Decimal(BigDecimal(58.000001), 8, 6))) + checkEvaluation(makeTimeExpr, LocalTime.parse("00:00:58.000001")) + } + test("millennium") { val date = MakeDate(Literal(2019), Literal(1), Literal(1)) checkEvaluation(Millennium(date), 3) From e41f52c99a610f79bb71d8046dc4a46a95b0f59d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 20:18:11 +0500 Subject: [PATCH 04/18] Converters for LocalTime --- .../sql/catalyst/CatalystTypeConverters.scala | 16 +++++++++- .../sql/catalyst/util/DateTimeUtils.scala | 5 +++ .../CatalystTypeConvertersSuite.scala | 31 ++++++++++++++++++- 3 files changed, 50 insertions(+), 2 deletions(-) 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 488252aa0c7b..d2c7df11e4db 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 @@ -21,7 +21,7 @@ import java.lang.{Iterable => JavaIterable} import java.math.{BigDecimal => JavaBigDecimal} import java.math.{BigInteger => JavaBigInteger} import java.sql.{Date, Timestamp} -import java.time.{Instant, LocalDate} +import java.time.{Instant, LocalDate, LocalTime} import java.util.{Map => JavaMap} import javax.annotation.Nullable @@ -66,6 +66,7 @@ object CatalystTypeConverters { case DateType => DateConverter case TimestampType if SQLConf.get.datetimeJava8ApiEnabled => InstantConverter case TimestampType => TimestampConverter + case TimeType => LocalTimeConverter case dt: DecimalType => new DecimalConverter(dt) case BooleanType => BooleanConverter case ByteType => ByteConverter @@ -341,6 +342,18 @@ object CatalystTypeConverters { DateTimeUtils.microsToInstant(row.getLong(column)) } + private object LocalTimeConverter extends CatalystTypeConverter[LocalTime, LocalTime, Any] { + override def toCatalystImpl(scalaValue: LocalTime): Long = { + DateTimeUtils.localTimeToMicros(scalaValue) + } + override def toScala(catalystValue: Any): LocalTime = { + if (catalystValue == null) null + else DateTimeUtils.microsToLocalTime(catalystValue.asInstanceOf[Long]) + } + override def toScalaImpl(row: InternalRow, column: Int): LocalTime = + DateTimeUtils.microsToLocalTime(row.getLong(column)) + } + private class DecimalConverter(dataType: DecimalType) extends CatalystTypeConverter[Any, JavaBigDecimal, Decimal] { @@ -452,6 +465,7 @@ object CatalystTypeConverters { case ld: LocalDate => LocalDateConverter.toCatalyst(ld) case t: Timestamp => TimestampConverter.toCatalyst(t) case i: Instant => InstantConverter.toCatalyst(i) + case t: LocalTime => LocalTimeConverter.toCatalyst(t) case d: BigDecimal => new DecimalConverter(DecimalType(d.precision, d.scale)).toCatalyst(d) case d: JavaBigDecimal => new DecimalConverter(DecimalType(d.precision, d.scale)).toCatalyst(d) case seq: Seq[Any] => new GenericArrayData(seq.map(convertToCatalyst).toArray) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 65a9bee5eaed..da7acc80bfc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.time._ import java.time.temporal.{ChronoField, ChronoUnit, IsoFields} +import java.time.temporal.ChronoField.MICRO_OF_DAY import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ @@ -363,6 +364,10 @@ object DateTimeUtils { def daysToLocalDate(days: Int): LocalDate = LocalDate.ofEpochDay(days) + def localTimeToMicros(localTime: LocalTime): Long = localTime.getLong(MICRO_OF_DAY) + + def microsToLocalTime(us: Long): LocalTime = LocalTime.ofNanoOfDay(us * NANOS_PER_MICROS) + /** * Trim and parse a given UTF8 date string to a corresponding [[Int]] value. * The return type is [[Option]] in order to distinguish between 0 and null. The following 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 b9e7cf304989..08b50229df95 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import java.time.{Instant, LocalDate} +import java.time.{Instant, LocalDate, LocalTime} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row @@ -216,4 +216,33 @@ class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { } } } + + test("converting java.time.LocalTime to TimeType") { + Seq( + "00:00:00", + "01:02:03.999", + "02:59:01", + "12:30:02.0", + "22:00:00.000001", + "23:59:59.999999").foreach { time => + val input = LocalTime.parse(time) + val result = CatalystTypeConverters.convertToCatalyst(input) + val expected = DateTimeUtils.localTimeToMicros(input) + assert(result === expected) + } + } + + test("converting TimeType to java.time.LocalTime") { + Seq( + 0, + 1, + 59000000, + 3600000001L, + 43200999999L, + 86399000000L, + 86399999999L).foreach { us => + val localTime = DateTimeUtils.microsToLocalTime(us) + assert(CatalystTypeConverters.createToScalaConverter(TimeType)(us) === localTime) + } + } } From baa632b9a8073a56cf22866203cd01d82109b1df Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 20:20:20 +0500 Subject: [PATCH 05/18] Reuse localTimeToMicros in MakeTime --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 74081b1dc1fa..202b87b21ae0 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 @@ -1950,7 +1950,7 @@ case class MakeTime(hour: Expression, min: Expression, sec: Expression) } else { LocalTime.of(hour, min, seconds, nanos) } - localTime.getLong(MICRO_OF_DAY) + localTimeToMicros(localTime) } catch { case _: DateTimeException => null } @@ -1977,7 +1977,7 @@ case class MakeTime(hour: Expression, min: Expression, sec: Expression) } else { localTime = java.time.LocalTime.of($hour, $min, seconds, nanos); } - localTime.getLong(java.time.temporal.ChronoField.MICRO_OF_DAY); + ${ev.value} = $dtu.localTimeToMicros(localTime); } catch (java.time.DateTimeException e) { ${ev.isNull} = true; }""" From cee85e12eb4dca3634c6bd4f00f974edb3b84ecf Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 22:40:10 +0500 Subject: [PATCH 06/18] Support Row encoding --- .../spark/sql/catalyst/DeserializerBuildHelper.scala | 9 +++++++++ .../apache/spark/sql/catalyst/JavaTypeInference.scala | 6 ++++++ .../apache/spark/sql/catalyst/ScalaReflection.scala | 11 ++++++++++- .../spark/sql/catalyst/SerializerBuildHelper.scala | 9 +++++++++ .../spark/sql/catalyst/encoders/RowEncoder.scala | 6 ++++++ .../catalyst/expressions/codegen/CodeGenerator.scala | 2 +- .../spark/sql/catalyst/encoders/RowEncoderSuite.scala | 10 ++++++++++ 7 files changed, 51 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index e55c25c4b0c5..8d67fe0c23eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -118,6 +118,15 @@ object DeserializerBuildHelper { returnNullable = false) } + def createDeserializerForLocalTime(path: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + ObjectType(classOf[java.time.LocalTime]), + "microsToLocalTime", + path :: Nil, + returnNullable = false) + } + def createDeserializerForJavaBigDecimal( path: Expression, returnNullable: Boolean): Expression = { 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 c5be3efc6371..060b63d37b18 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 @@ -106,6 +106,7 @@ object JavaTypeInference { case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) 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.LocalTime] => (TimeType, true) case _ if typeToken.isArray => val (dataType, nullable) = inferDataType(typeToken.getComponentType, seenTypeSet) @@ -235,6 +236,9 @@ object JavaTypeInference { case c if c == classOf[java.sql.Timestamp] => createDeserializerForSqlTimestamp(path) + case c if c == classOf[java.time.LocalTime] => + createDeserializerForLocalTime(path) + case c if c == classOf[java.lang.String] => createDeserializerForString(path, returnNullable = true) @@ -390,6 +394,8 @@ object JavaTypeInference { case c if c == classOf[java.sql.Date] => createSerializerForSqlDate(inputObject) + case c if c == classOf[java.time.LocalTime] => createSerializerForJavaLocalTime(inputObject) + case c if c == classOf[java.math.BigDecimal] => createSerializerForJavaBigDecimal(inputObject) 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 2a4e84241a94..9a9354475035 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 @@ -230,6 +230,9 @@ object ScalaReflection extends ScalaReflection { case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => createDeserializerForSqlTimestamp(path) + case t if isSubtype(t, localTypeOf[java.time.LocalTime]) => + createDeserializerForLocalTime(path) + case t if isSubtype(t, localTypeOf[java.lang.String]) => createDeserializerForString(path, returnNullable = false) @@ -496,6 +499,9 @@ object ScalaReflection extends ScalaReflection { case t if isSubtype(t, localTypeOf[java.sql.Date]) => createSerializerForSqlDate(inputObject) + case t if isSubtype(t, localTypeOf[java.time.LocalTime]) => + createSerializerForJavaLocalTime(inputObject) + case t if isSubtype(t, localTypeOf[BigDecimal]) => createSerializerForScalaBigDecimal(inputObject) @@ -671,6 +677,7 @@ 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[java.time.LocalTime]) => Schema(TimeType, nullable = true) case t if isSubtype(t, localTypeOf[BigDecimal]) => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => @@ -771,6 +778,7 @@ object ScalaReflection extends ScalaReflection { StringType -> classOf[UTF8String], DateType -> classOf[DateType.InternalType], TimestampType -> classOf[TimestampType.InternalType], + TimeType -> classOf[TimeType.InternalType], BinaryType -> classOf[BinaryType.InternalType], CalendarIntervalType -> classOf[CalendarInterval] ) @@ -784,7 +792,8 @@ object ScalaReflection extends ScalaReflection { FloatType -> classOf[java.lang.Float], DoubleType -> classOf[java.lang.Double], DateType -> classOf[java.lang.Integer], - TimestampType -> classOf[java.lang.Long] + TimestampType -> classOf[java.lang.Long], + TimeType -> classOf[java.lang.Long] ) def dataTypeJavaClass(dt: DataType): Class[_] = { 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 75c278e78114..870dd9efb63e 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 @@ -101,6 +101,15 @@ object SerializerBuildHelper { returnNullable = false) } + def createSerializerForJavaLocalTime(inputObject: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + TimeType, + "localTimeToMicros", + inputObject :: Nil, + returnNullable = false) + } + def createSerializerForJavaBigDecimal(inputObject: Expression): Expression = { CheckOverflow(StaticInvoke( Decimal.getClass, 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 afe8a23f8f15..9ebcb4820134 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 @@ -53,6 +53,7 @@ import org.apache.spark.sql.types._ * TimestampType -> java.sql.Timestamp if spark.sql.datetime.java8API.enabled is false * TimestampType -> java.time.Instant if spark.sql.datetime.java8API.enabled is true * + * TimeType -> java.time.LocalTime * BinaryType -> byte array * ArrayType -> scala.collection.Seq or Array * MapType -> scala.collection.Map @@ -108,6 +109,8 @@ object RowEncoder { createSerializerForSqlDate(inputObject) } + case TimeType => createSerializerForJavaLocalTime(inputObject) + case d: DecimalType => CheckOverflow(StaticInvoke( Decimal.getClass, @@ -226,6 +229,7 @@ object RowEncoder { } else { ObjectType(classOf[java.sql.Date]) } + case TimeType => ObjectType(classOf[java.time.LocalTime]) case _: DecimalType => ObjectType(classOf[java.math.BigDecimal]) case StringType => ObjectType(classOf[java.lang.String]) case _: ArrayType => ObjectType(classOf[scala.collection.Seq[_]]) @@ -281,6 +285,8 @@ object RowEncoder { createDeserializerForSqlDate(input) } + case TimeType => createDeserializerForLocalTime(input) + case _: DecimalType => createDeserializerForJavaBigDecimal(input, returnNullable = false) case StringType => createDeserializerForString(input, returnNullable = false) 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 95fad412002e..a7ad0da15745 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 @@ -1630,7 +1630,7 @@ object CodeGenerator extends Logging { case ByteType => JAVA_BYTE case ShortType => JAVA_SHORT case IntegerType | DateType => JAVA_INT - case LongType | TimestampType => JAVA_LONG + case LongType | TimestampType | TimeType => JAVA_LONG case FloatType => JAVA_FLOAT case DoubleType => JAVA_DOUBLE case _: DecimalType => "Decimal" 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 5d21e4a2a83c..577594c5b34c 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 @@ -332,6 +332,16 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } + test("encoding/decoding TimeType to/from java.time.LocalTime") { + val schema = new StructType().add("t", TimeType) + val encoder = RowEncoder(schema).resolveAndBind() + val localTime = java.time.LocalTime.parse("20:38:45.123456") + val row = encoder.toRow(Row(localTime)) + assert(row.getLong(0) === DateTimeUtils.localTimeToMicros(localTime)) + val readback = encoder.fromRow(row) + assert(readback.get(0).equals(localTime)) + } + for { elementType <- Seq(IntegerType, StringType) containsNull <- Seq(true, false) From 51d48d5d67cc00c092f640301f2e476c1bf55016 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 23:00:37 +0500 Subject: [PATCH 07/18] Support TimeType as Literal --- .../sql/catalyst/expressions/SpecializedGettersReader.java | 3 +++ .../sql/catalyst/expressions/InterpretedUnsafeProjection.scala | 2 +- .../org/apache/spark/sql/catalyst/expressions/literals.scala | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java index ea0648a6cb90..7576eee98fe1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java @@ -83,6 +83,9 @@ public static Object read( if (handleUserDefinedType && dataType instanceof UserDefinedType) { return obj.get(ordinal, ((UserDefinedType)dataType).sqlType()); } + if (dataType instanceof TimeType) { + return obj.getLong(ordinal); + } throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString()); } 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 55a5bd380859..8fca20b9a2d5 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 @@ -146,7 +146,7 @@ object InterpretedUnsafeProjection { case IntegerType | DateType => (v, i) => writer.write(i, v.getInt(i)) - case LongType | TimestampType => + case LongType | TimestampType | TimeType => (v, i) => writer.write(i, v.getLong(i)) case FloatType => 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..b91301caed23 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 @@ -179,7 +179,7 @@ object Literal { case ByteType => v.isInstanceOf[Byte] case ShortType => v.isInstanceOf[Short] case IntegerType | DateType => v.isInstanceOf[Int] - case LongType | TimestampType => v.isInstanceOf[Long] + case LongType | TimestampType | TimeType => v.isInstanceOf[Long] case FloatType => v.isInstanceOf[Float] case DoubleType => v.isInstanceOf[Double] case _: DecimalType => v.isInstanceOf[Decimal] From 61329e00e0dd0bba61b4422b780d49f09a9fb871 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 3 Sep 2019 23:18:25 +0500 Subject: [PATCH 08/18] Register make_time --- .../apache/spark/sql/catalyst/analysis/FunctionRegistry.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 5177f1e55829..609883d1251e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -417,6 +417,7 @@ object FunctionRegistry { expression[TimeWindow]("window"), expression[MakeDate]("make_date"), expression[MakeTimestamp]("make_timestamp"), + expression[MakeTime]("make_time"), // collection functions expression[CreateArray]("array"), From a2a5d9dc99e388031d3032324afb833c6bbcc944 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 14:57:49 +0500 Subject: [PATCH 09/18] Implicit encoder for LocalTime --- .../src/main/scala/org/apache/spark/sql/Encoders.scala | 8 ++++++++ .../main/scala/org/apache/spark/sql/SQLImplicits.scala | 3 +++ .../java/test/org/apache/spark/sql/JavaDatasetSuite.java | 9 +++++++++ .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 5 +++++ 4 files changed, 25 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 5d31b5bbf12a..68327bb3c705 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -128,6 +128,14 @@ object Encoders { */ def INSTANT: Encoder[java.time.Instant] = ExpressionEncoder() + /** + * Creates an encoder that serializes instances of the `java.time.LocalTime` class + * to the internal representation of nullable Catalyst's DateType. + * + * @since 3.0.0 + */ + def LOCALTIME: Encoder[java.time.LocalTime] = ExpressionEncoder() + /** * An encoder for arrays of bytes. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 71cbc3ab14d9..895a782923d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -88,6 +88,9 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits { /** @since 3.0.0 */ implicit def newInstantEncoder: Encoder[java.time.Instant] = Encoders.INSTANT + /** @since 3.0.0 */ + implicit def newLocalTimeEncoder: Encoder[java.time.LocalTime] = Encoders.LOCALTIME + // Boxed primitives /** @since 2.0.0 */ diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 1e5f55e494b7..976059411d6c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -23,6 +23,7 @@ import java.sql.Timestamp; import java.time.Instant; import java.time.LocalDate; +import java.time.LocalTime; import java.util.*; import org.apache.spark.sql.streaming.GroupStateTimeout; @@ -411,6 +412,14 @@ public void testLocalDateAndInstantEncoders() { Assert.assertEquals(data, ds.collectAsList()); } + @Test + public void testLocalTimeEncoder() { + Encoder encoder = Encoders.LOCALTIME(); + List data = Arrays.asList(LocalTime.ofNanoOfDay(0)); + Dataset ds = spark.createDataset(data, encoder); + Assert.assertEquals(data, ds.collectAsList()); + } + public static class KryoSerializable { String value; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 42e5ee58954e..ced9c46ab1de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1841,6 +1841,11 @@ class DatasetSuite extends QueryTest with SharedSparkSession { val instant = java.time.Instant.parse("2019-03-30T09:54:00Z") assert(spark.range(1).map { _ => instant }.head === instant) } + + test("implicit encoder for LocalTime") { + val localTime = java.time.LocalTime.of(19, 30, 30) + assert(spark.range(1).map { _ => localTime }.head === localTime) + } } object AssertExecutionId { From b67f27012e12843723738556a2df6e7830361a7d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 16:24:40 +0500 Subject: [PATCH 10/18] Construct literals from java.time.LocalTime --- .../spark/sql/catalyst/InternalRow.scala | 5 ++-- .../spark/sql/catalyst/dsl/package.scala | 3 +++ .../expressions/SpecificInternalRow.scala | 4 ++-- .../expressions/codegen/CodeGenerator.scala | 2 +- .../sql/catalyst/expressions/literals.scala | 9 +++++--- .../org/apache/spark/sql/types/DataType.scala | 2 +- .../expressions/LiteralExpressionSuite.scala | 23 ++++++++++++++++++- 7 files changed, 38 insertions(+), 10 deletions(-) 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 bdab407688a6..1da87dc7a2af 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 @@ -131,7 +131,7 @@ object InternalRow { case ByteType => (input, ordinal) => input.getByte(ordinal) case ShortType => (input, ordinal) => input.getShort(ordinal) case IntegerType | DateType => (input, ordinal) => input.getInt(ordinal) - case LongType | TimestampType => (input, ordinal) => input.getLong(ordinal) + case LongType | TimestampType | TimeType => (input, ordinal) => input.getLong(ordinal) case FloatType => (input, ordinal) => input.getFloat(ordinal) case DoubleType => (input, ordinal) => input.getDouble(ordinal) case StringType => (input, ordinal) => input.getUTF8String(ordinal) @@ -166,7 +166,8 @@ object InternalRow { case ByteType => (input, v) => input.setByte(ordinal, v.asInstanceOf[Byte]) case ShortType => (input, v) => input.setShort(ordinal, v.asInstanceOf[Short]) case IntegerType | DateType => (input, v) => input.setInt(ordinal, v.asInstanceOf[Int]) - case LongType | TimestampType => (input, v) => input.setLong(ordinal, v.asInstanceOf[Long]) + case LongType | TimestampType | TimeType => + (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]) case DecimalType.Fixed(precision, _) => 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 796043fff665..f0c1dbe634e0 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 @@ -252,6 +252,9 @@ package object dsl { /** Creates a new AttributeReference of type timestamp */ def timestamp: AttributeReference = AttributeReference(s, TimestampType, nullable = true)() + /** Creates a new AttributeReference of type date */ + def time: AttributeReference = AttributeReference(s, TimeType, nullable = true)() + /** Creates a new AttributeReference of type binary */ def binary: AttributeReference = AttributeReference(s, BinaryType, nullable = true)() 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 75feaf670c84..7b7d62d5a975 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 @@ -200,8 +200,8 @@ final class SpecificInternalRow(val values: Array[MutableValue]) extends BaseGen case ShortType => new MutableShort // We use INT for DATE internally case IntegerType | DateType => new MutableInt - // We use Long for Timestamp internally - case LongType | TimestampType => new MutableLong + // We use Long for Timestamp and Time internally + case LongType | TimestampType | TimeType => new MutableLong case FloatType => new MutableFloat case DoubleType => new MutableDouble case _ => new MutableAny 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 a7ad0da15745..aad3de010600 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 @@ -1651,7 +1651,7 @@ object CodeGenerator extends Logging { case ByteType => java.lang.Byte.TYPE case ShortType => java.lang.Short.TYPE case IntegerType | DateType => java.lang.Integer.TYPE - case LongType | TimestampType => java.lang.Long.TYPE + case LongType | TimestampType | TimeType => java.lang.Long.TYPE case FloatType => java.lang.Float.TYPE case DoubleType => java.lang.Double.TYPE case _: DecimalType => classOf[Decimal] 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 b91301caed23..5c72a50ddca4 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 @@ -27,7 +27,7 @@ import java.lang.{Short => JavaShort} import java.math.{BigDecimal => JavaBigDecimal} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} -import java.time.{Instant, LocalDate} +import java.time.{Instant, LocalDate, LocalTime} import java.util import java.util.Objects import javax.xml.bind.DatatypeConverter @@ -42,7 +42,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localTimeToMicros} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ @@ -71,6 +71,7 @@ object Literal { case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), TimestampType) case ld: LocalDate => Literal(ld.toEpochDay.toInt, DateType) case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType) + case lt: LocalTime => Literal(localTimeToMicros(lt), TimeType) case a: Array[Byte] => Literal(a, BinaryType) case a: collection.mutable.WrappedArray[_] => apply(a.array) case a: Array[_] => @@ -105,6 +106,7 @@ object Literal { case _ if clz == classOf[Date] => DateType case _ if clz == classOf[Instant] => TimestampType case _ if clz == classOf[Timestamp] => TimestampType + case _ if clz == classOf[LocalTime] => TimeType case _ if clz == classOf[JavaBigDecimal] => DecimalType.SYSTEM_DEFAULT case _ if clz == classOf[Array[Byte]] => BinaryType case _ if clz == classOf[JavaShort] => ShortType @@ -160,6 +162,7 @@ object Literal { case dt: DecimalType => Literal(Decimal(0, dt.precision, dt.scale)) case DateType => create(0, DateType) case TimestampType => create(0L, TimestampType) + case TimeType => create(0L, TimeType) case StringType => Literal("") case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) case CalendarIntervalType => Literal(new CalendarInterval(0, 0)) @@ -336,7 +339,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { } case ByteType | ShortType => ExprCode.forNonNullValue(JavaCode.expression(s"($javaType)$value", dataType)) - case TimestampType | LongType => + case TimestampType | LongType | TimeType => toExprCode(s"${value}L") case _ => val constRef = ctx.addReferenceObj("literal", value, javaType) 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 3a10a56f6937..8ae8069e4a7a 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 @@ -130,7 +130,7 @@ object DataType { private val nonDecimalNameToType = { Seq(NullType, DateType, TimestampType, BinaryType, IntegerType, BooleanType, LongType, - DoubleType, FloatType, ShortType, ByteType, StringType, CalendarIntervalType) + DoubleType, FloatType, ShortType, ByteType, StringType, CalendarIntervalType, TimeType) .map(t => t.typeName -> t).toMap } 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..51b0802631ca 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.nio.charset.StandardCharsets -import java.time.{Instant, LocalDate, LocalDateTime, ZoneOffset} +import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneOffset} import java.util.TimeZone import scala.reflect.runtime.universe.TypeTag @@ -81,6 +81,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.default(StructType(StructField("a", StringType) :: Nil)), Row("")) // ExamplePointUDT.sqlType is ArrayType(DoubleType, false). checkEvaluation(Literal.default(new ExamplePointUDT), Array()) + checkEvaluation(Literal.default(TimeType), LocalTime.ofNanoOfDay(0)) } test("boolean literals") { @@ -316,4 +317,24 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { assert(literalStr === expected) } } + + test("construct literals from java.time.LocalTime") { + Seq( + LocalTime.of(0, 0, 0), + LocalTime.of(0, 0, 0, 1000), + LocalTime.of(1, 1, 1), + LocalTime.of(23, 59, 59), + LocalTime.of(23, 59, 59, 999999000), + LocalTime.of(10, 11, 12, 131415000), + LocalTime.of(21, 0, 0)).foreach { localTime => + checkEvaluation(Literal(localTime), localTime) + } + } + + test("construct literals from arrays of java.time.LocalTime") { + val localTime0 = LocalTime.of(10, 10, 10) + checkEvaluation(Literal(Array(localTime0)), Array(localTime0)) + val localTime1 = LocalTime.of(21, 21, 21) + checkEvaluation(Literal(Array(localTime0, localTime1)), Array(localTime0, localTime1)) + } } From 1d68f5829ce32439fdbcd15896464b91074aa277 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 17:34:36 +0500 Subject: [PATCH 11/18] Test make_time --- .../spark/sql/execution/HiveResult.scala | 1 + .../resources/sql-tests/inputs/pgSQL/date.sql | 6 +-- .../sql-tests/results/pgSQL/date.sql.out | 44 ++++++++++++++----- 3 files changed, 38 insertions(+), 13 deletions(-) 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 eec8d70b5adf..6a693fdd1d16 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 @@ -75,6 +75,7 @@ object HiveResult { ShortType, DateType, TimestampType, + TimeType, BinaryType) private lazy val dateFormatter = DateFormatter() diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index b9a6b998e52f..d1c37b29ed91 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -350,12 +350,12 @@ SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-0 select make_date(2013, 7, 15); -- [SPARK-28471] Formatting dates with negative years select make_date(-44, 3, 15); --- select make_time(8, 20, 0.0); +select make_time(8, 20, 0.0); -- should fail select make_date(2013, 2, 30); select make_date(2013, 13, 1); select make_date(2013, 11, -1); --- select make_time(10, 55, 100.1); --- select make_time(24, 0, 2.1); +select make_time(10, 55, 100.1); +select make_time(24, 0, 2.1); DROP TABLE DATE_TBL; diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index cb2be6d1cd22..e7207d2a3e2a 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 91 +-- Number of queries: 94 -- !query 0 @@ -822,32 +822,56 @@ struct -- !query 87 -select make_date(2013, 2, 30) +select make_time(8, 20, 0.0) -- !query 87 schema -struct +struct -- !query 87 output -NULL +08:20 -- !query 88 -select make_date(2013, 13, 1) +select make_date(2013, 2, 30) -- !query 88 schema -struct +struct -- !query 88 output NULL -- !query 89 -select make_date(2013, 11, -1) +select make_date(2013, 13, 1) -- !query 89 schema -struct +struct -- !query 89 output NULL -- !query 90 -DROP TABLE DATE_TBL +select make_date(2013, 11, -1) -- !query 90 schema -struct<> +struct -- !query 90 output +NULL + + +-- !query 91 +select make_time(10, 55, 100.1) +-- !query 91 schema +struct +-- !query 91 output +NULL + + +-- !query 92 +select make_time(24, 0, 2.1) +-- !query 92 schema +struct +-- !query 92 output +NULL + + +-- !query 93 +DROP TABLE DATE_TBL +-- !query 93 schema +struct<> +-- !query 93 output From ef53768911f6787ee5cb4152625fd68cd03fcd0f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 17:35:02 +0500 Subject: [PATCH 12/18] Revert "Test make_time" This reverts commit 1d68f5829ce32439fdbcd15896464b91074aa277. --- .../spark/sql/execution/HiveResult.scala | 1 - .../resources/sql-tests/inputs/pgSQL/date.sql | 6 +-- .../sql-tests/results/pgSQL/date.sql.out | 44 +++++-------------- 3 files changed, 13 insertions(+), 38 deletions(-) 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 6a693fdd1d16..eec8d70b5adf 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 @@ -75,7 +75,6 @@ object HiveResult { ShortType, DateType, TimestampType, - TimeType, BinaryType) private lazy val dateFormatter = DateFormatter() diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index d1c37b29ed91..b9a6b998e52f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -350,12 +350,12 @@ SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-0 select make_date(2013, 7, 15); -- [SPARK-28471] Formatting dates with negative years select make_date(-44, 3, 15); -select make_time(8, 20, 0.0); +-- select make_time(8, 20, 0.0); -- should fail select make_date(2013, 2, 30); select make_date(2013, 13, 1); select make_date(2013, 11, -1); -select make_time(10, 55, 100.1); -select make_time(24, 0, 2.1); +-- select make_time(10, 55, 100.1); +-- select make_time(24, 0, 2.1); DROP TABLE DATE_TBL; diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index e7207d2a3e2a..cb2be6d1cd22 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 94 +-- Number of queries: 91 -- !query 0 @@ -822,56 +822,32 @@ struct -- !query 87 -select make_time(8, 20, 0.0) +select make_date(2013, 2, 30) -- !query 87 schema -struct +struct -- !query 87 output -08:20 +NULL -- !query 88 -select make_date(2013, 2, 30) +select make_date(2013, 13, 1) -- !query 88 schema -struct +struct -- !query 88 output NULL -- !query 89 -select make_date(2013, 13, 1) +select make_date(2013, 11, -1) -- !query 89 schema -struct +struct -- !query 89 output NULL -- !query 90 -select make_date(2013, 11, -1) --- !query 90 schema -struct --- !query 90 output -NULL - - --- !query 91 -select make_time(10, 55, 100.1) --- !query 91 schema -struct --- !query 91 output -NULL - - --- !query 92 -select make_time(24, 0, 2.1) --- !query 92 schema -struct --- !query 92 output -NULL - - --- !query 93 DROP TABLE DATE_TBL --- !query 93 schema +-- !query 90 schema struct<> --- !query 93 output +-- !query 90 output From a1ba921c2824ca6f27e95e8d484a9151b815c81c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 17:35:21 +0500 Subject: [PATCH 13/18] Revert "Register make_time" This reverts commit 61329e00e0dd0bba61b4422b780d49f09a9fb871. --- .../apache/spark/sql/catalyst/analysis/FunctionRegistry.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 609883d1251e..5177f1e55829 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -417,7 +417,6 @@ object FunctionRegistry { expression[TimeWindow]("window"), expression[MakeDate]("make_date"), expression[MakeTimestamp]("make_timestamp"), - expression[MakeTime]("make_time"), // collection functions expression[CreateArray]("array"), From 33244442390fa77a9ced4a85fe7e4fb425abb29c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 17:35:49 +0500 Subject: [PATCH 14/18] Revert "Reuse localTimeToMicros in MakeTime" This reverts commit baa632b9a8073a56cf22866203cd01d82109b1df. --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 202b87b21ae0..74081b1dc1fa 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 @@ -1950,7 +1950,7 @@ case class MakeTime(hour: Expression, min: Expression, sec: Expression) } else { LocalTime.of(hour, min, seconds, nanos) } - localTimeToMicros(localTime) + localTime.getLong(MICRO_OF_DAY) } catch { case _: DateTimeException => null } @@ -1977,7 +1977,7 @@ case class MakeTime(hour: Expression, min: Expression, sec: Expression) } else { localTime = java.time.LocalTime.of($hour, $min, seconds, nanos); } - ${ev.value} = $dtu.localTimeToMicros(localTime); + localTime.getLong(java.time.temporal.ChronoField.MICRO_OF_DAY); } catch (java.time.DateTimeException e) { ${ev.isNull} = true; }""" From 0202151b94163875835588ad9b314c7a8f922d92 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 17:36:07 +0500 Subject: [PATCH 15/18] Revert "Add tests to DateExpressionsSuite" This reverts commit 50a7b41a4ba5c88bd59a573ca40b184d3fef3da6. --- .../expressions/DateExpressionsSuite.scala | 24 +------------------ 1 file changed, 1 insertion(+), 23 deletions(-) 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 8b1b8cde2b41..ae3549b4aaf5 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{LocalDateTime, LocalTime, ZoneId, ZoneOffset} +import java.time.{LocalDateTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -973,28 +973,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-08-12 00:00:58.000001")) } - test("creating values of TimeType via make_time") { - var makeTimeExpr = MakeTime(Literal(8), Literal(15), Literal(Decimal(BigDecimal(23.5), 8, 6))) - val expected = LocalTime.parse("08:15:23.5") - checkEvaluation(makeTimeExpr, expected) - - checkEvaluation(makeTimeExpr.copy(hour = Literal.create(null, IntegerType)), null) - checkEvaluation(makeTimeExpr.copy(hour = Literal(25)), null) - - checkEvaluation(makeTimeExpr.copy(min = Literal.create(null, IntegerType)), null) - checkEvaluation(makeTimeExpr.copy(min = Literal(65)), null) - - checkEvaluation(makeTimeExpr.copy(sec = Literal.create(null, DecimalType(8, 6))), null) - checkEvaluation(makeTimeExpr.copy(sec = Literal(Decimal(BigDecimal(70.0), 8, 6))), null) - - makeTimeExpr = MakeTime(Literal(0), Literal(59), Literal(Decimal(BigDecimal(60.0), 8, 6))) - checkEvaluation(makeTimeExpr, LocalTime.parse("01:00:00")) - checkEvaluation(makeTimeExpr.copy(sec = Literal(Decimal(BigDecimal(60.5), 8, 6))), null) - - makeTimeExpr = MakeTime(Literal(0), Literal(0), Literal(Decimal(BigDecimal(58.000001), 8, 6))) - checkEvaluation(makeTimeExpr, LocalTime.parse("00:00:58.000001")) - } - test("millennium") { val date = MakeDate(Literal(2019), Literal(1), Literal(1)) checkEvaluation(Millennium(date), 3) From 50eaeb01f490b48e816c320a92fdb8878d7f6b0b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 17:36:30 +0500 Subject: [PATCH 16/18] Revert "Add the MakeTime expression" This reverts commit d3b5764844e65c1123a9ea1dfd86843760fcf403. --- .../expressions/datetimeExpressions.scala | 96 +------------------ 1 file changed, 1 insertion(+), 95 deletions(-) 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 74081b1dc1fa..1ce493ece18b 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 @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, LocalTime, ZoneId} -import java.time.temporal.ChronoField.MICRO_OF_DAY +import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, ZoneId} import java.time.temporal.IsoFields import java.util.{Locale, TimeZone} @@ -1894,99 +1893,6 @@ case class MakeTimestamp( override def prettyName: String = "make_timestamp" } -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(hour, min, sec) - Create time from hour, min, sec fields.", - arguments = """ - Arguments: - * hour - the hour-of-day to represent, from 0 to 23 - * min - the minute-of-hour to represent, from 0 to 59 - * sec - the second-of-minute and its micro-fraction to represent, from - 0 to 60. If the sec argument equals to 60, the seconds field is set - to 0 and 1 minute is added to the final timestamp. - """, - examples = """ - Examples: - > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887); - 2014-12-28 06:30:45.887 - > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET'); - 2014-12-28 10:30:45.887 - > SELECT _FUNC_(2019, 6, 30, 23, 59, 60) - 2019-07-01 00:00:00 - > SELECT _FUNC_(2019, 13, 1, 10, 11, 12, 13); - NULL - > SELECT _FUNC_(null, 7, 22, 15, 30, 0); - NULL - """, - since = "3.0.0") -// scalastyle:on line.size.limit -case class MakeTime(hour: Expression, min: Expression, sec: Expression) - extends TernaryExpression with ImplicitCastInputTypes { - - override def children: Seq[Expression] = Seq(hour, min, sec) - // Accept `sec` as DecimalType to avoid loosing precision of microseconds while converting - // them to the fractional part of `sec`. - override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, IntegerType, DecimalType(8, 6)) - override def dataType: DataType = TimeType - override def nullable: Boolean = true - - override def nullSafeEval(hourOfDay: Any, minutes: Any, seconds: Any): Any = { - val hour = hourOfDay.asInstanceOf[Int] - val min = minutes.asInstanceOf[Int] - val secAndNanos = seconds.asInstanceOf[Decimal] - - try { - val secFloor = secAndNanos.floor - val nanosPerSec = Decimal(NANOS_PER_SECOND, 10, 0) - val nanos = ((secAndNanos - secFloor) * nanosPerSec).toInt - val seconds = secFloor.toInt - val localTime = if (seconds == 60) { - if (nanos == 0) { - // This case of sec = 60 and nanos = 0 is supported for compatibility with PostgreSQL - LocalTime.of(hour, min, 0, 0).plusMinutes(1) - } else { - throw new DateTimeException("The fraction of sec must be zero. Valid range is [0, 60].") - } - } else { - LocalTime.of(hour, min, seconds, nanos) - } - localTime.getLong(MICRO_OF_DAY) - } catch { - case _: DateTimeException => null - } - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val d = Decimal.getClass.getName.stripSuffix("$") - nullSafeCodeGen(ctx, ev, (hour, min, secAndNanos) => { - s""" - try { - org.apache.spark.sql.types.Decimal secFloor = $secAndNanos.floor(); - org.apache.spark.sql.types.Decimal nanosPerSec = $d$$.MODULE$$.apply(1000000000L, 10, 0); - int nanos = (($secAndNanos.$$minus(secFloor)).$$times(nanosPerSec)).toInt(); - int seconds = secFloor.toInt(); - java.time.LocalTime localTime; - if (seconds == 60) { - if (nanos == 0) { - localTime = java.time.LocalTime.of($hour, $min, 0, 0).plusMinutes(1); - } else { - throw new java.time.DateTimeException( - "The fraction of sec must be zero. Valid range is [0, 60]."); - } - } else { - localTime = java.time.LocalTime.of($hour, $min, seconds, nanos); - } - localTime.getLong(java.time.temporal.ChronoField.MICRO_OF_DAY); - } catch (java.time.DateTimeException e) { - ${ev.isNull} = true; - }""" - }) - } - - override def prettyName: String = "make_time" -} - case class Millennium(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) From a8adadea8951abae4d2edfad93d838ecdfeef4b3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 18:15:42 +0500 Subject: [PATCH 17/18] Fix a typo --- sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 68327bb3c705..74e9c063606f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -130,7 +130,7 @@ object Encoders { /** * Creates an encoder that serializes instances of the `java.time.LocalTime` class - * to the internal representation of nullable Catalyst's DateType. + * to the internal representation of nullable Catalyst's TimeType. * * @since 3.0.0 */ From bd6b976b06a16b968b744fc4c11c6ac2f3999790 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 4 Sep 2019 18:35:34 +0500 Subject: [PATCH 18/18] Mark TimeType unstable --- .../main/scala/org/apache/spark/sql/types/TimeType.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala index bfbefa769424..e6d1e2a0a828 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimeType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.Stable +import org.apache.spark.annotation.Unstable /** * The time type represents local time in microsecond precision. @@ -29,7 +29,7 @@ import org.apache.spark.annotation.Stable * Please use the singleton `DataTypes.TimeType` to refer the type. * @since 3.0.0 */ -@Stable +@Unstable class TimeType private () extends AtomicType { /** @@ -57,5 +57,5 @@ class TimeType private () extends AtomicType { * * @since 3.0.0 */ -@Stable +@Unstable case object TimeType extends TimeType