This is an automated email from the ASF dual-hosted git repository. srowen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 9b55722 [SPARK-27031][SQL] Avoid double formatting in timestampToString 9b55722 is described below commit 9b55722161e050fc104f5aac86cffd79bcd6905c Author: Maxim Gekk <max.g...@gmail.com> AuthorDate: Wed Mar 6 08:26:59 2019 -0600 [SPARK-27031][SQL] Avoid double formatting in timestampToString ## What changes were proposed in this pull request? Removed unnecessary conversion of microseconds in `DateTimeUtils.timestampToString` to `java.sql.Timestamp` which aims to output fraction of seconds by casting it to string. This was replaced by special `TimestampFormatter` which appends the fraction formatter to `DateTimeFormatterBuilder`: `appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)`. The former one means trailing zeros in second's fraction should be truncated while formatting. ## How was this patch tested? By existing test suites like `CastSuite`, `DateTimeUtilsSuite`, `JDBCSuite`, and by new test in `TimestampFormatterSuite`. Closes #23936 from MaxGekk/timestamp-to-string. Lead-authored-by: Maxim Gekk <max.g...@gmail.com> Co-authored-by: Maxim Gekk <maxim.g...@databricks.com> Signed-off-by: Sean Owen <sean.o...@databricks.com> --- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../catalyst/util/DateTimeFormatterHelper.scala | 26 ++++++++++++++++++---- .../spark/sql/catalyst/util/DateTimeUtils.scala | 10 +-------- .../sql/catalyst/util/TimestampFormatter.scala | 21 ++++++++++++++++- .../sql/catalyst/util/DateTimeUtilsSuite.scala | 2 +- .../spark/sql/util/TimestampFormatterSuite.scala | 11 ++++++++- .../apache/spark/sql/execution/HiveResult.scala | 2 +- .../execution/datasources/jdbc/JDBCRelation.scala | 3 ++- 8 files changed, 58 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 84087ae..c238ccb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -233,7 +233,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) private lazy val dateFormatter = DateFormatter() - private lazy val timestampFormatter = TimestampFormatter(timeZone) + private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(timeZone) // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala index 81ad6ad..a0c0db3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -62,10 +62,12 @@ private object DateTimeFormatterHelper { .maximumSize(128) .build[(String, Locale), DateTimeFormatter]() - def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .appendPattern(pattern) + def createBuilder(): DateTimeFormatterBuilder = { + new DateTimeFormatterBuilder().parseCaseInsensitive() + } + + def toFormatter(builder: DateTimeFormatterBuilder, locale: Locale): DateTimeFormatter = { + builder .parseDefaulting(ChronoField.ERA, 1) .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) @@ -75,4 +77,20 @@ private object DateTimeFormatterHelper { .withChronology(IsoChronology.INSTANCE) .withResolverStyle(ResolverStyle.STRICT) } + + def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { + val builder = createBuilder().appendPattern(pattern) + toFormatter(builder, locale) + } + + lazy val fractionFormatter: DateTimeFormatter = { + val builder = createBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .appendValue(ChronoField.HOUR_OF_DAY, 2).appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2).appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + toFormatter(builder, TimestampFormatter.defaultLocale) + } } 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 5064220..627ee14 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 @@ -99,15 +99,7 @@ object DateTimeUtils { // Converts Timestamp to string according to Hive TimestampWritable convention. def timestampToString(tf: TimestampFormatter, us: SQLTimestamp): String = { - val ts = toJavaTimestamp(us) - val timestampString = ts.toString - val formatted = tf.format(us) - - if (timestampString.length > 19 && timestampString.substring(19) != ".0") { - formatted + timestampString.substring(19) - } else { - formatted - } + tf.format(us) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index c254815..e559b6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -47,7 +47,7 @@ class Iso8601TimestampFormatter( timeZone: TimeZone, locale: Locale) extends TimestampFormatter with DateTimeFormatterHelper { @transient - private lazy val formatter = getOrCreateFormatter(pattern, locale) + protected lazy val formatter = getOrCreateFormatter(pattern, locale) private def toInstant(s: String): Instant = { val temporalAccessor = formatter.parse(s) @@ -66,6 +66,21 @@ class Iso8601TimestampFormatter( } } +/** + * The formatter parses/formats timestamps according to the pattern `yyyy-MM-dd HH:mm:ss.[..fff..]` + * where `[..fff..]` is a fraction of second up to microsecond resolution. The formatter does not + * output trailing zeros in the fraction. For example, the timestamp `2019-03-05 15:00:01.123400` is + * formatted as the string `2019-03-05 15:00:01.1234`. + * + * @param timeZone the time zone in which the formatter parses or format timestamps + */ +class FractionTimestampFormatter(timeZone: TimeZone) + extends Iso8601TimestampFormatter("", timeZone, TimestampFormatter.defaultLocale) { + + @transient + override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter +} + object TimestampFormatter { val defaultPattern: String = "yyyy-MM-dd HH:mm:ss" val defaultLocale: Locale = Locale.US @@ -81,4 +96,8 @@ object TimestampFormatter { def apply(timeZone: TimeZone): TimestampFormatter = { apply(defaultPattern, timeZone, defaultLocale) } + + def getFractionFormatter(timeZone: TimeZone): TimestampFormatter = { + new FractionTimestampFormatter(timeZone) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index e270b91..39eb7d1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -33,7 +33,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { private def defaultTz = DateTimeUtils.defaultTimeZone() test("nanoseconds truncation") { - val tf = TimestampFormatter(DateTimeUtils.defaultTimeZone()) + val tf = TimestampFormatter.getFractionFormatter(DateTimeUtils.defaultTimeZone()) def checkStringToTimestamp(originalTime: String, expectedParsedTime: String) { val parsedTimestampOp = DateTimeUtils.stringToTimestamp( UTF8String.fromString(originalTime), defaultTz) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index d007adf..1675b61 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.util import java.time.{LocalDateTime, ZoneOffset} -import java.util.{Locale, TimeZone} +import java.util.TimeZone import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite @@ -117,4 +117,13 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { assert(micros === TimeUnit.SECONDS.toMicros( LocalDateTime.of(2009, 3, 20, 11, 30, 1).toEpochSecond(ZoneOffset.UTC))) } + + test("format fraction of second") { + val formatter = TimestampFormatter.getFractionFormatter(TimeZone.getTimeZone("UTC")) + assert(formatter.format(0) === "1970-01-01 00:00:00") + assert(formatter.format(1) === "1970-01-01 00:00:00.000001") + assert(formatter.format(1000) === "1970-01-01 00:00:00.001") + assert(formatter.format(900000) === "1970-01-01 00:00:00.9") + assert(formatter.format(1000000) === "1970-01-01 00:00:01") + } } 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 24fba79..38ef72e 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 @@ -78,7 +78,7 @@ object HiveResult { BinaryType) private lazy val dateFormatter = DateFormatter() - private lazy val timestampFormatter = TimestampFormatter( + private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone)) /** Hive outputs fields of structs slightly differently than top level attributes. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index c0f78b5..724a0f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -199,7 +199,8 @@ private[sql] object JDBCRelation extends Logging { val dateTimeStr = columnType match { case DateType => DateFormatter().format(value.toInt) case TimestampType => - val timestampFormatter = TimestampFormatter(DateTimeUtils.getTimeZone(timeZoneId)) + val timestampFormatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getTimeZone(timeZoneId)) DateTimeUtils.timestampToString(timestampFormatter, value) } s"'$dateTimeStr'" --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org