Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
60 commits
Select commit Hold shift + click to select a range
ba506a2
Move TimeZoneGMT/UTC to DateTimeTestUtils
MaxGekk Feb 17, 2020
6933070
Remove TimeZoneGMT
MaxGekk Feb 17, 2020
e195d25
gmtId -> utcId
MaxGekk Feb 17, 2020
cf620cb
Remove defaultTimeZone
MaxGekk Feb 17, 2020
b487933
Use Date/TimestampType.InternalType
MaxGekk Feb 17, 2020
b26730c
Use TimeZoneUTC
MaxGekk Feb 17, 2020
0b5711e
Use TimeZoneUTC instead of GMT
MaxGekk Feb 17, 2020
9ba2d3d
Remove SQLDate and SQLTimestamp
MaxGekk Feb 21, 2020
c475f27
Improve a comment
MaxGekk Feb 21, 2020
cbf69fb
Renaming
MaxGekk Feb 21, 2020
dde714b
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Feb 24, 2020
6840291
Fix import
MaxGekk Feb 24, 2020
6a41a3c
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Feb 25, 2020
86ccef1
Fix merge
MaxGekk Feb 26, 2020
1fd3aa2
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 11, 2020
51e0bf1
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 12, 2020
54c0fe9
Fix merge issues
MaxGekk Mar 12, 2020
43246af
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 17, 2020
e12ec28
Fix merge issues
MaxGekk Mar 17, 2020
8cf53c5
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 19, 2020
9d78a7d
Restore TimeZoneUTC in DateTimeUtils
MaxGekk Mar 19, 2020
d45dd85
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 20, 2020
8669e29
Fix merge issue
MaxGekk Mar 20, 2020
aa8a62f
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 23, 2020
0075718
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 24, 2020
9ec98eb
Fix merge issue
MaxGekk Mar 24, 2020
2ca2540
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 26, 2020
d7f343b
Fix merge issues
MaxGekk Mar 26, 2020
aa2059a
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Mar 28, 2020
3b23c52
Fix merge issues
MaxGekk Mar 28, 2020
f79e7b6
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Apr 10, 2020
3215948
Fix merge conflicts
MaxGekk Apr 10, 2020
77a780f
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Apr 15, 2020
7dea87d
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Apr 25, 2020
28c8443
Fix merge conflicts
MaxGekk Apr 25, 2020
113d9a9
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Apr 27, 2020
70a057f
Fix merge conflicts
MaxGekk Apr 27, 2020
a9e73db
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Apr 30, 2020
5dfa257
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk May 14, 2020
1eb480a
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk May 21, 2020
9c53d3d
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk May 29, 2020
b43fe03
Avoid one `*` in fromJulianDay
MaxGekk May 29, 2020
61b85ac
Use toTotalMonths in subtractDates
MaxGekk May 29, 2020
311a47e
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Jun 10, 2020
6c084a1
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Jun 12, 2020
6e7aa16
Merge remote-tracking branch 'remotes/origin/master' into move-time-z…
MaxGekk Jun 16, 2020
b18859a
Resolve merge issue
MaxGekk Jun 16, 2020
f477a71
Remove timestampToString()
MaxGekk Jun 18, 2020
ec386b7
Remove microsToEpochDays() and epochDaysToMicros()
MaxGekk Jun 18, 2020
cf85866
Re-use getLocalDateTime in convertTz
MaxGekk Jun 18, 2020
89d7fad
Re-use getLocalDateTime
MaxGekk Jun 18, 2020
d9699c4
Remove julianCommonEraStart
MaxGekk Jun 18, 2020
d261ba7
Remove instantToDays
MaxGekk Jun 18, 2020
53a33da
Make splitDate private
MaxGekk Jun 18, 2020
ee91211
Remove def daysToMicros(days: Int): Long
MaxGekk Jun 18, 2020
386f6cf
Remove def microsToDays(micros: Long): Int
MaxGekk Jun 18, 2020
c47d522
1.1.1970 -> 1970-01-01
MaxGekk Jun 18, 2020
025ea80
1970-01-01 00:00:00 -> 1970-01-01 00:00:00Z
MaxGekk Jun 18, 2020
8a7ac3f
Make Scala style checker happy
MaxGekk Jun 18, 2020
863d747
Update comments
MaxGekk Jun 18, 2020
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 @@ -303,7 +303,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes)
case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d)))
case TimestampType => buildCast[Long](_,
t => UTF8String.fromString(DateTimeUtils.timestampToString(timestampFormatter, t)))
t => UTF8String.fromString(timestampFormatter.format(t)))
case ArrayType(et, _) =>
buildCast[ArrayData](_, array => {
val builder = new UTF8StringBuilder
Expand Down Expand Up @@ -443,7 +443,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
case ByteType =>
buildCast[Byte](_, b => longToTimestamp(b.toLong))
case DateType =>
buildCast[Int](_, d => epochDaysToMicros(d, zoneId))
buildCast[Int](_, d => daysToMicros(d, zoneId))
// TimestampWritable.decimalToTimestamp
case DecimalType() =>
buildCast[Decimal](_, d => decimalToTimestamp(d))
Expand Down Expand Up @@ -480,7 +480,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
case TimestampType =>
// throw valid precision more than seconds, according to Hive.
// Timestamp.nanos is in 0 to 999,999,999, no more than a second.
buildCast[Long](_, t => microsToEpochDays(t, zoneId))
buildCast[Long](_, t => microsToDays(t, zoneId))
}

// IntervalConverter
Expand Down Expand Up @@ -1034,8 +1034,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
val tf = JavaCode.global(
ctx.addReferenceObj("timestampFormatter", timestampFormatter),
timestampFormatter.getClass)
(c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString(
org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));"""
(c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString($tf.format($c));"""
case CalendarIntervalType =>
(c, evPrim, _) => code"""$evPrim = UTF8String.fromString($c.toString());"""
case ArrayType(et, _) =>
Expand Down Expand Up @@ -1120,7 +1119,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
val zid = getZoneId()
(c, evPrim, evNull) =>
code"""$evPrim =
org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);"""
org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToDays($c, $zid);"""
case _ =>
(c, evPrim, evNull) => code"$evNull = true;"
}
Expand Down Expand Up @@ -1247,7 +1246,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
zoneIdClass)
(c, evPrim, evNull) =>
code"""$evPrim =
org.apache.spark.sql.catalyst.util.DateTimeUtils.epochDaysToMicros($c, $zid);"""
org.apache.spark.sql.catalyst.util.DateTimeUtils.daysToMicros($c, $zid);"""
case DecimalType() =>
(c, evPrim, evNull) => code"$evPrim = ${decimalToTimestampCode(c)};"
case DoubleType =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -895,7 +895,7 @@ abstract class ToTimestamp
} else {
left.dataType match {
case DateType =>
epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor
daysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor
case TimestampType =>
t.asInstanceOf[Long] / downScaleFactor
case StringType =>
Expand Down Expand Up @@ -975,7 +975,7 @@ abstract class ToTimestamp
boolean ${ev.isNull} = ${eval1.isNull};
$javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
if (!${ev.isNull}) {
${ev.value} = $dtu.epochDaysToMicros(${eval1.value}, $zid) / $downScaleFactor;
${ev.value} = $dtu.daysToMicros(${eval1.value}, $zid) / $downScaleFactor;
}""")
}
}
Expand Down Expand Up @@ -1242,10 +1242,10 @@ case class DateAddInterval(
if (ansiEnabled || itvl.microseconds == 0) {
DateTimeUtils.dateAddInterval(start.asInstanceOf[Int], itvl)
} else {
val startTs = DateTimeUtils.epochDaysToMicros(start.asInstanceOf[Int], zoneId)
val startTs = DateTimeUtils.daysToMicros(start.asInstanceOf[Int], zoneId)
val resultTs = DateTimeUtils.timestampAddInterval(
startTs, itvl.months, itvl.days, itvl.microseconds, zoneId)
DateTimeUtils.microsToEpochDays(resultTs, zoneId)
DateTimeUtils.microsToDays(resultTs, zoneId)
}
}

Expand All @@ -1261,10 +1261,10 @@ case class DateAddInterval(
|if ($i.microseconds == 0) {
| ${ev.value} = $dtu.dateAddInterval($sd, $i);
|} else {
| long $startTs = $dtu.epochDaysToMicros($sd, $zid);
| long $startTs = $dtu.daysToMicros($sd, $zid);
| long $resultTs =
| $dtu.timestampAddInterval($startTs, $i.months, $i.days, $i.microseconds, $zid);
| ${ev.value} = $dtu.microsToEpochDays($resultTs, $zid);
| ${ev.value} = $dtu.microsToDays($resultTs, $zid);
|}
|""".stripMargin
})
Expand Down
Loading