@@ -42,8 +42,8 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
4242 */
4343object DateTimeUtils {
4444
45- // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian
46- // it 's 2440587.5, rounding up to compatible with Hive
45+ // See http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian
46+ // It 's 2440587.5, rounding up to be compatible with Hive.
4747 final val JULIAN_DAY_OF_EPOCH = 2440588
4848
4949 final val TimeZoneUTC = TimeZone .getTimeZone(" UTC" )
@@ -53,10 +53,16 @@ object DateTimeUtils {
5353 def getZoneId (timeZoneId : String ): ZoneId = ZoneId .of(timeZoneId, ZoneId .SHORT_IDS )
5454 def getTimeZone (timeZoneId : String ): TimeZone = TimeZone .getTimeZone(getZoneId(timeZoneId))
5555
56+ /**
57+ * Converts microseconds since 1970-01-01 00:00:00Z to days since 1970-01-01 at the given zone ID.
58+ */
5659 def microsToDays (micros : Long , zoneId : ZoneId ): Int = {
5760 localDateToDays(getLocalDateTime(micros, zoneId).toLocalDate)
5861 }
5962
63+ /**
64+ * Converts days since 1970-01-01 at the given zone ID to microseconds since 1970-01-01 00:00:00Z.
65+ */
6066 def daysToMicros (days : Int , zoneId : ZoneId ): Long = {
6167 val instant = daysToLocalDate(days).atStartOfDay(zoneId).toInstant
6268 instantToMicros(instant)
@@ -180,7 +186,7 @@ object DateTimeUtils {
180186 }
181187
182188 /**
183- * Converts the timestamp to milliseconds since epoch. In spark timestamp values have microseconds
189+ * Converts the timestamp to milliseconds since epoch. In Spark timestamp values have microseconds
184190 * precision, so this conversion is lossy.
185191 */
186192 def microsToMillis (micros : Long ): Long = {
@@ -197,8 +203,8 @@ object DateTimeUtils {
197203 Math .multiplyExact(millis, MICROS_PER_MILLIS )
198204 }
199205
200- // A method called by JSON/CSV parser to clean up the legacy timestamp string by removing the
201- // "GMT" string.
206+ // The method is called by JSON/CSV parser to clean up the legacy timestamp string by removing
207+ // the "GMT" string.
202208 def cleanLegacyTimestampStr (s : String ): String = {
203209 val indexOfGMT = s.indexOf(" GMT" )
204210 if (indexOfGMT != - 1 ) {
@@ -213,8 +219,8 @@ object DateTimeUtils {
213219 }
214220
215221 /**
216- * Trim and parse a given UTF8 date string to the corresponding a corresponding [[Long ]] value.
217- * The return type is [[Option ]] in order to distinguish between 0L and null. The following
222+ * Trims and parses a given UTF8 timestamp string to the corresponding a corresponding [[Long ]]
223+ * value. The return type is [[Option ]] in order to distinguish between 0L and null. The following
218224 * formats are allowed:
219225 *
220226 * `yyyy`
@@ -369,12 +375,21 @@ object DateTimeUtils {
369375 }
370376 }
371377
378+ /**
379+ * Gets the number of microseconds since the epoch of 1970-01-01 00:00:00Z from the given
380+ * instance of `java.time.Instant`. The epoch microsecond count is a simple incrementing count of
381+ * microseconds where microsecond 0 is 1970-01-01 00:00:00Z.
382+ */
372383 def instantToMicros (instant : Instant ): Long = {
373384 val us = Math .multiplyExact(instant.getEpochSecond, MICROS_PER_SECOND )
374385 val result = Math .addExact(us, NANOSECONDS .toMicros(instant.getNano))
375386 result
376387 }
377388
389+ /**
390+ * Obtains an instance of `java.time.Instant` using microseconds from
391+ * the epoch of 1970-01-01 00:00:00Z.
392+ */
378393 def microsToInstant (micros : Long ): Instant = {
379394 val secs = Math .floorDiv(micros, MICROS_PER_SECOND )
380395 // Unfolded Math.floorMod(us, MICROS_PER_SECOND) to reuse the result of
@@ -383,12 +398,18 @@ object DateTimeUtils {
383398 Instant .ofEpochSecond(secs, mos * NANOS_PER_MICROS )
384399 }
385400
401+ /**
402+ * Converts the local date to the number of days since 1970-01-01.
403+ */
386404 def localDateToDays (localDate : LocalDate ): Int = Math .toIntExact(localDate.toEpochDay)
387405
406+ /**
407+ * Obtains an instance of `java.time.LocalDate` from the epoch day count.
408+ */
388409 def daysToLocalDate (days : Int ): LocalDate = LocalDate .ofEpochDay(days)
389410
390411 /**
391- * Trim and parse a given UTF8 date string to a corresponding [[Int ]] value.
412+ * Trims and parses a given UTF8 date string to a corresponding [[Int ]] value.
392413 * The return type is [[Option ]] in order to distinguish between 0 and null. The following
393414 * formats are allowed:
394415 *
@@ -447,6 +468,8 @@ object DateTimeUtils {
447468 }
448469 }
449470
471+ // Gets the local date-time parts (year, month, day and time) of the instant expressed as the
472+ // number of microseconds since the epoch at the given time zone ID.
450473 private def getLocalDateTime (micros : Long , zoneId : ZoneId ): LocalDateTime = {
451474 microsToInstant(micros).atZone(zoneId).toLocalDateTime
452475 }
@@ -523,16 +546,17 @@ object DateTimeUtils {
523546 def getDayOfMonth (days : Int ): Int = daysToLocalDate(days).getDayOfMonth
524547
525548 /**
526- * Add date and year-month interval.
527- * Returns a date value, expressed in days since 1970-01-01.
549+ * Adds an year-month interval to a date represented as days since 1970-01-01 .
550+ * @return a date value, expressed in days since 1970-01-01.
528551 */
529552 def dateAddMonths (days : Int , months : Int ): Int = {
530553 localDateToDays(daysToLocalDate(days).plusMonths(months))
531554 }
532555
533556 /**
534- * Add timestamp and full interval.
535- * Returns a timestamp value, expressed in microseconds since 1970-01-01 00:00:00Z.
557+ * Adds a full interval (months, days, microseconds) a timestamp represented as the number of
558+ * microseconds since 1970-01-01 00:00:00Z.
559+ * @return A timestamp value, expressed in microseconds since 1970-01-01 00:00:00Z.
536560 */
537561 def timestampAddInterval (
538562 start : Long ,
@@ -549,8 +573,8 @@ object DateTimeUtils {
549573 }
550574
551575 /**
552- * Add the date and the interval's months and days.
553- * Returns a date value, expressed in days since 1970-01-01.
576+ * Adds the interval's months and days to a date expressed as days since the epoch .
577+ * @return A date value, expressed in days since 1970-01-01.
554578 *
555579 * @throws DateTimeException if the result exceeds the supported date range
556580 * @throws IllegalArgumentException if the interval has `microseconds` part
@@ -565,7 +589,7 @@ object DateTimeUtils {
565589 }
566590
567591 /**
568- * Split date (expressed in days since 1970-01-01) into four fields:
592+ * Splits date (expressed in days since 1970-01-01) into four fields:
569593 * year, month (Jan is Month 1), dayInMonth, daysToMonthEnd (0 if it's last day of month).
570594 */
571595 private def splitDate (days : Int ): (Int , Int , Int , Int ) = {
@@ -744,19 +768,18 @@ object DateTimeUtils {
744768 }
745769
746770 /**
747- * Convert the timestamp `micros` from one timezone to another.
771+ * Converts the timestamp `micros` from one timezone to another.
748772 *
749- * TODO: Because of DST, the conversion between UTC and human time is not exactly one-to-one
750- * mapping, the conversion here may return wrong result, we should make the timestamp
751- * timezone-aware.
773+ * Time-zone rules, such as daylight savings, mean that not every local date-time
774+ * is valid for the `toZone` time zone, thus the local date-time may be adjusted.
752775 */
753776 def convertTz (micros : Long , fromZone : ZoneId , toZone : ZoneId ): Long = {
754777 val rebasedDateTime = getLocalDateTime(micros, toZone).atZone(fromZone)
755778 instantToMicros(rebasedDateTime.toInstant)
756779 }
757780
758781 /**
759- * Returns a timestamp of given timezone from utc timestamp, with the same string
782+ * Returns a timestamp of given timezone from UTC timestamp, with the same string
760783 * representation in their timezone.
761784 */
762785 def fromUTCTime (micros : Long , timeZone : String ): Long = {
@@ -771,8 +794,14 @@ object DateTimeUtils {
771794 convertTz(micros, getZoneId(timeZone), ZoneOffset .UTC )
772795 }
773796
797+ /**
798+ * Obtains the current instant as microseconds since the epoch at the UTC time zone.
799+ */
774800 def currentTimestamp (): Long = instantToMicros(Instant .now())
775801
802+ /**
803+ * Obtains the current date as days since the epoch in the specified time-zone.
804+ */
776805 def currentDate (zoneId : ZoneId ): Int = localDateToDays(LocalDate .now(zoneId))
777806
778807 private def today (zoneId : ZoneId ): ZonedDateTime = {
@@ -783,6 +812,7 @@ object DateTimeUtils {
783812
784813 /**
785814 * Extracts special values from an input string ignoring case.
815+ *
786816 * @param input A trimmed string
787817 * @param zoneId Zone identifier used to get the current date.
788818 * @return Some special value in lower case or None.
@@ -812,6 +842,7 @@ object DateTimeUtils {
812842
813843 /**
814844 * Converts notational shorthands that are converted to ordinary timestamps.
845+ *
815846 * @param input A trimmed string
816847 * @param zoneId Zone identifier used to get the current date.
817848 * @return Some of microseconds since the epoch if the conversion completed
@@ -838,6 +869,7 @@ object DateTimeUtils {
838869
839870 /**
840871 * Converts notational shorthands that are converted to ordinary dates.
872+ *
841873 * @param input A trimmed string
842874 * @param zoneId Zone identifier used to get the current date.
843875 * @return Some of days since the epoch if the conversion completed successfully otherwise None.
@@ -861,7 +893,8 @@ object DateTimeUtils {
861893 }
862894
863895 /**
864- * Subtracts two dates.
896+ * Subtracts two dates expressed as days since 1970-01-01.
897+ *
865898 * @param endDay The end date, exclusive
866899 * @param startDay The start date, inclusive
867900 * @return An interval between two dates. The interval can be negative
0 commit comments