You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/02/11 02:24:02 UTC

[GitHub] [spark] HyukjinKwon commented on a change in pull request #27524: [WIP][SQL] Support `SimpleDateFormat` and `FastDateFormat` as legacy date/timestamp formatters

HyukjinKwon commented on a change in pull request #27524: [WIP][SQL] Support `SimpleDateFormat` and `FastDateFormat` as legacy date/timestamp formatters
URL: https://github.com/apache/spark/pull/27524#discussion_r377421940
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
 ##########
 @@ -90,47 +91,168 @@ class FractionTimestampFormatter(zoneId: ZoneId)
   override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter
 }
 
-class LegacyTimestampFormatter(
+/**
+ * The custom sub-class of `GregorianCalendar` is needed to get access to
+ * protected `fields` immediately after parsing. We cannot use
+ * the `get()` method because it performs normalization of the fraction
+ * part. Accordingly, the `MILLISECOND` field doesn't contain original value.
+ *
+ * Also this class allows to set raw value to the `MILLISECOND` field
+ * directly before formatting.
+ */
+class MicrosCalendar(tz: TimeZone, digitsInFraction: Int)
+  extends GregorianCalendar(tz, Locale.US) {
+  // Converts parsed `MILLISECOND` field to seconds fraction in microsecond precision.
+  // For example if the fraction pattern is `SSSS` then `digitsInFraction` = 4, and
+  // if the `MILLISECOND` field was parsed to `1234`.
+  def getMicros(): SQLTimestamp = {
+    // Append 6 zeros to the field: 1234 -> 1234000000
+    val d = fields(Calendar.MILLISECOND) * MICROS_PER_SECOND
+    // Take the first 6 digits from `d`: 1234000000 -> 123400
+    // The rest contains exactly `digitsInFraction`: `0000` = 10 ^ digitsInFraction
+    // So, the result is `(1234 * 1000000) / (10 ^ digitsInFraction)
+    d / Decimal.POW_10(digitsInFraction)
+  }
+
+  // Converts the seconds fraction in microsecond precision to a value
+  // that can be correctly formatted according to the specified fraction pattern.
+  // The method performs operations opposite to `getMicros()`.
+  def setMicros(micros: Long): Unit = {
+    val d = micros * Decimal.POW_10(digitsInFraction)
+    fields(Calendar.MILLISECOND) = (d / MICROS_PER_SECOND).toInt
+  }
+}
+
+/**
+ * An instance of the class is aimed to re-use many times. It contains helper objects
+ * `cal` which is reused between `parse()` and `format` invokes.
+ */
+class LegacyFastDateFormat(fastDateFormat: FastDateFormat) {
+  private val cal = new MicrosCalendar(
+    fastDateFormat.getTimeZone,
+    fastDateFormat.getPattern.count(_ == 'S'))
+
+  def parse(s: String): SQLTimestamp = {
+    cal.clear() // Clear the calendar because it can be re-used many times
+    if (!fastDateFormat.parse(s, new ParsePosition(0), cal)) {
+      throw new IllegalArgumentException(s"'$s' is an invalid timestamp")
+    }
+    val micros = cal.getMicros()
+    cal.set(Calendar.MILLISECOND, 0)
+    cal.getTimeInMillis * MICROS_PER_MILLIS + micros
+  }
+
+  def format(timestamp: SQLTimestamp): String = {
+    cal.setTimeInMillis(Math.floorDiv(timestamp, MICROS_PER_SECOND) * MILLIS_PER_SECOND)
+    cal.setMicros(Math.floorMod(timestamp, MICROS_PER_SECOND))
+    fastDateFormat.format(cal)
+  }
+}
+
+class LegacyFastTimestampFormatter(
     pattern: String,
     zoneId: ZoneId,
     locale: Locale) extends TimestampFormatter {
 
-  @transient private lazy val format =
+  @transient private lazy val fastDateFormat =
     FastDateFormat.getInstance(pattern, TimeZone.getTimeZone(zoneId), locale)
+  @transient private lazy val cal = new MicrosCalendar(
+    fastDateFormat.getTimeZone,
+    fastDateFormat.getPattern.count(_ == 'S'))
 
 Review comment:
   I forgot. is this also ported from 2.4?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org