You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2020/03/17 14:08:49 UTC

[spark] branch branch-3.0 updated: [SPARK-31150][SQL] Parsing seconds fraction with variable length for timestamp

This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new 41c1dbd  [SPARK-31150][SQL] Parsing seconds fraction with variable length for timestamp
41c1dbd is described below

commit 41c1dbd2fa77816582351bb129a6a7d343f56cc2
Author: Kent Yao <ya...@hotmail.com>
AuthorDate: Tue Mar 17 21:53:46 2020 +0800

    [SPARK-31150][SQL] Parsing seconds fraction with variable length for timestamp
    
    ### What changes were proposed in this pull request?
    This PR is to support parsing timestamp values with variable length second fraction parts.
    
    e.g. 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]' can parse timestamp with 0~6 digit-length second fraction but fail >=7
    ```sql
    select to_timestamp(v, 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') from values
     ('2019-10-06 10:11:12.'),
     ('2019-10-06 10:11:12.0'),
     ('2019-10-06 10:11:12.1'),
     ('2019-10-06 10:11:12.12'),
     ('2019-10-06 10:11:12.123UTC'),
     ('2019-10-06 10:11:12.1234'),
     ('2019-10-06 10:11:12.12345CST'),
     ('2019-10-06 10:11:12.123456PST') t(v)
    2019-10-06 03:11:12.123
    2019-10-06 08:11:12.12345
    2019-10-06 10:11:12
    2019-10-06 10:11:12
    2019-10-06 10:11:12.1
    2019-10-06 10:11:12.12
    2019-10-06 10:11:12.1234
    2019-10-06 10:11:12.123456
    
    select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
    NULL
    ```
    Since 3.0, we use java 8 time API to parse and format timestamp values. when we create the `DateTimeFormatter`, we use `appendPattern` to create the build first, where the 'S..S' part will be parsed to a fixed-length(= `'S..S'.length`). This fits the formatting part but too strict for the parsing part because the trailing zeros are very likely to be truncated.
    
    ### Why are the changes needed?
    
    improve timestamp parsing and more compatible with 2.4.x
    
    ### Does this PR introduce any user-facing change?
    
    no, the related changes are newly added
    ### How was this patch tested?
    
    add uts
    
    Closes #27906 from yaooqinn/SPARK-31150.
    
    Authored-by: Kent Yao <ya...@hotmail.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit 0946a9514f56565c78b0555383c1ece14aaf2b7b)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../spark/sql/catalyst/catalog/interface.scala     |  11 +-
 .../spark/sql/catalyst/csv/CSVInferSchema.scala    |   3 +-
 .../sql/catalyst/csv/UnivocityGenerator.scala      |   3 +-
 .../spark/sql/catalyst/csv/UnivocityParser.scala   |   3 +-
 .../catalyst/expressions/datetimeExpressions.scala |  44 ++++-
 .../spark/sql/catalyst/json/JacksonGenerator.scala |   3 +-
 .../spark/sql/catalyst/json/JacksonParser.scala    |   3 +-
 .../spark/sql/catalyst/json/JsonInferSchema.scala  |   3 +-
 .../catalyst/util/DateTimeFormatterHelper.scala    |  50 ++++-
 .../sql/catalyst/util/TimestampFormatter.scala     |  37 ++--
 .../spark/sql/util/TimestampFormatterSuite.scala   |  24 +--
 .../benchmarks/DateTimeBenchmark-jdk11-results.txt | 218 ++++++++++-----------
 sql/core/benchmarks/DateTimeBenchmark-results.txt  | 218 ++++++++++-----------
 .../execution/datasources/PartitioningUtils.scala  |   5 +-
 .../test/resources/sql-tests/inputs/datetime.sql   |  25 +++
 .../resources/sql-tests/results/datetime.sql.out   | 170 +++++++++++++++-
 .../execution/benchmark/DateTimeBenchmark.scala    |   2 +-
 .../parquet/ParquetPartitionDiscoverySuite.scala   |   3 +-
 18 files changed, 555 insertions(+), 270 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index 6e965ef..d02776b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -521,8 +521,11 @@ object CatalogColumnStat extends Logging {
 
   val VERSION = 2
 
-  private def getTimestampFormatter(): TimestampFormatter = {
-    TimestampFormatter(format = "yyyy-MM-dd HH:mm:ss.SSSSSS", zoneId = ZoneOffset.UTC)
+  private def getTimestampFormatter(isParsing: Boolean): TimestampFormatter = {
+    TimestampFormatter(
+      format = "yyyy-MM-dd HH:mm:ss.SSSSSS",
+      zoneId = ZoneOffset.UTC,
+      needVarLengthSecondFraction = isParsing)
   }
 
   /**
@@ -535,7 +538,7 @@ object CatalogColumnStat extends Logging {
       case DateType => DateFormatter(ZoneOffset.UTC).parse(s)
       case TimestampType if version == 1 =>
         DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s))
-      case TimestampType => getTimestampFormatter().parse(s)
+      case TimestampType => getTimestampFormatter(isParsing = true).parse(s)
       case ByteType => s.toByte
       case ShortType => s.toShort
       case IntegerType => s.toInt
@@ -558,7 +561,7 @@ object CatalogColumnStat extends Logging {
   def toExternalString(v: Any, colName: String, dataType: DataType): String = {
     val externalValue = dataType match {
       case DateType => DateFormatter(ZoneOffset.UTC).format(v.asInstanceOf[Int])
-      case TimestampType => getTimestampFormatter().format(v.asInstanceOf[Long])
+      case TimestampType => getTimestampFormatter(isParsing = false).format(v.asInstanceOf[Long])
       case BooleanType | _: IntegralType | FloatType | DoubleType => v
       case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal
       // This version of Spark does not use min/max for binary/string types so we ignore it.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
index c6a0318..a7c2435 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
@@ -34,7 +34,8 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable {
     options.timestampFormat,
     options.zoneId,
     options.locale,
-    legacyFormat = FAST_DATE_FORMAT)
+    legacyFormat = FAST_DATE_FORMAT,
+    needVarLengthSecondFraction = true)
 
   private val decimalParser = if (options.locale == Locale.US) {
     // Special handling the default locale for backward compatibility
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
index 00e3d49..4990da2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
@@ -46,7 +46,8 @@ class UnivocityGenerator(
     options.timestampFormat,
     options.zoneId,
     options.locale,
-    legacyFormat = FAST_DATE_FORMAT)
+    legacyFormat = FAST_DATE_FORMAT,
+    needVarLengthSecondFraction = false)
   private val dateFormatter = DateFormatter(
     options.dateFormat,
     options.zoneId,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
index dd8537b..2c5a9d7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
@@ -89,7 +89,8 @@ class UnivocityParser(
     options.timestampFormat,
     options.zoneId,
     options.locale,
-    legacyFormat = FAST_DATE_FORMAT)
+    legacyFormat = FAST_DATE_FORMAT,
+    needVarLengthSecondFraction = true)
   private val dateFormatter = DateFormatter(
     options.dateFormat,
     options.zoneId,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 0b1b65f..d71db0d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -626,14 +626,22 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti
   @transient private lazy val formatter: Option[TimestampFormatter] = {
     if (right.foldable) {
       Option(right.eval()).map { format =>
-        TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT)
+        TimestampFormatter(
+          format.toString,
+          zoneId,
+          legacyFormat = SIMPLE_DATE_FORMAT,
+          needVarLengthSecondFraction = false)
       }
     } else None
   }
 
   override protected def nullSafeEval(timestamp: Any, format: Any): Any = {
     val tf = if (formatter.isEmpty) {
-      TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT)
+      TimestampFormatter(
+        format.toString,
+        zoneId,
+        legacyFormat = SIMPLE_DATE_FORMAT,
+        needVarLengthSecondFraction = false)
     } else {
       formatter.get
     }
@@ -654,7 +662,8 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti
         s"""|UTF8String.fromString($tf$$.MODULE$$.apply(
             |  $format.toString(),
             |  $zid,
-            |  $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT())
+            |  $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(),
+            |  false)
             |.format($timestamp))""".stripMargin
       })
     }
@@ -767,7 +776,11 @@ abstract class ToTimestamp
   private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String]
   private lazy val formatter: TimestampFormatter =
     try {
-      TimestampFormatter(constFormat.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT)
+      TimestampFormatter(
+        constFormat.toString,
+        zoneId,
+        legacyFormat = SIMPLE_DATE_FORMAT,
+        needVarLengthSecondFraction = true)
     } catch {
       case NonFatal(_) => null
     }
@@ -801,7 +814,11 @@ abstract class ToTimestamp
           } else {
             val formatString = f.asInstanceOf[UTF8String].toString
             try {
-              TimestampFormatter(formatString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT)
+              TimestampFormatter(
+                formatString,
+                zoneId,
+                legacyFormat = SIMPLE_DATE_FORMAT,
+                needVarLengthSecondFraction = true)
                 .parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor
             } catch {
               case e: SparkUpgradeException => throw e
@@ -850,7 +867,8 @@ abstract class ToTimestamp
               ${ev.value} = $tf$$.MODULE$$.apply(
                 $format.toString(),
                 $zid,
-                $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT())
+                $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(),
+                true)
               .parse($string.toString()) / $downScaleFactor;
             } catch (java.lang.IllegalArgumentException e) {
               ${ev.isNull} = true;
@@ -936,7 +954,11 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[
   private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String]
   private lazy val formatter: TimestampFormatter =
     try {
-      TimestampFormatter(constFormat.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT)
+      TimestampFormatter(
+        constFormat.toString,
+        zoneId,
+        legacyFormat = SIMPLE_DATE_FORMAT,
+        needVarLengthSecondFraction = false)
     } catch {
       case NonFatal(_) => null
     }
@@ -963,7 +985,11 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[
         } else {
           try {
             UTF8String.fromString(
-              TimestampFormatter(f.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT)
+              TimestampFormatter(
+                f.toString,
+                zoneId,
+                legacyFormat = SIMPLE_DATE_FORMAT,
+                needVarLengthSecondFraction = false)
                 .format(time.asInstanceOf[Long] * MICROS_PER_SECOND))
           } catch {
             case NonFatal(_) => null
@@ -1001,7 +1027,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[
         s"""
         try {
           ${ev.value} = UTF8String.fromString(
-            $tf$$.MODULE$$.apply($f.toString(), $zid, $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT())
+            $tf$$.MODULE$$.apply($f.toString(), $zid, $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT(), false)
               .format($seconds * 1000000L));
         } catch (java.lang.IllegalArgumentException e) {
           ${ev.isNull} = true;
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
index 141360f..f3938fe 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
@@ -82,7 +82,8 @@ private[sql] class JacksonGenerator(
     options.timestampFormat,
     options.zoneId,
     options.locale,
-    legacyFormat = FAST_DATE_FORMAT)
+    legacyFormat = FAST_DATE_FORMAT,
+    needVarLengthSecondFraction = false)
   private val dateFormatter = DateFormatter(
     options.dateFormat,
     options.zoneId,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
index d0db06c..8965a81 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
@@ -60,7 +60,8 @@ class JacksonParser(
     options.timestampFormat,
     options.zoneId,
     options.locale,
-    legacyFormat = FAST_DATE_FORMAT)
+    legacyFormat = FAST_DATE_FORMAT,
+    needVarLengthSecondFraction = true)
   private val dateFormatter = DateFormatter(
     options.dateFormat,
     options.zoneId,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
index 3dd8694..56b1278 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
@@ -42,7 +42,8 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable {
     options.timestampFormat,
     options.zoneId,
     options.locale,
-    legacyFormat = FAST_DATE_FORMAT)
+    legacyFormat = FAST_DATE_FORMAT,
+    needVarLengthSecondFraction = true)
 
   /**
    * Infer the type of a collection of json records in three stages:
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 a59f49b..668ce92 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
@@ -51,12 +51,16 @@ trait DateTimeFormatterHelper {
   // In this way, synchronised is intentionally omitted in this method to make parallel calls
   // less synchronised.
   // The Cache.get method is not used here to avoid creation of additional instances of Callable.
-  protected def getOrCreateFormatter(pattern: String, locale: Locale): DateTimeFormatter = {
+  protected def getOrCreateFormatter(
+      pattern: String,
+      locale: Locale,
+      needVarLengthSecondFraction: Boolean = false): DateTimeFormatter = {
     val newPattern = DateTimeUtils.convertIncompatiblePattern(pattern)
-    val key = (newPattern, locale)
+    val useVarLen = needVarLengthSecondFraction && newPattern.contains('S')
+    val key = (newPattern, locale, useVarLen)
     var formatter = cache.getIfPresent(key)
     if (formatter == null) {
-      formatter = buildFormatter(newPattern, locale)
+      formatter = buildFormatter(newPattern, locale, useVarLen)
       cache.put(key, formatter)
     }
     formatter
@@ -87,7 +91,9 @@ trait DateTimeFormatterHelper {
 private object DateTimeFormatterHelper {
   val cache = CacheBuilder.newBuilder()
     .maximumSize(128)
-    .build[(String, Locale), DateTimeFormatter]()
+    .build[(String, Locale, Boolean), DateTimeFormatter]()
+
+  final val extractor = "^([^S]*)(S*)(.*)$".r
 
   def createBuilder(): DateTimeFormatterBuilder = {
     new DateTimeFormatterBuilder().parseCaseInsensitive()
@@ -104,8 +110,40 @@ private object DateTimeFormatterHelper {
       .withResolverStyle(ResolverStyle.STRICT)
   }
 
-  def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = {
-    val builder = createBuilder().appendPattern(pattern)
+  /**
+   * Building a formatter for parsing seconds fraction with variable length
+   */
+  def createBuilderWithVarLengthSecondFraction(
+      pattern: String): DateTimeFormatterBuilder = {
+    val builder = createBuilder()
+    pattern.split("'").zipWithIndex.foreach {
+      case (pattenPart, idx) if idx % 2 == 0 =>
+        var rest = pattenPart
+        while (rest.nonEmpty) {
+          rest match {
+            case extractor(prefix, secondFraction, suffix) =>
+              builder.appendPattern(prefix)
+              if (secondFraction.nonEmpty) {
+                builder.appendFraction(ChronoField.NANO_OF_SECOND, 1, secondFraction.length, false)
+              }
+              rest = suffix
+            case _ => throw new IllegalArgumentException(s"Unrecognized datetime pattern: $pattern")
+          }
+        }
+      case (patternPart, _) => builder.appendLiteral(patternPart)
+    }
+    builder
+  }
+
+  def buildFormatter(
+      pattern: String,
+      locale: Locale,
+      varLenEnabled: Boolean): DateTimeFormatter = {
+    val builder = if (varLenEnabled) {
+      createBuilderWithVarLengthSecondFraction(pattern)
+    } else {
+      createBuilder().appendPattern(pattern)
+    }
     toFormatter(builder, locale)
   }
 
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 99139ca..0c4fa17 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
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util
 
 import java.text.{ParseException, ParsePosition, SimpleDateFormat}
 import java.time._
-import java.time.format.DateTimeParseException
+import java.time.format.{DateTimeFormatter, DateTimeParseException}
 import java.time.temporal.ChronoField.MICRO_OF_SECOND
 import java.time.temporal.TemporalQueries
 import java.util.{Calendar, GregorianCalendar, Locale, TimeZone}
@@ -55,10 +55,12 @@ class Iso8601TimestampFormatter(
     pattern: String,
     zoneId: ZoneId,
     locale: Locale,
-    legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT)
+    legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT,
+    needVarLengthSecondFraction: Boolean)
   extends TimestampFormatter with DateTimeFormatterHelper {
   @transient
-  protected lazy val formatter = getOrCreateFormatter(pattern, locale)
+  protected lazy val formatter: DateTimeFormatter =
+    getOrCreateFormatter(pattern, locale, needVarLengthSecondFraction)
 
   @transient
   protected lazy val legacyFormatter = TimestampFormatter.getLegacyFormatter(
@@ -95,7 +97,8 @@ class Iso8601TimestampFormatter(
  * @param zoneId the time zone identifier in which the formatter parses or format timestamps
  */
 class FractionTimestampFormatter(zoneId: ZoneId)
-  extends Iso8601TimestampFormatter("", zoneId, TimestampFormatter.defaultLocale) {
+  extends Iso8601TimestampFormatter(
+    "", zoneId, TimestampFormatter.defaultLocale, needVarLengthSecondFraction = false) {
 
   @transient
   override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter
@@ -199,12 +202,14 @@ object TimestampFormatter {
       format: Option[String],
       zoneId: ZoneId,
       locale: Locale = defaultLocale,
-      legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT): TimestampFormatter = {
+      legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT,
+      needVarLengthSecondFraction: Boolean = false): TimestampFormatter = {
     val pattern = format.getOrElse(defaultPattern)
     if (SQLConf.get.legacyTimeParserPolicy == LEGACY) {
       getLegacyFormatter(pattern, zoneId, locale, legacyFormat)
     } else {
-      new Iso8601TimestampFormatter(pattern, zoneId, locale, legacyFormat)
+      new Iso8601TimestampFormatter(
+        pattern, zoneId, locale, legacyFormat, needVarLengthSecondFraction)
     }
   }
 
@@ -227,16 +232,24 @@ object TimestampFormatter {
       format: String,
       zoneId: ZoneId,
       locale: Locale,
-      legacyFormat: LegacyDateFormat): TimestampFormatter = {
-    getFormatter(Some(format), zoneId, locale, legacyFormat)
+      legacyFormat: LegacyDateFormat,
+      needVarLengthSecondFraction: Boolean): TimestampFormatter = {
+    getFormatter(Some(format), zoneId, locale, legacyFormat, needVarLengthSecondFraction)
   }
 
-  def apply(format: String, zoneId: ZoneId, legacyFormat: LegacyDateFormat): TimestampFormatter = {
-    getFormatter(Some(format), zoneId, defaultLocale, legacyFormat)
+  def apply(
+      format: String,
+      zoneId: ZoneId,
+      legacyFormat: LegacyDateFormat,
+      needVarLengthSecondFraction: Boolean): TimestampFormatter = {
+    getFormatter(Some(format), zoneId, defaultLocale, legacyFormat, needVarLengthSecondFraction)
   }
 
-  def apply(format: String, zoneId: ZoneId): TimestampFormatter = {
-    getFormatter(Some(format), zoneId)
+  def apply(
+      format: String,
+      zoneId: ZoneId,
+      needVarLengthSecondFraction: Boolean = false): TimestampFormatter = {
+    getFormatter(Some(format), zoneId, needVarLengthSecondFraction = needVarLengthSecondFraction)
   }
 
   def apply(zoneId: ZoneId): TimestampFormatter = {
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 959ef8e..3cf1437 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
@@ -44,7 +44,8 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
     DateTimeTestUtils.outstandingTimezonesIds.foreach { zoneId =>
       val formatter = TimestampFormatter(
         "yyyy-MM-dd'T'HH:mm:ss.SSSSSS",
-        DateTimeUtils.getZoneId(zoneId))
+        DateTimeUtils.getZoneId(zoneId),
+        needVarLengthSecondFraction = true)
       val microsSinceEpoch = formatter.parse(localDate)
       assert(microsSinceEpoch === expectedMicros(zoneId))
     }
@@ -83,9 +84,9 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
         2177456523456789L,
         11858049903010203L).foreach { micros =>
         DateTimeTestUtils.outstandingZoneIds.foreach { zoneId =>
-          val formatter = TimestampFormatter(pattern, zoneId)
-          val timestamp = formatter.format(micros)
-          val parsed = formatter.parse(timestamp)
+          val timestamp = TimestampFormatter(pattern, zoneId).format(micros)
+          val parsed = TimestampFormatter(
+            pattern, zoneId, needVarLengthSecondFraction = true).parse(timestamp)
           assert(micros === parsed)
         }
       }
@@ -104,9 +105,10 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
       "2039-01-01T01:02:03.456789",
       "2345-10-07T22:45:03.010203").foreach { timestamp =>
       DateTimeTestUtils.outstandingZoneIds.foreach { zoneId =>
-        val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", zoneId)
-        val micros = formatter.parse(timestamp)
-        val formatted = formatter.format(micros)
+        val pattern = "yyyy-MM-dd'T'HH:mm:ss.SSSSSS"
+        val micros = TimestampFormatter(
+          pattern, zoneId, needVarLengthSecondFraction = true).parse(timestamp)
+        val formatted = TimestampFormatter(pattern, zoneId).format(micros)
         assert(timestamp === formatted)
       }
     }
@@ -159,7 +161,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
   test("parsing timestamp strings with various seconds fractions") {
     DateTimeTestUtils.outstandingZoneIds.foreach { zoneId =>
       def check(pattern: String, input: String, reference: String): Unit = {
-        val formatter = TimestampFormatter(pattern, zoneId)
+        val formatter = TimestampFormatter(pattern, zoneId, needVarLengthSecondFraction = true)
         val expected = DateTimeUtils.stringToTimestamp(
           UTF8String.fromString(reference), zoneId).get
         val actual = formatter.parse(input)
@@ -184,7 +186,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
         "2019-10-14T09:39:07.1", "2019-10-14T09:39:07.1")
 
       try {
-        TimestampFormatter("yyyy/MM/dd HH_mm_ss.SSSSSS", zoneId)
+        TimestampFormatter("yyyy/MM/dd HH_mm_ss.SSSSSS", zoneId, true)
           .parse("2019/11/14 20#25#30.123456")
         fail("Expected to throw an exception for the invalid input")
       } catch {
@@ -235,7 +237,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
   }
 
   test("SPARK-30958: parse timestamp with negative year") {
-    val formatter1 = TimestampFormatter("yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC)
+    val formatter1 = TimestampFormatter("yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC, true)
     assert(formatter1.parse("-1234-02-22 02:22:22") === instantToMicros(
       LocalDateTime.of(-1234, 2, 22, 2, 22, 22).toInstant(ZoneOffset.UTC)))
 
@@ -249,7 +251,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
     }
 
     // "yyyy" with "G" can't parse negative year or year 0000.
-    val formatter2 = TimestampFormatter("G yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC)
+    val formatter2 = TimestampFormatter("G yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC, true)
     assertParsingError(formatter2.parse("BC -1234-02-22 02:22:22"))
     assertParsingError(formatter2.parse("AC 0000-02-22 02:22:22"))
 
diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt
index 883f9de..3e415fa 100644
--- a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt
+++ b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt
@@ -6,92 +6,92 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 cast to timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to timestamp wholestage off                    221            232          16         45.3          22.1       1.0X
-cast to timestamp wholestage on                     213            256          71         46.9          21.3       1.0X
+cast to timestamp wholestage off                    217            225          10         46.0          21.7       1.0X
+cast to timestamp wholestage on                     195            224          35         51.2          19.5       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 year of timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-year of timestamp wholestage off                    863            961         139         11.6          86.3       1.0X
-year of timestamp wholestage on                     783            821          26         12.8          78.3       1.1X
+year of timestamp wholestage off                    735            739           5         13.6          73.5       1.0X
+year of timestamp wholestage on                     728            742          10         13.7          72.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 quarter of timestamp:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-quarter of timestamp wholestage off                1008           1013           7          9.9         100.8       1.0X
-quarter of timestamp wholestage on                  926            963          36         10.8          92.6       1.1X
+quarter of timestamp wholestage off                 940            941           2         10.6          94.0       1.0X
+quarter of timestamp wholestage on                  910            930          12         11.0          91.0       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 month of timestamp:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-month of timestamp wholestage off                   794            813          27         12.6          79.4       1.0X
-month of timestamp wholestage on                    737            758          23         13.6          73.7       1.1X
+month of timestamp wholestage off                   796            808          17         12.6          79.6       1.0X
+month of timestamp wholestage on                    688            706          11         14.5          68.8       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 weekofyear of timestamp:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-weekofyear of timestamp wholestage off             1063           1076          18          9.4         106.3       1.0X
-weekofyear of timestamp wholestage on              1070           1366         305          9.3         107.0       1.0X
+weekofyear of timestamp wholestage off             1013           1016           3          9.9         101.3       1.0X
+weekofyear of timestamp wholestage on               988           1011          18         10.1          98.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 day of timestamp:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-day of timestamp wholestage off                     847            854           9         11.8          84.7       1.0X
-day of timestamp wholestage on                      799            836          37         12.5          79.9       1.1X
+day of timestamp wholestage off                     686            698          17         14.6          68.6       1.0X
+day of timestamp wholestage on                      693            708          13         14.4          69.3       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 dayofyear of timestamp:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-dayofyear of timestamp wholestage off               816            830          20         12.2          81.6       1.0X
-dayofyear of timestamp wholestage on                856            925          62         11.7          85.6       1.0X
+dayofyear of timestamp wholestage off               711            716           7         14.1          71.1       1.0X
+dayofyear of timestamp wholestage on                713            723          12         14.0          71.3       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 dayofmonth of timestamp:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-dayofmonth of timestamp wholestage off             1150           1159          12          8.7         115.0       1.0X
-dayofmonth of timestamp wholestage on               945           1165         308         10.6          94.5       1.2X
+dayofmonth of timestamp wholestage off              706            726          28         14.2          70.6       1.0X
+dayofmonth of timestamp wholestage on               698            714          11         14.3          69.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 dayofweek of timestamp:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-dayofweek of timestamp wholestage off              1062           1074          16          9.4         106.2       1.0X
-dayofweek of timestamp wholestage on                942           1029          74         10.6          94.2       1.1X
+dayofweek of timestamp wholestage off               797            798           2         12.6          79.7       1.0X
+dayofweek of timestamp wholestage on                794            807          18         12.6          79.4       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 weekday of timestamp:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-weekday of timestamp wholestage off                 870            872           3         11.5          87.0       1.0X
-weekday of timestamp wholestage on                  821            855          21         12.2          82.1       1.1X
+weekday of timestamp wholestage off                 783            796          17         12.8          78.3       1.0X
+weekday of timestamp wholestage on                  759            778          21         13.2          75.9       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 hour of timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-hour of timestamp wholestage off                    578            582           5         17.3          57.8       1.0X
-hour of timestamp wholestage on                     533            544          14         18.8          53.3       1.1X
+hour of timestamp wholestage off                    573            575           2         17.5          57.3       1.0X
+hour of timestamp wholestage on                     512            522          16         19.5          51.2       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 minute of timestamp:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-minute of timestamp wholestage off                  611            613           3         16.4          61.1       1.0X
-minute of timestamp wholestage on                   551            557           8         18.2          55.1       1.1X
+minute of timestamp wholestage off                  550            555           7         18.2          55.0       1.0X
+minute of timestamp wholestage on                   507            527          21         19.7          50.7       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 second of timestamp:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-second of timestamp wholestage off                  705            716          16         14.2          70.5       1.0X
-second of timestamp wholestage on                   569            669          99         17.6          56.9       1.2X
+second of timestamp wholestage off                  562            587          35         17.8          56.2       1.0X
+second of timestamp wholestage on                   516            529          12         19.4          51.6       1.1X
 
 
 ================================================================================================
@@ -102,15 +102,15 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 current_date:                             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-current_date wholestage off                         177            177           0         56.6          17.7       1.0X
-current_date wholestage on                          164            180          13         60.9          16.4       1.1X
+current_date wholestage off                         161            164           4         61.9          16.1       1.0X
+current_date wholestage on                          163            170           8         61.2          16.3       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 current_timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-current_timestamp wholestage off                    182            182           1         55.1          18.2       1.0X
-current_timestamp wholestage on                     168            194          53         59.4          16.8       1.1X
+current_timestamp wholestage off                    174            177           4         57.5          17.4       1.0X
+current_timestamp wholestage on                     159            173          12         62.8          15.9       1.1X
 
 
 ================================================================================================
@@ -121,43 +121,43 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 cast to date:                             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to date wholestage off                         615            632          24         16.3          61.5       1.0X
-cast to date wholestage on                          596            660          40         16.8          59.6       1.0X
+cast to date wholestage off                         672            688          23         14.9          67.2       1.0X
+cast to date wholestage on                          592            605          10         16.9          59.2       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 last_day:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-last_day wholestage off                             717            728          16         13.9          71.7       1.0X
-last_day wholestage on                              764            834          42         13.1          76.4       0.9X
+last_day wholestage off                             806            809           4         12.4          80.6       1.0X
+last_day wholestage on                              700            713          14         14.3          70.0       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 next_day:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-next_day wholestage off                             677            677           0         14.8          67.7       1.0X
-next_day wholestage on                              645            796         140         15.5          64.5       1.1X
+next_day wholestage off                             639            683          63         15.7          63.9       1.0X
+next_day wholestage on                              628            640          15         15.9          62.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_add:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_add wholestage off                             637            663          37         15.7          63.7       1.0X
-date_add wholestage on                              659            675          16         15.2          65.9       1.0X
+date_add wholestage off                             688            695          10         14.5          68.8       1.0X
+date_add wholestage on                              588            603          13         17.0          58.8       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_sub:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_sub wholestage off                             630            682          73         15.9          63.0       1.0X
-date_sub wholestage on                              720            762          37         13.9          72.0       0.9X
+date_sub wholestage off                             588            600          18         17.0          58.8       1.0X
+date_sub wholestage on                              597            624          40         16.8          59.7       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 add_months:                               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-add_months wholestage off                          1064           1088          35          9.4         106.4       1.0X
-add_months wholestage on                            883            909          25         11.3          88.3       1.2X
+add_months wholestage off                           802            816          20         12.5          80.2       1.0X
+add_months wholestage on                            777            836          61         12.9          77.7       1.0X
 
 
 ================================================================================================
@@ -168,8 +168,8 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 format date:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-format date wholestage off                         3667           3854         265          2.7         366.7       1.0X
-format date wholestage on                          3355           3548         195          3.0         335.5       1.1X
+format date wholestage off                         3328           3635         435          3.0         332.8       1.0X
+format date wholestage on                          3335           3549         253          3.0         333.5       1.0X
 
 
 ================================================================================================
@@ -180,8 +180,8 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 from_unixtime:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-from_unixtime wholestage off                       4664           4704          57          2.1         466.4       1.0X
-from_unixtime wholestage on                        4596           4772         193          2.2         459.6       1.0X
+from_unixtime wholestage off                       4414           4548         189          2.3         441.4       1.0X
+from_unixtime wholestage on                        4455           4499          65          2.2         445.5       1.0X
 
 
 ================================================================================================
@@ -192,15 +192,15 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 from_utc_timestamp:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-from_utc_timestamp wholestage off                   810            819          13         12.3          81.0       1.0X
-from_utc_timestamp wholestage on                    795            814          16         12.6          79.5       1.0X
+from_utc_timestamp wholestage off                   781            790          13         12.8          78.1       1.0X
+from_utc_timestamp wholestage on                    819            823           5         12.2          81.9       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_utc_timestamp:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_utc_timestamp wholestage off                    1130           1146          22          8.8         113.0       1.0X
-to_utc_timestamp wholestage on                     1034           1063          40          9.7         103.4       1.1X
+to_utc_timestamp wholestage off                    1101           1108          10          9.1         110.1       1.0X
+to_utc_timestamp wholestage on                     1044           1055          10          9.6         104.4       1.1X
 
 
 ================================================================================================
@@ -211,29 +211,29 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 cast interval:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast interval wholestage off                        192            194           2         52.1          19.2       1.0X
-cast interval wholestage on                         172            180           5         58.2          17.2       1.1X
+cast interval wholestage off                        198            202           6         50.6          19.8       1.0X
+cast interval wholestage on                         181            193          14         55.2          18.1       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 datediff:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-datediff wholestage off                            1070           1082          17          9.3         107.0       1.0X
-datediff wholestage on                             1039           1059          13          9.6         103.9       1.0X
+datediff wholestage off                            1037           1043          10          9.6         103.7       1.0X
+datediff wholestage on                             1006           1037          21          9.9         100.6       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 months_between:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-months_between wholestage off                      3118           3137          27          3.2         311.8       1.0X
-months_between wholestage on                       3079           3122          48          3.2         307.9       1.0X
+months_between wholestage off                      2957           2972          21          3.4         295.7       1.0X
+months_between wholestage on                       2931           2952          16          3.4         293.1       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 window:                                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-window wholestage off                              1147           1157          14          0.9        1146.6       1.0X
-window wholestage on                              16698          17304         715          0.1       16698.2       0.1X
+window wholestage off                              1130           1147          24          0.9        1130.0       1.0X
+window wholestage on                              16253          17068         580          0.1       16253.2       0.1X
 
 
 ================================================================================================
@@ -244,134 +244,134 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc YEAR:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc YEAR wholestage off                     1478           1485           9          6.8         147.8       1.0X
-date_trunc YEAR wholestage on                      1458           1587         213          6.9         145.8       1.0X
+date_trunc YEAR wholestage off                     1361           1380          27          7.3         136.1       1.0X
+date_trunc YEAR wholestage on                      1346           1381          28          7.4         134.6       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc YYYY:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc YYYY wholestage off                     1467           1481          20          6.8         146.7       1.0X
-date_trunc YYYY wholestage on                      1457           1513          52          6.9         145.7       1.0X
+date_trunc YYYY wholestage off                     1352           1364          16          7.4         135.2       1.0X
+date_trunc YYYY wholestage on                      1391           1424          37          7.2         139.1       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc YY:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc YY wholestage off                       1551           1556           7          6.4         155.1       1.0X
-date_trunc YY wholestage on                        1468           1538          78          6.8         146.8       1.1X
+date_trunc YY wholestage off                       1346           1356          15          7.4         134.6       1.0X
+date_trunc YY wholestage on                        1355           1366           9          7.4         135.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MON:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MON wholestage off                      1481           1493          16          6.8         148.1       1.0X
-date_trunc MON wholestage on                       1476           1501          28          6.8         147.6       1.0X
+date_trunc MON wholestage off                      1355           1365          15          7.4         135.5       1.0X
+date_trunc MON wholestage on                       1388           1457          78          7.2         138.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MONTH:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MONTH wholestage off                    1470           1477          10          6.8         147.0       1.0X
-date_trunc MONTH wholestage on                     1449           1610         260          6.9         144.9       1.0X
+date_trunc MONTH wholestage off                    1336           1357          30          7.5         133.6       1.0X
+date_trunc MONTH wholestage on                     1361           1390          22          7.3         136.1       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MM:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MM wholestage off                       1487           1495          12          6.7         148.7       1.0X
-date_trunc MM wholestage on                        1467           1489          17          6.8         146.7       1.0X
+date_trunc MM wholestage off                       1350           1361          16          7.4         135.0       1.0X
+date_trunc MM wholestage on                        1370           1384          13          7.3         137.0       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc DAY:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc DAY wholestage off                      1321           1328          10          7.6         132.1       1.0X
-date_trunc DAY wholestage on                       1279           1306          28          7.8         127.9       1.0X
+date_trunc DAY wholestage off                      1328           1361          46          7.5         132.8       1.0X
+date_trunc DAY wholestage on                       1356           1432          69          7.4         135.6       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc DD:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc DD wholestage off                       1366           1368           3          7.3         136.6       1.0X
-date_trunc DD wholestage on                        1297           1314          18          7.7         129.7       1.1X
+date_trunc DD wholestage off                       1403           1445          59          7.1         140.3       1.0X
+date_trunc DD wholestage on                        1359           1405          49          7.4         135.9       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc HOUR:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc HOUR wholestage off                     1338           1354          23          7.5         133.8       1.0X
-date_trunc HOUR wholestage on                      1403           1482          71          7.1         140.3       1.0X
+date_trunc HOUR wholestage off                     1447           1494          67          6.9         144.7       1.0X
+date_trunc HOUR wholestage on                      1387           1416          31          7.2         138.7       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MINUTE:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MINUTE wholestage off                    241            242           2         41.5          24.1       1.0X
-date_trunc MINUTE wholestage on                     196            204           6         51.0          19.6       1.2X
+date_trunc MINUTE wholestage off                    206            215          13         48.5          20.6       1.0X
+date_trunc MINUTE wholestage on                     233            240           4         42.9          23.3       0.9X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc SECOND:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc SECOND wholestage off                    211            222          15         47.3          21.1       1.0X
-date_trunc SECOND wholestage on                     205            243          64         48.8          20.5       1.0X
+date_trunc SECOND wholestage off                    200            213          18         49.9          20.0       1.0X
+date_trunc SECOND wholestage on                     219            227           7         45.6          21.9       0.9X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc WEEK:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc WEEK wholestage off                     1492           1515          32          6.7         149.2       1.0X
-date_trunc WEEK wholestage on                      1465           1490          39          6.8         146.5       1.0X
+date_trunc WEEK wholestage off                     1287           1293           9          7.8         128.7       1.0X
+date_trunc WEEK wholestage on                      1310           1341          28          7.6         131.0       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc QUARTER:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc QUARTER wholestage off                  1908           1982         104          5.2         190.8       1.0X
-date_trunc QUARTER wholestage on                   1879           2020          99          5.3         187.9       1.0X
+date_trunc QUARTER wholestage off                  1852           1880          39          5.4         185.2       1.0X
+date_trunc QUARTER wholestage on                   1857           1877          14          5.4         185.7       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc year:                               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc year wholestage off                           326            419         132         30.7          32.6       1.0X
-trunc year wholestage on                            187            199           8         53.4          18.7       1.7X
+trunc year wholestage off                           172            178           9         58.2          17.2       1.0X
+trunc year wholestage on                            213            222           8         47.0          21.3       0.8X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc yyyy:                               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc yyyy wholestage off                           194            194           0         51.6          19.4       1.0X
-trunc yyyy wholestage on                            192            207          10         52.1          19.2       1.0X
+trunc yyyy wholestage off                           175            182           9         57.1          17.5       1.0X
+trunc yyyy wholestage on                            224            233          12         44.7          22.4       0.8X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc yy:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc yy wholestage off                             177            180           4         56.5          17.7       1.0X
-trunc yy wholestage on                              185            193           9         53.9          18.5       1.0X
+trunc yy wholestage off                             167            180          18         59.7          16.7       1.0X
+trunc yy wholestage on                              212            226           9         47.3          21.2       0.8X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc mon:                                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc mon wholestage off                            208            210           3         48.0          20.8       1.0X
-trunc mon wholestage on                             195            204           8         51.3          19.5       1.1X
+trunc mon wholestage off                            175            175           1         57.2          17.5       1.0X
+trunc mon wholestage on                             219            241          17         45.7          21.9       0.8X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc month:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc month wholestage off                          182            185           4         55.1          18.2       1.0X
-trunc month wholestage on                           185            222          70         54.0          18.5       1.0X
+trunc month wholestage off                          176            176           0         56.8          17.6       1.0X
+trunc month wholestage on                           220            226           6         45.4          22.0       0.8X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc mm:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc mm wholestage off                             208            218          14         48.0          20.8       1.0X
-trunc mm wholestage on                              197            206          10         50.8          19.7       1.1X
+trunc mm wholestage off                             173            173           1         57.9          17.3       1.0X
+trunc mm wholestage on                              219            230           8         45.7          21.9       0.8X
 
 
 ================================================================================================
@@ -382,36 +382,36 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to timestamp str:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to timestamp str wholestage off                     121            130          13          8.3         120.7       1.0X
-to timestamp str wholestage on                      128            134           6          7.8         128.4       0.9X
+to timestamp str wholestage off                      98            102           5         10.2          97.9       1.0X
+to timestamp str wholestage on                      102            112           5          9.8         102.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_timestamp:                             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_timestamp wholestage off                        1201           1272         100          0.8        1200.8       1.0X
-to_timestamp wholestage on                          973           1019          53          1.0         972.7       1.2X
+to_timestamp wholestage off                        1373           1380          11          0.7        1373.0       1.0X
+to_timestamp wholestage on                          928           1100         103          1.1         928.1       1.5X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_unix_timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_unix_timestamp wholestage off                   1002           1077         106          1.0        1001.8       1.0X
-to_unix_timestamp wholestage on                     988           1079         100          1.0         987.8       1.0X
+to_unix_timestamp wholestage off                   1416           1438          32          0.7        1416.0       1.0X
+to_unix_timestamp wholestage on                    1094           1261         194          0.9        1093.6       1.3X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to date str:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to date str wholestage off                          157            165          12          6.4         157.2       1.0X
-to date str wholestage on                           137            140           3          7.3         137.0       1.1X
+to date str wholestage off                          120            123           5          8.3         120.0       1.0X
+to date str wholestage on                           142            148           6          7.0         142.1       0.8X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_date:                                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_date wholestage off                             1597           1614          25          0.6        1597.0       1.0X
-to_date wholestage on                              1503           1573          58          0.7        1502.8       1.1X
+to_date wholestage off                             1661           1673          17          0.6        1661.0       1.0X
+to_date wholestage on                              1567           1617          55          0.6        1567.3       1.1X
 
 
 ================================================================================================
@@ -422,8 +422,8 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-From java.sql.Timestamp                             197            202           6         25.4          39.4       1.0X
-Collect longs                                       794            911         110          6.3         158.7       0.2X
-Collect timestamps                                 1030           1079          51          4.9         205.9       0.2X
+From java.sql.Timestamp                             216            234          16         23.1          43.2       1.0X
+Collect longs                                       913           1204         266          5.5         182.6       0.2X
+Collect timestamps                                 1392           1515         124          3.6         278.3       0.2X
 
 
diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt
index 2ea26c3..c77d076 100644
--- a/sql/core/benchmarks/DateTimeBenchmark-results.txt
+++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt
@@ -6,92 +6,92 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 cast to timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to timestamp wholestage off                    230            240          13         43.5          23.0       1.0X
-cast to timestamp wholestage on                     194            208          20         51.4          19.4       1.2X
+cast to timestamp wholestage off                    243            256          18         41.2          24.3       1.0X
+cast to timestamp wholestage on                     193            206          19         51.9          19.3       1.3X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 year of timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-year of timestamp wholestage off                    806            822          22         12.4          80.6       1.0X
-year of timestamp wholestage on                     748            761          13         13.4          74.8       1.1X
+year of timestamp wholestage off                    784            786           3         12.8          78.4       1.0X
+year of timestamp wholestage on                     738            758          15         13.5          73.8       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 quarter of timestamp:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-quarter of timestamp wholestage off                 828            832           5         12.1          82.8       1.0X
-quarter of timestamp wholestage on                  821            858          45         12.2          82.1       1.0X
+quarter of timestamp wholestage off                 823            825           3         12.2          82.3       1.0X
+quarter of timestamp wholestage on                  791            804          12         12.6          79.1       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 month of timestamp:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-month of timestamp wholestage off                   709            713           5         14.1          70.9       1.0X
-month of timestamp wholestage on                    714            722          10         14.0          71.4       1.0X
+month of timestamp wholestage off                   681            692          16         14.7          68.1       1.0X
+month of timestamp wholestage on                    683            703          14         14.6          68.3       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 weekofyear of timestamp:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-weekofyear of timestamp wholestage off             1217           1220           4          8.2         121.7       1.0X
-weekofyear of timestamp wholestage on              1019           1043          24          9.8         101.9       1.2X
+weekofyear of timestamp wholestage off             1015           1032          23          9.8         101.5       1.0X
+weekofyear of timestamp wholestage on              1016           1106          72          9.8         101.6       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 day of timestamp:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-day of timestamp wholestage off                     706            712           9         14.2          70.6       1.0X
-day of timestamp wholestage on                      694            704           8         14.4          69.4       1.0X
+day of timestamp wholestage off                     683            689           9         14.6          68.3       1.0X
+day of timestamp wholestage on                      685            702          11         14.6          68.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 dayofyear of timestamp:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-dayofyear of timestamp wholestage off               717            728          16         14.0          71.7       1.0X
-dayofyear of timestamp wholestage on                724            736          10         13.8          72.4       1.0X
+dayofyear of timestamp wholestage off               713            716           4         14.0          71.3       1.0X
+dayofyear of timestamp wholestage on                704            713           9         14.2          70.4       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 dayofmonth of timestamp:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-dayofmonth of timestamp wholestage off              719            721           3         13.9          71.9       1.0X
-dayofmonth of timestamp wholestage on               706            714           8         14.2          70.6       1.0X
+dayofmonth of timestamp wholestage off              738            746          11         13.5          73.8       1.0X
+dayofmonth of timestamp wholestage on               684            704          16         14.6          68.4       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 dayofweek of timestamp:                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-dayofweek of timestamp wholestage off               904            979         106         11.1          90.4       1.0X
-dayofweek of timestamp wholestage on                796            819          14         12.6          79.6       1.1X
+dayofweek of timestamp wholestage off               810            821          15         12.3          81.0       1.0X
+dayofweek of timestamp wholestage on                784            789           5         12.8          78.4       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 weekday of timestamp:                     Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-weekday of timestamp wholestage off                 813            819           9         12.3          81.3       1.0X
-weekday of timestamp wholestage on                  788            809          13         12.7          78.8       1.0X
+weekday of timestamp wholestage off                 767            777          13         13.0          76.7       1.0X
+weekday of timestamp wholestage on                  765            777          11         13.1          76.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 hour of timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-hour of timestamp wholestage off                    567            582          21         17.6          56.7       1.0X
-hour of timestamp wholestage on                     539            551          11         18.6          53.9       1.1X
+hour of timestamp wholestage off                    534            543          13         18.7          53.4       1.0X
+hour of timestamp wholestage on                     499            529          22         20.0          49.9       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 minute of timestamp:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-minute of timestamp wholestage off                  554            563          13         18.1          55.4       1.0X
-minute of timestamp wholestage on                   520            531           7         19.2          52.0       1.1X
+minute of timestamp wholestage off                  526            544          26         19.0          52.6       1.0X
+minute of timestamp wholestage on                   495            515          20         20.2          49.5       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 second of timestamp:                      Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-second of timestamp wholestage off                  651            654           5         15.4          65.1       1.0X
-second of timestamp wholestage on                   535            546          12         18.7          53.5       1.2X
+second of timestamp wholestage off                  537            538           2         18.6          53.7       1.0X
+second of timestamp wholestage on                   508            512           5         19.7          50.8       1.1X
 
 
 ================================================================================================
@@ -102,15 +102,15 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 current_date:                             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-current_date wholestage off                         172            175           4         58.0          17.2       1.0X
-current_date wholestage on                          174            177           3         57.5          17.4       1.0X
+current_date wholestage off                         164            169           6         60.8          16.4       1.0X
+current_date wholestage on                          165            171           4         60.4          16.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 current_timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-current_timestamp wholestage off                    184            185           0         54.3          18.4       1.0X
-current_timestamp wholestage on                     196            212          14         51.0          19.6       0.9X
+current_timestamp wholestage off                    161            168          10         62.0          16.1       1.0X
+current_timestamp wholestage on                     161            165           5         62.1          16.1       1.0X
 
 
 ================================================================================================
@@ -121,43 +121,43 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 cast to date:                             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to date wholestage off                         652            662          13         15.3          65.2       1.0X
-cast to date wholestage on                          645            690          29         15.5          64.5       1.0X
+cast to date wholestage off                         691            692           2         14.5          69.1       1.0X
+cast to date wholestage on                          572            586          16         17.5          57.2       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 last_day:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-last_day wholestage off                             801            808          11         12.5          80.1       1.0X
-last_day wholestage on                              765            829          37         13.1          76.5       1.0X
+last_day wholestage off                             713            723          14         14.0          71.3       1.0X
+last_day wholestage on                              695            747          41         14.4          69.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 next_day:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-next_day wholestage off                             676            676           0         14.8          67.6       1.0X
-next_day wholestage on                              674            708          33         14.8          67.4       1.0X
+next_day wholestage off                             627            630           4         15.9          62.7       1.0X
+next_day wholestage on                              615            660          39         16.3          61.5       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_add:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_add wholestage off                             644            646           2         15.5          64.4       1.0X
-date_add wholestage on                              640            661          32         15.6          64.0       1.0X
+date_add wholestage off                             604            649          63         16.5          60.4       1.0X
+date_add wholestage on                              590            598           8         17.0          59.0       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_sub:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_sub wholestage off                             704            718          20         14.2          70.4       1.0X
-date_sub wholestage on                              684            728          34         14.6          68.4       1.0X
+date_sub wholestage off                             588            592           5         17.0          58.8       1.0X
+date_sub wholestage on                              597            620          39         16.7          59.7       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 add_months:                               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-add_months wholestage off                          1011           1017           8          9.9         101.1       1.0X
-add_months wholestage on                            837            860          25         11.9          83.7       1.2X
+add_months wholestage off                           777            782           8         12.9          77.7       1.0X
+add_months wholestage on                            769            786          22         13.0          76.9       1.0X
 
 
 ================================================================================================
@@ -168,8 +168,8 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 format date:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-format date wholestage off                         3467           3591         176          2.9         346.7       1.0X
-format date wholestage on                          3417           3482          66          2.9         341.7       1.0X
+format date wholestage off                         3335           3498         231          3.0         333.5       1.0X
+format date wholestage on                          3386           3488         103          3.0         338.6       1.0X
 
 
 ================================================================================================
@@ -180,8 +180,8 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 from_unixtime:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-from_unixtime wholestage off                       4823           4850          38          2.1         482.3       1.0X
-from_unixtime wholestage on                        4774           4811          29          2.1         477.4       1.0X
+from_unixtime wholestage off                       5291           5601         438          1.9         529.1       1.0X
+from_unixtime wholestage on                        5684           5952         256          1.8         568.4       0.9X
 
 
 ================================================================================================
@@ -192,15 +192,15 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 from_utc_timestamp:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-from_utc_timestamp wholestage off                   703            717          20         14.2          70.3       1.0X
-from_utc_timestamp wholestage on                    665            671           7         15.0          66.5       1.1X
+from_utc_timestamp wholestage off                   963            981          25         10.4          96.3       1.0X
+from_utc_timestamp wholestage on                    731            794          40         13.7          73.1       1.3X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_utc_timestamp:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_utc_timestamp wholestage off                    1000           1016          22         10.0         100.0       1.0X
-to_utc_timestamp wholestage on                      917            933          11         10.9          91.7       1.1X
+to_utc_timestamp wholestage off                    1129           1135           7          8.9         112.9       1.0X
+to_utc_timestamp wholestage on                      921            973          68         10.9          92.1       1.2X
 
 
 ================================================================================================
@@ -211,29 +211,29 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 cast interval:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast interval wholestage off                        195            198           3         51.2          19.5       1.0X
-cast interval wholestage on                         176            181           5         56.9          17.6       1.1X
+cast interval wholestage off                        197            197           0         50.8          19.7       1.0X
+cast interval wholestage on                         172            179           8         58.3          17.2       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 datediff:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-datediff wholestage off                            1022           1025           5          9.8         102.2       1.0X
-datediff wholestage on                             1003           1015          10         10.0         100.3       1.0X
+datediff wholestage off                            1039           1105          93          9.6         103.9       1.0X
+datediff wholestage on                             1129           1162          22          8.9         112.9       0.9X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 months_between:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-months_between wholestage off                      2562           2579          24          3.9         256.2       1.0X
-months_between wholestage on                       2503           2524          17          4.0         250.3       1.0X
+months_between wholestage off                      2624           2654          42          3.8         262.4       1.0X
+months_between wholestage on                       2612           2709          57          3.8         261.2       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 window:                                   Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-window wholestage off                              1067           1077          15          0.9        1066.6       1.0X
-window wholestage on                              14503          19089         NaN          0.1       14502.6       0.1X
+window wholestage off                              1229           1247          25          0.8        1229.4       1.0X
+window wholestage on                              14409          15033         534          0.1       14409.2       0.1X
 
 
 ================================================================================================
@@ -244,134 +244,134 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc YEAR:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc YEAR wholestage off                     1497           1543          64          6.7         149.7       1.0X
-date_trunc YEAR wholestage on                      1412           1533         138          7.1         141.2       1.1X
+date_trunc YEAR wholestage off                     1203           1218          22          8.3         120.3       1.0X
+date_trunc YEAR wholestage on                      1162           1172          10          8.6         116.2       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc YYYY:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc YYYY wholestage off                     1725           1729           5          5.8         172.5       1.0X
-date_trunc YYYY wholestage on                      1413           1481          56          7.1         141.3       1.2X
+date_trunc YYYY wholestage off                     1185           1199          20          8.4         118.5       1.0X
+date_trunc YYYY wholestage on                      1148           1173          22          8.7         114.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc YY:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc YY wholestage off                       1456           1459           5          6.9         145.6       1.0X
-date_trunc YY wholestage on                        1369           1380           7          7.3         136.9       1.1X
+date_trunc YY wholestage off                       1189           1192           4          8.4         118.9       1.0X
+date_trunc YY wholestage on                        1168           1179          16          8.6         116.8       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MON:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MON wholestage off                      1459           1471          17          6.9         145.9       1.0X
-date_trunc MON wholestage on                       1359           1377          21          7.4         135.9       1.1X
+date_trunc MON wholestage off                      1308           1317          14          7.6         130.8       1.0X
+date_trunc MON wholestage on                       1303           1352          37          7.7         130.3       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MONTH:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MONTH wholestage off                    1455           1480          35          6.9         145.5       1.0X
-date_trunc MONTH wholestage on                     1344           1468         170          7.4         134.4       1.1X
+date_trunc MONTH wholestage off                    1301           1320          27          7.7         130.1       1.0X
+date_trunc MONTH wholestage on                     1171           1226          41          8.5         117.1       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MM:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MM wholestage off                       1437           1463          37          7.0         143.7       1.0X
-date_trunc MM wholestage on                        1273           1285           9          7.9         127.3       1.1X
+date_trunc MM wholestage off                       1237           1259          31          8.1         123.7       1.0X
+date_trunc MM wholestage on                        1181           1195          11          8.5         118.1       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc DAY:                           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc DAY wholestage off                      1198           1209          16          8.3         119.8       1.0X
-date_trunc DAY wholestage on                       1137           1192          70          8.8         113.7       1.1X
+date_trunc DAY wholestage off                      1364           1370           8          7.3         136.4       1.0X
+date_trunc DAY wholestage on                       1189           1223          31          8.4         118.9       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc DD:                            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc DD wholestage off                       1201           1215          20          8.3         120.1       1.0X
-date_trunc DD wholestage on                        1132           1144          14          8.8         113.2       1.1X
+date_trunc DD wholestage off                       1297           1328          45          7.7         129.7       1.0X
+date_trunc DD wholestage on                        1171           1242          43          8.5         117.1       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc HOUR:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc HOUR wholestage off                     1236           1238           3          8.1         123.6       1.0X
-date_trunc HOUR wholestage on                      1134           1150          15          8.8         113.4       1.1X
+date_trunc HOUR wholestage off                     1261           1317          79          7.9         126.1       1.0X
+date_trunc HOUR wholestage on                      1171           1225          48          8.5         117.1       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc MINUTE:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc MINUTE wholestage off                    206            207           2         48.6          20.6       1.0X
-date_trunc MINUTE wholestage on                     185            186           2         54.0          18.5       1.1X
+date_trunc MINUTE wholestage off                    210            211           2         47.7          21.0       1.0X
+date_trunc MINUTE wholestage on                     178            200          18         56.2          17.8       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc SECOND:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc SECOND wholestage off                    209            210           2         47.9          20.9       1.0X
-date_trunc SECOND wholestage on                     174            185           7         57.4          17.4       1.2X
+date_trunc SECOND wholestage off                    206            211           7         48.5          20.6       1.0X
+date_trunc SECOND wholestage on                     186            200           9         53.9          18.6       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc WEEK:                          Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc WEEK wholestage off                     1231           1293          86          8.1         123.1       1.0X
-date_trunc WEEK wholestage on                      1186           1219          35          8.4         118.6       1.0X
+date_trunc WEEK wholestage off                     1275           1307          46          7.8         127.5       1.0X
+date_trunc WEEK wholestage on                      1190           1244          63          8.4         119.0       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 date_trunc QUARTER:                       Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-date_trunc QUARTER wholestage off                  1785           1792          10          5.6         178.5       1.0X
-date_trunc QUARTER wholestage on                   1671           1708          28          6.0         167.1       1.1X
+date_trunc QUARTER wholestage off                  1688           1719          44          5.9         168.8       1.0X
+date_trunc QUARTER wholestage on                   1732           1795          45          5.8         173.2       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc year:                               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc year wholestage off                           186            199          18         53.8          18.6       1.0X
-trunc year wholestage on                            161            165           5         62.3          16.1       1.2X
+trunc year wholestage off                           201            214          19         49.7          20.1       1.0X
+trunc year wholestage on                            150            157          11         66.5          15.0       1.3X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc yyyy:                               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc yyyy wholestage off                           184            184           0         54.4          18.4       1.0X
-trunc yyyy wholestage on                            156            162           6         64.3          15.6       1.2X
+trunc yyyy wholestage off                           169            174           6         59.0          16.9       1.0X
+trunc yyyy wholestage on                            159            174          10         62.9          15.9       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc yy:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc yy wholestage off                             171            174           5         58.6          17.1       1.0X
-trunc yy wholestage on                              161            172          10         62.1          16.1       1.1X
+trunc yy wholestage off                             171            179          12         58.5          17.1       1.0X
+trunc yy wholestage on                              170            180           9         58.8          17.0       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc mon:                                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc mon wholestage off                            175            176           1         57.1          17.5       1.0X
-trunc mon wholestage on                             166            167           1         60.3          16.6       1.1X
+trunc mon wholestage off                            174            177           4         57.5          17.4       1.0X
+trunc mon wholestage on                             162            171           7         61.7          16.2       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc month:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc month wholestage off                          177            186          12         56.4          17.7       1.0X
-trunc month wholestage on                           160            168           8         62.4          16.0       1.1X
+trunc month wholestage off                          195            196           2         51.2          19.5       1.0X
+trunc month wholestage on                           166            176           8         60.2          16.6       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 trunc mm:                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-trunc mm wholestage off                             173            175           2         57.8          17.3       1.0X
-trunc mm wholestage on                              167            178          10         59.9          16.7       1.0X
+trunc mm wholestage off                             156            160           6         64.2          15.6       1.0X
+trunc mm wholestage on                              147            162          13         68.1          14.7       1.1X
 
 
 ================================================================================================
@@ -382,36 +382,36 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to timestamp str:                         Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to timestamp str wholestage off                     166            168           3          6.0         166.3       1.0X
-to timestamp str wholestage on                      170            192          37          5.9         169.8       1.0X
+to timestamp str wholestage off                     154            154           1          6.5         153.8       1.0X
+to timestamp str wholestage on                      127            132           6          7.9         126.5       1.2X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_timestamp:                             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_timestamp wholestage off                         975           1042          95          1.0         975.1       1.0X
-to_timestamp wholestage on                          976            982           5          1.0         976.0       1.0X
+to_timestamp wholestage off                         973           1001          40          1.0         972.6       1.0X
+to_timestamp wholestage on                          912            922          12          1.1         912.3       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_unix_timestamp:                        Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_unix_timestamp wholestage off                    978            984           8          1.0         978.4       1.0X
-to_unix_timestamp wholestage on                     956            979          20          1.0         955.8       1.0X
+to_unix_timestamp wholestage off                    970            986          23          1.0         969.7       1.0X
+to_unix_timestamp wholestage on                     907            910           3          1.1         906.9       1.1X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to date str:                              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to date str wholestage off                          169            173           5          5.9         168.9       1.0X
-to date str wholestage on                           155            159           4          6.4         155.4       1.1X
+to date str wholestage off                          154            155           1          6.5         154.3       1.0X
+to date str wholestage on                           156            165           6          6.4         155.9       1.0X
 
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 to_date:                                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-to_date wholestage off                             1583           1594          15          0.6        1583.2       1.0X
-to_date wholestage on                              1586           1635          52          0.6        1586.1       1.0X
+to_date wholestage off                             1757           1760           4          0.6        1756.6       1.0X
+to_date wholestage on                              1563           1626          41          0.6        1562.7       1.1X
 
 
 ================================================================================================
@@ -422,8 +422,8 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.3
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 To/from java.sql.Timestamp:               Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-From java.sql.Timestamp                             174            176           3         28.8          34.7       1.0X
-Collect longs                                       945           1275         538          5.3         189.0       0.2X
-Collect timestamps                                 1019           1422         633          4.9         203.8       0.2X
+From java.sql.Timestamp                             182            186           4         27.5          36.4       1.0X
+Collect longs                                      1041           1652         964          4.8         208.2       0.2X
+Collect timestamps                                 1003           1029          31          5.0         200.7       0.2X
 
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
index b0ec24e..292ac6d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
@@ -131,7 +131,10 @@ object PartitioningUtils {
     }
 
     val dateFormatter = DateFormatter(zoneId)
-    val timestampFormatter = TimestampFormatter(timestampPartitionPattern, zoneId)
+    val timestampFormatter = TimestampFormatter(
+      timestampPartitionPattern,
+      zoneId,
+      needVarLengthSecondFraction = true)
     // First, we need to parse every partition's path and see if we can find partition values.
     val (partitionValues, optDiscoveredBasePaths) = paths.map { path =>
       parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes,
diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
index b14778b..9026427 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
@@ -75,3 +75,28 @@ select date '2001-09-28' - null;
 -- subtract dates
 select null - date '2019-10-06';
 select date '2001-10-01' - date '2001-09-28';
+
+-- variable-length tests
+select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+-- exceeded max variable length
+select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+-- special cases
+select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]');
+select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]');
+select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]');
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]');
+select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]');
+select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS");
+select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm");
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm");
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm");
+select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm");
diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
index a7b098d..7258d00 100755
--- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 47
+-- Number of queries: 68
 
 
 -- !query
@@ -391,3 +391,171 @@ select date '2001-10-01' - date '2001-09-28'
 struct<subtractdates(DATE '2001-10-01', DATE '2001-09-28'):interval>
 -- !query output
 3 days
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.1
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 03:11:12.123
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 08:11:12.12345
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.123456
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.123456
+
+
+-- !query
+select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.123
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'):timestamp>
+-- !query output
+2019-10-06 10:11:12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'):timestamp>
+-- !query output
+2019-10-06 10:11:12.12
+
+
+-- !query
+select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]')
+-- !query schema
+struct<to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'):timestamp>
+-- !query output
+2019-10-06 10:11:00
+
+
+-- !query
+select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS")
+-- !query schema
+struct<to_timestamp('2019-10-06S10:11:12.12345', 'yyyy-MM-dd\'S\'HH:mm:ss.SSSSSS'):timestamp>
+-- !query output
+2019-10-06 10:11:12.12345
+
+
+-- !query
+select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
+-- !query schema
+struct<to_timestamp('12.12342019-10-06S10:11', 'ss.SSSSyyyy-MM-dd\'S\'HH:mm'):timestamp>
+-- !query output
+2019-10-06 10:11:12.1234
+
+
+-- !query
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm")
+-- !query schema
+struct<to_timestamp('12.1232019-10-06S10:11', 'ss.SSSSyyyy-MM-dd\'S\'HH:mm'):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm")
+-- !query schema
+struct<to_timestamp('12.1232019-10-06S10:11', 'ss.SSSSyy-MM-dd\'S\'HH:mm'):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm")
+-- !query schema
+struct<to_timestamp('12.1234019-10-06S10:11', 'ss.SSSSy-MM-dd\'S\'HH:mm'):timestamp>
+-- !query output
+0019-10-06 10:11:12.1234
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala
index 8de9b8d..d5ab1b9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala
@@ -114,7 +114,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark {
     }
     runBenchmark("Parsing") {
       val n = 1000000
-      val timestampStrExpr = "concat('2019-01-27 11:02:01.', rpad(mod(id, 1000), 3, '0'))"
+      val timestampStrExpr = "concat('2019-01-27 11:02:01.', cast(mod(id, 1000) as string))"
       val pattern = "'yyyy-MM-dd HH:mm:ss.SSS'"
       run(n, "to timestamp str", timestampStrExpr)
       run(n, "to_timestamp", s"to_timestamp($timestampStrExpr, $pattern)")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
index e639294..f4499fe 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
@@ -59,7 +59,8 @@ abstract class ParquetPartitionDiscoverySuite
 
   val timeZoneId = ZoneId.systemDefault()
   val df = DateFormatter(timeZoneId)
-  val tf = TimestampFormatter(timestampPartitionPattern, timeZoneId)
+  val tf = TimestampFormatter(
+    timestampPartitionPattern, timeZoneId, needVarLengthSecondFraction = true)
 
   protected override def beforeAll(): Unit = {
     super.beforeAll()


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