Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,8 @@
*/
package org.apache.spark.sql.catalyst.expressions

import java.util.{Comparator, TimeZone}
import java.time.ZoneId
import java.util.Comparator

import scala.collection.mutable
import scala.reflect.ClassTag
Expand Down Expand Up @@ -2459,10 +2460,10 @@ case class Sequence(
new IntegralSequenceImpl(iType)(ct, iType.integral)

case TimestampType =>
new TemporalSequenceImpl[Long](LongType, 1, identity, timeZone)
new TemporalSequenceImpl[Long](LongType, 1, identity, zoneId)

case DateType =>
new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, timeZone)
new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, zoneId)
}

override def eval(input: InternalRow): Any = {
Expand Down Expand Up @@ -2603,7 +2604,7 @@ object Sequence {
}

private class TemporalSequenceImpl[T: ClassTag]
(dt: IntegralType, scale: Long, fromLong: Long => T, timeZone: TimeZone)
(dt: IntegralType, scale: Long, fromLong: Long => T, zoneId: ZoneId)
(implicit num: Integral[T]) extends SequenceImpl {

override val defaultStep: DefaultStep = new DefaultStep(
Expand Down Expand Up @@ -2642,7 +2643,7 @@ object Sequence {
while (t < exclusiveItem ^ stepSign < 0) {
arr(i) = fromLong(t / scale)
i += 1
t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, timeZone)
t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, zoneId)
}

// truncate array to the correct length
Expand All @@ -2668,7 +2669,7 @@ object Sequence {
val exclusiveItem = ctx.freshName("exclusiveItem")
val t = ctx.freshName("t")
val i = ctx.freshName("i")
val genTimeZone = ctx.addReferenceObj("timeZone", timeZone, classOf[TimeZone].getName)
val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)

val sequenceLengthCode =
s"""
Expand Down Expand Up @@ -2701,7 +2702,7 @@ object Sequence {
| $arr[$i] = ($elemType) ($t / ${scale}L);
| $i += 1;
| $t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval(
| $startMicros, $i * $stepMonths, $i * $stepMicros, $genTimeZone);
| $startMicros, $i * $stepMonths, $i * $stepMicros, $zid);
| }
|
| if ($arr.length > $i) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -996,14 +996,14 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S
override def nullSafeEval(start: Any, interval: Any): Any = {
val itvl = interval.asInstanceOf[CalendarInterval]
DateTimeUtils.timestampAddInterval(
start.asInstanceOf[Long], itvl.months, itvl.microseconds, timeZone)
start.asInstanceOf[Long], itvl.months, itvl.microseconds, zoneId)
}

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val tz = ctx.addReferenceObj("timeZone", timeZone)
val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
defineCodeGen(ctx, ev, (sd, i) => {
s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $tz)"""
s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $zid)"""
})
}
}
Expand Down Expand Up @@ -1111,14 +1111,14 @@ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[S
override def nullSafeEval(start: Any, interval: Any): Any = {
val itvl = interval.asInstanceOf[CalendarInterval]
DateTimeUtils.timestampAddInterval(
start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, timeZone)
start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, zoneId)
}

override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val tz = ctx.addReferenceObj("timeZone", timeZone)
val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
defineCodeGen(ctx, ev, (sd, i) => {
s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $tz)"""
s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $zid)"""
})
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@ package org.apache.spark.sql.catalyst.util

import java.sql.{Date, Timestamp}
import java.time._
import java.time.Year.isLeap
import java.time.temporal.IsoFields
import java.time.temporal.{ChronoUnit, IsoFields}
import java.util.{Locale, TimeZone}
import java.util.concurrent.TimeUnit._

Expand Down Expand Up @@ -521,12 +520,12 @@ object DateTimeUtils {
start: SQLTimestamp,
months: Int,
microseconds: Long,
timeZone: TimeZone): SQLTimestamp = {
val days = millisToDays(MICROSECONDS.toMillis(start), timeZone)
val newDays = dateAddMonths(days, months)
start +
MILLISECONDS.toMicros(daysToMillis(newDays, timeZone) - daysToMillis(days, timeZone)) +
microseconds
zoneId: ZoneId): SQLTimestamp = {
val resultTimestamp = microsToInstant(start)
.atZone(zoneId)
.plusMonths(months)
.plus(microseconds, ChronoUnit.MICROS)
instantToMicros(resultTimestamp.toInstant)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ import org.apache.spark.unsafe.types.UTF8String
class DateTimeUtilsSuite extends SparkFunSuite {

val TimeZonePST = TimeZone.getTimeZone("PST")
private def defaultTz = DateTimeUtils.defaultTimeZone()
private def defaultZoneId = ZoneId.systemDefault()

test("nanoseconds truncation") {
Expand Down Expand Up @@ -366,13 +365,13 @@ class DateTimeUtilsSuite extends SparkFunSuite {
test("timestamp add months") {
val ts1 = date(1997, 2, 28, 10, 30, 0)
val ts2 = date(2000, 2, 28, 10, 30, 0, 123000)
assert(timestampAddInterval(ts1, 36, 123000, defaultTz) === ts2)
assert(timestampAddInterval(ts1, 36, 123000, defaultZoneId) === ts2)

val ts3 = date(1997, 2, 27, 16, 0, 0, 0, TimeZonePST)
val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, TimeZonePST)
val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, TimeZoneGMT)
assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST) === ts4)
assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT) === ts5)
assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST.toZoneId) === ts4)
assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT.toZoneId) === ts5)
}

test("monthsBetween") {
Expand Down