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/04/22 10:26:18 UTC

[spark] branch branch-3.0 updated: [SPARK-31507][SQL] Remove uncommon fields support and update some fields with meaningful names for extract function

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 004952d  [SPARK-31507][SQL] Remove uncommon fields support  and update some fields with meaningful names for extract function
004952d is described below

commit 004952d26139cee9c03bf5300fc815940efb79c3
Author: Kent Yao <ya...@hotmail.com>
AuthorDate: Wed Apr 22 10:24:49 2020 +0000

    [SPARK-31507][SQL] Remove uncommon fields support  and update some fields with meaningful names for extract function
    
    ### What changes were proposed in this pull request?
    
    Extracting millennium, century, decade, millisecond, microsecond and epoch from datetime is neither ANSI standard nor quite common in modern SQL platforms. Most of the systems listing below does not support these except PostgreSQL and redshift.
    
    https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF
    
    https://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
    
    https://prestodb.io/docs/current/functions/datetime.html
    
    https://docs.cloudera.com/documentation/enterprise/5-8-x/topics/impala_datetime_functions.html
    
    https://docs.snowflake.com/en/sql-reference/functions-date-time.html#label-supported-date-time-parts
    
    https://www.postgresql.org/docs/9.1/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT
    
    This PR removes these extract fields support from extract function for date and timestamp values
    
    `isoyear` is PostgreSQL specific but `yearofweek` is more commonly used across platforms
    `isodow` is PostgreSQL specific but `iso` as a suffix is more commonly used across platforms so, `dow_iso` and `dayofweek_iso` is used to replace it.
    
    For historical reasons, we have [`dayofweek`, `dow`] implemented for representing a non-ISO day-of-week and a newly added `isodow` from PostgreSQL for ISO day-of-week. Many other systems only have one week-numbering system support and use either full names or abbreviations. Things in spark become a little bit complicated.
    1. because of the existence of `isodow`, so we need to add iso-prefix to `dayofweek` to make a pair for it too. [`dayofweek`, `isodayofweek`, `dow` and `isodow`]
    2. because there are rare `iso`-prefixed systems and more systems choose `iso`-suffixed way, so we may result in [`dayofweek`, `dayofweekiso`, `dow`, `dowiso`]
    3. `dayofweekiso` looks nice and has use cases in the platforms listed above, e.g. snowflake, but `dowiso` looks weird and no use cases found.
    4. with a discussion the community,we have agreed with an underscore before `iso` may look much better because `isodow` is new and there is no standard for `iso` kind of things, so this may be good for us to make it simple and clear for end-users if they are well documented too.
    
    Thus, we finally result in [`dayofweek`, `dow`] for Non-ISO day-of-week system and [`dayofweek_iso`, `dow_iso`] for ISO system
    
    ### Why are the changes needed?
    
    Remove some nonstandard and uncommon features as we can add them back if necessary
    
    ### Does this PR introduce any user-facing change?
    
    NO, we should target this to 3.0.0 and these are added during 3.0.0
    
    ### How was this patch tested?
    
    Remove unused tests
    
    Closes #28284 from yaooqinn/SPARK-31507.
    
    Authored-by: Kent Yao <ya...@hotmail.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit 37d2e037ed804a414ed874c829e0139a277a5ae8)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../catalyst/expressions/datetimeExpressions.scala | 152 +----
 .../spark/sql/catalyst/util/DateTimeUtils.scala    |  39 +-
 .../expressions/DateExpressionsSuite.scala         |  89 +--
 .../benchmarks/ExtractBenchmark-jdk11-results.txt  | 168 +++---
 sql/core/benchmarks/ExtractBenchmark-results.txt   | 180 +++---
 .../test/resources/sql-tests/inputs/date_part.sql  | 111 ----
 .../test/resources/sql-tests/inputs/extract.sql    |  92 ++-
 .../resources/sql-tests/inputs/postgreSQL/date.sql |  76 +--
 .../sql-tests/inputs/postgreSQL/timestamp.sql      |  18 +-
 .../resources/sql-tests/results/date_part.sql.out  | 670 ---------------------
 .../resources/sql-tests/results/extract.sql.out    | 441 +++++++++-----
 .../sql-tests/results/postgreSQL/date.sql.out      | 306 +---------
 .../sql-tests/results/postgreSQL/timestamp.sql.out |  32 +-
 .../sql/execution/benchmark/ExtractBenchmark.scala |   8 +-
 14 files changed, 576 insertions(+), 1806 deletions(-)

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 63d6fcc..6ca6ab7 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
@@ -334,48 +334,6 @@ case class SecondWithFraction(child: Expression, timeZoneId: Option[String] = No
   }
 }
 
-case class Milliseconds(child: Expression, timeZoneId: Option[String] = None)
-  extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType)
-  // DecimalType is used here to not lose precision while converting microseconds to
-  // the fractional part of milliseconds. Scale 3 is taken to have all microseconds as
-  // the fraction. The precision 8 should cover 2 digits for seconds, 3 digits for
-  // milliseconds and 3 digits for microseconds.
-  override def dataType: DataType = DecimalType(8, 3)
-  override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
-    copy(timeZoneId = Option(timeZoneId))
-
-  override protected def nullSafeEval(timestamp: Any): Any = {
-    DateTimeUtils.getMilliseconds(timestamp.asInstanceOf[Long], zoneId)
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    defineCodeGen(ctx, ev, c => s"$dtu.getMilliseconds($c, $zid)")
-  }
-}
-
-case class Microseconds(child: Expression, timeZoneId: Option[String] = None)
-  extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType)
-  override def dataType: DataType = IntegerType
-  override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
-    copy(timeZoneId = Option(timeZoneId))
-
-  override protected def nullSafeEval(timestamp: Any): Any = {
-    DateTimeUtils.getMicroseconds(timestamp.asInstanceOf[Long], zoneId)
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    defineCodeGen(ctx, ev, c => s"$dtu.getMicroseconds($c, $zid)")
-  }
-}
-
 @ExpressionDescription(
   usage = "_FUNC_(date) - Returns the day of year of the date/timestamp.",
   examples = """
@@ -426,19 +384,19 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu
   }
 }
 
-case class IsoYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
+case class YearOfWeek(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
   override def inputTypes: Seq[AbstractDataType] = Seq(DateType)
 
   override def dataType: DataType = IntegerType
 
   override protected def nullSafeEval(date: Any): Any = {
-    DateTimeUtils.getIsoYear(date.asInstanceOf[Int])
+    DateTimeUtils.getWeekBasedYear(date.asInstanceOf[Int])
   }
 
   override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
     val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    defineCodeGen(ctx, ev, c => s"$dtu.getIsoYear($c)")
+    defineCodeGen(ctx, ev, c => s"$dtu.getWeekBasedYear($c)")
   }
 }
 
@@ -2032,108 +1990,26 @@ case class MakeTimestamp(
   override def prettyName: String = "make_timestamp"
 }
 
-case class Millennium(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(DateType)
-
-  override def dataType: DataType = IntegerType
-
-  override protected def nullSafeEval(date: Any): Any = {
-    DateTimeUtils.getMillennium(date.asInstanceOf[Int])
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    defineCodeGen(ctx, ev, c => s"$dtu.getMillennium($c)")
-  }
-}
-
-case class Century(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(DateType)
-
-  override def dataType: DataType = IntegerType
-
-  override protected def nullSafeEval(date: Any): Any = {
-    DateTimeUtils.getCentury(date.asInstanceOf[Int])
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    defineCodeGen(ctx, ev, c => s"$dtu.getCentury($c)")
-  }
-}
-
-case class Decade(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(DateType)
-
-  override def dataType: DataType = IntegerType
-
-  override protected def nullSafeEval(date: Any): Any = {
-    DateTimeUtils.getDecade(date.asInstanceOf[Int])
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    defineCodeGen(ctx, ev, c => s"$dtu.getDecade($c)")
-  }
-}
-
-case class Epoch(child: Expression, timeZoneId: Option[String] = None)
-    extends UnaryExpression with ImplicitCastInputTypes with TimeZoneAwareExpression {
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType)
-  // DecimalType is used to not lose precision while converting microseconds to
-  // the fractional part of seconds. Scale 6 is taken to have all microseconds as
-  // the fraction. The precision 20 should cover whole valid range of years [1, 9999]
-  // plus negative years that can be used in some cases though are not officially supported.
-  override def dataType: DataType = DecimalType(20, 6)
-  override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
-    copy(timeZoneId = Option(timeZoneId))
-
-  override protected def nullSafeEval(timestamp: Any): Any = {
-    DateTimeUtils.getEpoch(timestamp.asInstanceOf[Long], zoneId)
-  }
-
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName)
-    defineCodeGen(ctx, ev, c => s"$dtu.getEpoch($c, $zid)")
-  }
-}
-
 object DatePart {
 
   def parseExtractField(
       extractField: String,
       source: Expression,
       errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match {
-    case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source)
-    case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source)
-    case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source)
     case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source)
-    case "ISOYEAR" => IsoYear(source)
+    case "YEAROFWEEK" => YearOfWeek(source)
     case "QUARTER" | "QTR" => Quarter(source)
     case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source)
     case "WEEK" | "W" | "WEEKS" => WeekOfYear(source)
     case "DAY" | "D" | "DAYS" => DayOfMonth(source)
     case "DAYOFWEEK" | "DOW" => DayOfWeek(source)
-    case "ISODOW" => Add(WeekDay(source), Literal(1))
+    case "DAYOFWEEK_ISO" | "DOW_ISO" => Add(WeekDay(source), Literal(1))
     case "DOY" => DayOfYear(source)
     case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source)
     case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source)
     case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => SecondWithFraction(source)
-    case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" =>
-      Milliseconds(source)
-    case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" =>
-      Microseconds(source)
-    case "EPOCH" => Epoch(source)
     case _ => errorHandleFunc
   }
-}
-
-object DatePartLike {
 
   def toEquivalentExpr(field: Expression, source: Expression): Expression = {
     if (!field.foldable) {
@@ -2191,7 +2067,7 @@ case class DatePart(field: Expression, source: Expression, child: Expression)
   extends RuntimeReplaceable {
 
   def this(field: Expression, source: Expression) = {
-    this(field, source, DatePartLike.toEquivalentExpr(field, source))
+    this(field, source, DatePart.toEquivalentExpr(field, source))
   }
 
   override def flatArguments: Iterator[Any] = Iterator(field, source)
@@ -2205,26 +2081,20 @@ case class DatePart(field: Expression, source: Expression, child: Expression)
   arguments = """
     Arguments:
       * field - selects which part of the source should be extracted
-          - Supported string values of `field` for dates and timestamps are:
-              - "MILLENNIUM", ("MILLENNIA", "MIL", "MILS") - the conventional numbering of millennia
-              - "CENTURY", ("CENTURIES", "C", "CENT") - the conventional numbering of centuries
-              - "DECADE", ("DECADES", "DEC", "DECS") - the year field divided by 10
+          - Supported string values of `field` for dates and timestamps are(case insensitive):
               - "YEAR", ("Y", "YEARS", "YR", "YRS") - the year field
-              - "ISOYEAR" - the ISO 8601 week-numbering year that the datetime falls in
+              - "YEAROFWEEK" - the ISO 8601 week-numbering year that the datetime falls in. For example, 2005-01-02 is part of the 53rd week of year 2004, so the result is 2004
               - "QUARTER", ("QTR") - the quarter (1 - 4) of the year that the datetime falls in
               - "MONTH", ("MON", "MONS", "MONTHS") - the month field (1 - 12)
               - "WEEK", ("W", "WEEKS") - the number of the ISO 8601 week-of-week-based-year. A week is considered to start on a Monday and week 1 is the first week with >3 days. In the ISO week-numbering system, it is possible for early-January dates to be part of the 52nd or 53rd week of the previous year, and for late-December dates to be part of the first week of the next year. For example, 2005-01-02 is part of the 53rd week of year 2004, while 2012-12-31 is part of the first week of 2013
               - "DAY", ("D", "DAYS") - the day of the month field (1 - 31)
               - "DAYOFWEEK",("DOW") - the day of the week for datetime as Sunday(1) to Saturday(7)
-              - "ISODOW" - ISO 8601 based day of the week for datetime as Monday(1) to Sunday(7)
+              - "DAYOFWEEK_ISO",("DOW_ISO") - ISO 8601 based day of the week for datetime as Monday(1) to Sunday(7)
               - "DOY" - the day of the year (1 - 365/366)
               - "HOUR", ("H", "HOURS", "HR", "HRS") - The hour field (0 - 23)
               - "MINUTE", ("M", "MIN", "MINS", "MINUTES") - the minutes field (0 - 59)
               - "SECOND", ("S", "SEC", "SECONDS", "SECS") - the seconds field, including fractional parts
-              - "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS") - the seconds field, including fractional parts, multiplied by 1000. Note that this includes full seconds
-              - "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US") - The seconds field, including fractional parts, multiplied by 1000000. Note that this includes full seconds
-              - "EPOCH" - the number of seconds with fractional part in microsecond precision since 1970-01-01 00:00:00 local time (can be negative)
-          - Supported string values of `field` for interval(which consists of `months`, `days`, `microseconds`) are:
+          - Supported string values of `field` for interval(which consists of `months`, `days`, `microseconds`) are(case insensitive):
               - "YEAR", ("Y", "YEARS", "YR", "YRS") - the total `months` / 12
               - "MONTH", ("MON", "MONS", "MONTHS") - the total `months` % 12
               - "DAY", ("D", "DAYS") - the `days` part of interval
@@ -2257,7 +2127,7 @@ case class Extract(field: Expression, source: Expression, child: Expression)
   extends RuntimeReplaceable {
 
   def this(field: Expression, source: Expression) = {
-    this(field, source, DatePartLike.toEquivalentExpr(field, source))
+    this(field, source, DatePart.toEquivalentExpr(field, source))
   }
 
   override def flatArguments: Iterator[Any] = Iterator(field, source)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 8486bba..0e9a15b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -527,14 +527,6 @@ object DateTimeUtils {
   }
 
   /**
-   * Returns seconds, including fractional parts, multiplied by 1000. The timestamp
-   * is expressed in microseconds since the epoch.
-   */
-  def getMilliseconds(timestamp: SQLTimestamp, zoneId: ZoneId): Decimal = {
-    Decimal(getMicroseconds(timestamp, zoneId), 8, 3)
-  }
-
-  /**
    * Returns seconds, including fractional parts, multiplied by 1000000. The timestamp
    * is expressed in microseconds since the epoch.
    */
@@ -551,24 +543,6 @@ object DateTimeUtils {
     LocalDate.ofEpochDay(date).getDayOfYear
   }
 
-  private def extractFromYear(date: SQLDate, divider: Int): Int = {
-    val localDate = daysToLocalDate(date)
-    val yearOfEra = localDate.get(ChronoField.YEAR_OF_ERA)
-    var result = yearOfEra / divider
-    if ((yearOfEra % divider) != 0 || yearOfEra <= 1) result += 1
-    if (localDate.get(ChronoField.ERA) == 0) result = -result
-    result
-  }
-
-  /** Returns the millennium for the given date. The date is expressed in days since 1.1.1970. */
-  def getMillennium(date: SQLDate): Int = extractFromYear(date, 1000)
-
-  /** Returns the century for the given date. The date is expressed in days since 1.1.1970. */
-  def getCentury(date: SQLDate): Int = extractFromYear(date, 100)
-
-  /** Returns the decade for the given date. The date is expressed in days since 1.1.1970. */
-  def getDecade(date: SQLDate): Int = Math.floorDiv(getYear(date), 10)
-
   /**
    * Returns the year value for the given date. The date is expressed in days
    * since 1.1.1970.
@@ -581,7 +555,7 @@ object DateTimeUtils {
    * Returns the year which conforms to ISO 8601. Each ISO 8601 week-numbering
    * year begins with the Monday of the week containing the 4th of January.
    */
-  def getIsoYear(date: SQLDate): Int = {
+  def getWeekBasedYear(date: SQLDate): Int = {
     daysToLocalDate(date).get(IsoFields.WEEK_BASED_YEAR)
   }
 
@@ -869,17 +843,6 @@ object DateTimeUtils {
     convertTz(time, getZoneId(timeZone), ZoneOffset.UTC)
   }
 
-  /**
-   * Returns the number of seconds with fractional part in microsecond precision
-   * since 1970-01-01 00:00:00 local time.
-   */
-  def getEpoch(timestamp: SQLTimestamp, zoneId: ZoneId): Decimal = {
-    val offset = SECONDS.toMicros(
-      zoneId.getRules.getOffset(microsToInstant(timestamp)).getTotalSeconds)
-    val sinceEpoch = timestamp + offset
-    Decimal(sinceEpoch, 20, 6)
-  }
-
   def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now())
 
   def currentDate(zoneId: ZoneId): SQLDate = localDateToDays(LocalDate.now(zoneId))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 7f235d6..b354717 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -1032,94 +1032,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(makeTimestampExpr, Timestamp.valueOf("2019-08-12 00:00:58.000001"))
   }
 
-  test("millennium") {
-    val date = MakeDate(Literal(2019), Literal(1), Literal(1))
-    checkEvaluation(Millennium(date), 3)
-    checkEvaluation(Millennium(date.copy(year = Literal(2001))), 3)
-    checkEvaluation(Millennium(date.copy(year = Literal(2000))), 2)
-    checkEvaluation(Millennium(date.copy(year = Literal(1001), day = Literal(28))), 2)
-    checkEvaluation(Millennium(date.copy(year = Literal(1))), 1)
-    checkEvaluation(Millennium(date.copy(year = Literal(-1))), -1)
-    checkEvaluation(Millennium(date.copy(year = Literal(-100), month = Literal(12))), -1)
-    checkEvaluation(Millennium(date.copy(year = Literal(-2019))), -3)
-  }
-
-  test("century") {
-    val date = MakeDate(Literal(2019), Literal(1), Literal(1))
-    checkEvaluation(Century(date), 21)
-    checkEvaluation(Century(date.copy(year = Literal(2001))), 21)
-    checkEvaluation(Century(date.copy(year = Literal(2000))), 20)
-    checkEvaluation(Century(date.copy(year = Literal(1001), day = Literal(28))), 11)
-    checkEvaluation(Century(date.copy(year = Literal(1))), 1)
-    checkEvaluation(Century(date.copy(year = Literal(-1))), -1)
-    checkEvaluation(Century(date.copy(year = Literal(-100), month = Literal(12))), -2)
-    checkEvaluation(Century(date.copy(year = Literal(-2019))), -21)
-  }
-
-  test("decade") {
-    val date = MakeDate(Literal(2019), Literal(8), Literal(8))
-    checkEvaluation(Decade(date), 201)
-    checkEvaluation(Decade(date.copy(year = Literal(2011))), 201)
-    checkEvaluation(Decade(date.copy(year = Literal(2010))), 201)
-    checkEvaluation(Decade(date.copy(year = Literal(2009))), 200)
-    checkEvaluation(Decade(date.copy(year = Literal(10))), 1)
-    checkEvaluation(Decade(date.copy(year = Literal(1))), 0)
-    checkEvaluation(Decade(date.copy(year = Literal(-1))), -1)
-    checkEvaluation(Decade(date.copy(year = Literal(-10))), -1)
-    checkEvaluation(Decade(date.copy(year = Literal(-11))), -2)
-    checkEvaluation(Decade(date.copy(year = Literal(-2019))), -202)
-  }
-
-  test("milliseconds and microseconds") {
-    outstandingTimezonesIds.foreach { timezone =>
-      var timestamp = MakeTimestamp(Literal(2019), Literal(8), Literal(10),
-        Literal(0), Literal(0), Literal(Decimal(BigDecimal(10.123456789), 8, 6)),
-        Some(Literal(timezone)), Some(timezone))
-      def millis(ts: MakeTimestamp): Milliseconds = Milliseconds(timestamp, Some(timezone))
-      def micros(ts: MakeTimestamp): Microseconds = Microseconds(timestamp, Some(timezone))
-
-      checkEvaluation(millis(timestamp), Decimal(BigDecimal(10123.457), 8, 3))
-      checkEvaluation(
-        millis(timestamp.copy(year = Literal(10))),
-        Decimal(BigDecimal(10123.457), 8, 3))
-
-      checkEvaluation(micros(timestamp), 10123457)
-      checkEvaluation(
-        micros(timestamp.copy(year = Literal(10))),
-        10123457)
-
-      timestamp = timestamp.copy(sec = Literal(Decimal(0.0, 8, 6)))
-      checkEvaluation(millis(timestamp), Decimal(0, 8, 3))
-      checkEvaluation(micros(timestamp), 0)
-
-      timestamp = timestamp.copy(sec = Literal(Decimal(BigDecimal(59.999999), 8, 6)))
-      checkEvaluation(millis(timestamp), Decimal(BigDecimal(59999.999), 8, 3))
-      checkEvaluation(micros(timestamp), 59999999)
-
-      timestamp = timestamp.copy(sec = Literal(Decimal(BigDecimal(60.0), 8, 6)))
-      checkEvaluation(millis(timestamp), Decimal(0, 8, 3))
-      checkEvaluation(micros(timestamp), 0)
-    }
-  }
-
-  test("epoch") {
-    val zoneId = ZoneId.systemDefault()
-    val nanos = 123456000
-    val timestamp = Epoch(MakeTimestamp(
-      Literal(2019), Literal(8), Literal(9), Literal(0), Literal(0),
-      Literal(Decimal(nanos / NANOS_PER_SECOND.toDouble, 8, 6)),
-      Some(Literal(zoneId.getId))))
-    val instant = LocalDateTime.of(2019, 8, 9, 0, 0, 0, nanos)
-      .atZone(zoneId).toInstant
-    val expected = Decimal(BigDecimal(nanos) / NANOS_PER_SECOND +
-      instant.getEpochSecond +
-      zoneId.getRules.getOffset(instant).getTotalSeconds)
-    checkEvaluation(timestamp, expected)
-  }
-
   test("ISO 8601 week-numbering year") {
-    checkEvaluation(IsoYear(MakeDate(Literal(2006), Literal(1), Literal(1))), 2005)
-    checkEvaluation(IsoYear(MakeDate(Literal(2006), Literal(1), Literal(2))), 2006)
+    checkEvaluation(YearOfWeek(MakeDate(Literal(2006), Literal(1), Literal(1))), 2005)
+    checkEvaluation(YearOfWeek(MakeDate(Literal(2006), Literal(1), Literal(2))), 2006)
   }
 
   test("extract the seconds part with fraction from timestamps") {
diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt
index 41bb42c..77f1bdb 100644
--- a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt
+++ b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt
@@ -2,123 +2,103 @@ Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke extract for timestamp:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to timestamp                                   322            327           4         31.1          32.2       1.0X
-MILLENNIUM of timestamp                             834            841           8         12.0          83.4       0.4X
-CENTURY of timestamp                                828            841          15         12.1          82.8       0.4X
-DECADE of timestamp                                 813            825          13         12.3          81.3       0.4X
-YEAR of timestamp                                   800            817          18         12.5          80.0       0.4X
-ISOYEAR of timestamp                                882            889           6         11.3          88.2       0.4X
-QUARTER of timestamp                                900            908          12         11.1          90.0       0.4X
-MONTH of timestamp                                  809            816           8         12.4          80.9       0.4X
-WEEK of timestamp                                  1119           1123           4          8.9         111.9       0.3X
-DAY of timestamp                                    801            808           7         12.5          80.1       0.4X
-DAYOFWEEK of timestamp                              946            952           5         10.6          94.6       0.3X
-DOW of timestamp                                    939            945          10         10.6          93.9       0.3X
-ISODOW of timestamp                                 869            874           5         11.5          86.9       0.4X
-DOY of timestamp                                    822            835          14         12.2          82.2       0.4X
-HOUR of timestamp                                   633            647          12         15.8          63.3       0.5X
-MINUTE of timestamp                                 635            636           1         15.8          63.5       0.5X
-SECOND of timestamp                                 774            778           4         12.9          77.4       0.4X
-MILLISECONDS of timestamp                           735            743           7         13.6          73.5       0.4X
-MICROSECONDS of timestamp                           652            657           4         15.3          65.2       0.5X
-EPOCH of timestamp                                  794            800           5         12.6          79.4       0.4X
+cast to timestamp                                   333            345          13         30.0          33.3       1.0X
+YEAR of timestamp                                   826            840          16         12.1          82.6       0.4X
+YEAROFWEEK of timestamp                            1161           1168           7          8.6         116.1       0.3X
+QUARTER of timestamp                                961            966           6         10.4          96.1       0.3X
+MONTH of timestamp                                  831            838           6         12.0          83.1       0.4X
+WEEK of timestamp                                  1155           1159           4          8.7         115.5       0.3X
+DAY of timestamp                                    822            828           6         12.2          82.2       0.4X
+DAYOFWEEK of timestamp                              968            972           6         10.3          96.8       0.3X
+DOW of timestamp                                    960            966           9         10.4          96.0       0.3X
+DOW_ISO of timestamp                                961            967           9         10.4          96.1       0.3X
+DAYOFWEEK_ISO of timestamp                          965            968           5         10.4          96.5       0.3X
+DOY of timestamp                                    864            873           9         11.6          86.4       0.4X
+HOUR of timestamp                                   631            647          14         15.9          63.1       0.5X
+MINUTE of timestamp                                 638            651          12         15.7          63.8       0.5X
+SECOND of timestamp                                 758            760           2         13.2          75.8       0.4X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke date_part for timestamp:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to timestamp                                   281            291          11         35.6          28.1       1.0X
-MILLENNIUM of timestamp                             814            825          19         12.3          81.4       0.3X
-CENTURY of timestamp                                842            850           7         11.9          84.2       0.3X
-DECADE of timestamp                                 812            814           3         12.3          81.2       0.3X
-YEAR of timestamp                                   787            798          10         12.7          78.7       0.4X
-ISOYEAR of timestamp                                956            962           6         10.5          95.6       0.3X
-QUARTER of timestamp                                948            954           6         10.6          94.8       0.3X
-MONTH of timestamp                                  797            817          26         12.5          79.7       0.4X
-WEEK of timestamp                                  1126           1134          10          8.9         112.6       0.2X
-DAY of timestamp                                    800            803           2         12.5          80.0       0.4X
-DAYOFWEEK of timestamp                              953            969          16         10.5          95.3       0.3X
-DOW of timestamp                                    978            986           9         10.2          97.8       0.3X
-ISODOW of timestamp                                 907            913           7         11.0          90.7       0.3X
-DOY of timestamp                                    831            844          14         12.0          83.1       0.3X
-HOUR of timestamp                                   646            650           3         15.5          64.6       0.4X
-MINUTE of timestamp                                 639            643           5         15.7          63.9       0.4X
-SECOND of timestamp                                 740            747           6         13.5          74.0       0.4X
-MILLISECONDS of timestamp                           745            752           7         13.4          74.5       0.4X
-MICROSECONDS of timestamp                           653            658           5         15.3          65.3       0.4X
-EPOCH of timestamp                                  724            738          12         13.8          72.4       0.4X
+cast to timestamp                                   301            307           8         33.2          30.1       1.0X
+YEAR of timestamp                                   797            805           7         12.6          79.7       0.4X
+YEAROFWEEK of timestamp                            1147           1156           8          8.7         114.7       0.3X
+QUARTER of timestamp                                965            974           9         10.4          96.5       0.3X
+MONTH of timestamp                                  801            815          17         12.5          80.1       0.4X
+WEEK of timestamp                                  1133           1157          21          8.8         113.3       0.3X
+DAY of timestamp                                    825            828           4         12.1          82.5       0.4X
+DAYOFWEEK of timestamp                              929            937           8         10.8          92.9       0.3X
+DOW of timestamp                                    933            940          11         10.7          93.3       0.3X
+DOW_ISO of timestamp                                944            954          11         10.6          94.4       0.3X
+DAYOFWEEK_ISO of timestamp                          948            965          21         10.5          94.8       0.3X
+DOY of timestamp                                    894            898           6         11.2          89.4       0.3X
+HOUR of timestamp                                   678            683           6         14.7          67.8       0.4X
+MINUTE of timestamp                                 669            670           2         15.0          66.9       0.5X
+SECOND of timestamp                                 762            771          12         13.1          76.2       0.4X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke extract for date:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to date                                        613            621           8         16.3          61.3       1.0X
-MILLENNIUM of date                                  738            741           3         13.6          73.8       0.8X
-CENTURY of date                                     735            737           3         13.6          73.5       0.8X
-DECADE of date                                      722            731           7         13.8          72.2       0.8X
-YEAR of date                                        715            719           4         14.0          71.5       0.9X
-ISOYEAR of date                                     881            883           2         11.4          88.1       0.7X
-QUARTER of date                                     857            867          10         11.7          85.7       0.7X
-MONTH of date                                       719            727           7         13.9          71.9       0.9X
-WEEK of date                                       1038           1043           8          9.6         103.8       0.6X
-DAY of date                                         715            719           4         14.0          71.5       0.9X
-DAYOFWEEK of date                                   861            873          12         11.6          86.1       0.7X
-DOW of date                                         847            865          16         11.8          84.7       0.7X
-ISODOW of date                                      788            796           7         12.7          78.8       0.8X
-DOY of date                                         744            747           4         13.4          74.4       0.8X
-HOUR of date                                       1485           1506          30          6.7         148.5       0.4X
-MINUTE of date                                     1502           1503           1          6.7         150.2       0.4X
-SECOND of date                                     1641           1650           9          6.1         164.1       0.4X
-MILLISECONDS of date                               1624           1639          14          6.2         162.4       0.4X
-MICROSECONDS of date                               1504           1505           2          6.6         150.4       0.4X
-EPOCH of date                                      1733           1748          19          5.8         173.3       0.4X
+cast to date                                        737            751          23         13.6          73.7       1.0X
+YEAR of date                                        804            811          11         12.4          80.4       0.9X
+YEAROFWEEK of date                                 1151           1155           4          8.7         115.1       0.6X
+QUARTER of date                                     964            987          26         10.4          96.4       0.8X
+MONTH of date                                       831            835           6         12.0          83.1       0.9X
+WEEK of date                                       1162           1163           2          8.6         116.2       0.6X
+DAY of date                                         826            833           7         12.1          82.6       0.9X
+DAYOFWEEK of date                                   984           1003          26         10.2          98.4       0.7X
+DOW of date                                         995            997           2         10.0          99.5       0.7X
+DOW_ISO of date                                     985            992           8         10.2          98.5       0.7X
+DAYOFWEEK_ISO of date                               985           1001          19         10.1          98.5       0.7X
+DOY of date                                         884            890           8         11.3          88.4       0.8X
+HOUR of date                                       1630           1640          13          6.1         163.0       0.5X
+MINUTE of date                                     1631           1636           6          6.1         163.1       0.5X
+SECOND of date                                     1737           1739           2          5.8         173.7       0.4X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke date_part for date:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to date                                        613            627          12         16.3          61.3       1.0X
-MILLENNIUM of date                                  722            734          11         13.9          72.2       0.8X
-CENTURY of date                                     740            746           8         13.5          74.0       0.8X
-DECADE of date                                      723            726           3         13.8          72.3       0.8X
-YEAR of date                                        705            717          11         14.2          70.5       0.9X
-ISOYEAR of date                                     876            885           9         11.4          87.6       0.7X
-QUARTER of date                                     857            877          21         11.7          85.7       0.7X
-MONTH of date                                       714            715           2         14.0          71.4       0.9X
-WEEK of date                                       1040           1043           4          9.6         104.0       0.6X
-DAY of date                                         703            714          10         14.2          70.3       0.9X
-DAYOFWEEK of date                                   861            869           7         11.6          86.1       0.7X
-DOW of date                                         856            863           8         11.7          85.6       0.7X
-ISODOW of date                                      791            797           5         12.6          79.1       0.8X
-DOY of date                                         721            728           6         13.9          72.1       0.9X
-HOUR of date                                       1488           1493           7          6.7         148.8       0.4X
-MINUTE of date                                     1488           1502          12          6.7         148.8       0.4X
-SECOND of date                                     1636           1651          14          6.1         163.6       0.4X
-MILLISECONDS of date                               1644           1665          34          6.1         164.4       0.4X
-MICROSECONDS of date                               1482           1516          42          6.7         148.2       0.4X
-EPOCH of date                                      1752           1790          34          5.7         175.2       0.4X
+cast to date                                        746            753           8         13.4          74.6       1.0X
+YEAR of date                                        841            843           1         11.9          84.1       0.9X
+YEAROFWEEK of date                                 1169           1191          20          8.6         116.9       0.6X
+QUARTER of date                                    1008           1012           4          9.9         100.8       0.7X
+MONTH of date                                       838            844           6         11.9          83.8       0.9X
+WEEK of date                                       1179           1182           3          8.5         117.9       0.6X
+DAY of date                                         849            850           1         11.8          84.9       0.9X
+DAYOFWEEK of date                                   984            988           4         10.2          98.4       0.8X
+DOW of date                                         986            994           7         10.1          98.6       0.8X
+DOW_ISO of date                                     974            992          17         10.3          97.4       0.8X
+DAYOFWEEK_ISO of date                               979            994          14         10.2          97.9       0.8X
+DOY of date                                         874            878           3         11.4          87.4       0.9X
+HOUR of date                                       1644           1649           6          6.1         164.4       0.5X
+MINUTE of date                                     1610           1626          15          6.2         161.0       0.5X
+SECOND of date                                     1768           1789          30          5.7         176.8       0.4X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke extract for interval:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to interval                                    921            929           9         10.9          92.1       1.0X
-YEAR of interval                                    927            942          22         10.8          92.7       1.0X
-MONTH of interval                                   956            974          22         10.5          95.6       1.0X
-DAY of interval                                     897            915          16         11.2          89.7       1.0X
-HOUR of interval                                    952            975          35         10.5          95.2       1.0X
-MINUTE of interval                                  938            955          14         10.7          93.8       1.0X
-SECOND of interval                                 1066           1080          24          9.4         106.6       0.9X
+cast to interval                                    989           1003          13         10.1          98.9       1.0X
+YEAR of interval                                    980            999          16         10.2          98.0       1.0X
+MONTH of interval                                   988           1000          15         10.1          98.8       1.0X
+DAY of interval                                     970            983          11         10.3          97.0       1.0X
+HOUR of interval                                    989           1000          13         10.1          98.9       1.0X
+MINUTE of interval                                 1019           1034          13          9.8         101.9       1.0X
+SECOND of interval                                 1084           1095          10          9.2         108.4       0.9X
 
 Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke date_part for interval:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to interval                                    944            960          21         10.6          94.4       1.0X
-YEAR of interval                                    978           1011          33         10.2          97.8       1.0X
-MONTH of interval                                   961            984          27         10.4          96.1       1.0X
-DAY of interval                                     921            927           7         10.9          92.1       1.0X
-HOUR of interval                                    913            919           8         11.0          91.3       1.0X
-MINUTE of interval                                  920            941          22         10.9          92.0       1.0X
-SECOND of interval                                  994           1012          19         10.1          99.4       0.9X
+cast to interval                                    955            969          15         10.5          95.5       1.0X
+YEAR of interval                                   1004           1007           3         10.0         100.4       1.0X
+MONTH of interval                                   989            990           2         10.1          98.9       1.0X
+DAY of interval                                    1006           1011           6          9.9         100.6       0.9X
+HOUR of interval                                    986            997          10         10.1          98.6       1.0X
+MINUTE of interval                                 1004           1014          15         10.0         100.4       1.0X
+SECOND of interval                                 1083           1088           6          9.2         108.3       0.9X
 
diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt
index bc766b4..11144a0 100644
--- a/sql/core/benchmarks/ExtractBenchmark-results.txt
+++ b/sql/core/benchmarks/ExtractBenchmark-results.txt
@@ -1,124 +1,104 @@
-Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke extract for timestamp:             Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to timestamp                                   306            337          27         32.7          30.6       1.0X
-MILLENNIUM of timestamp                            1002           1012          15         10.0         100.2       0.3X
-CENTURY of timestamp                               1055           1064           8          9.5         105.5       0.3X
-DECADE of timestamp                                 940            968          36         10.6          94.0       0.3X
-YEAR of timestamp                                   922            938          27         10.8          92.2       0.3X
-ISOYEAR of timestamp                               1036           1061          23          9.7         103.6       0.3X
-QUARTER of timestamp                               1081           1098          27          9.2         108.1       0.3X
-MONTH of timestamp                                  876            888          17         11.4          87.6       0.3X
-WEEK of timestamp                                  1218           1259          55          8.2         121.8       0.3X
-DAY of timestamp                                    890            911          33         11.2          89.0       0.3X
-DAYOFWEEK of timestamp                             1006           1019          17          9.9         100.6       0.3X
-DOW of timestamp                                    990           1006          22         10.1          99.0       0.3X
-ISODOW of timestamp                                 953            992          35         10.5          95.3       0.3X
-DOY of timestamp                                    883            889           6         11.3          88.3       0.3X
-HOUR of timestamp                                   661            667           6         15.1          66.1       0.5X
-MINUTE of timestamp                                 665            678          12         15.0          66.5       0.5X
-SECOND of timestamp                                 787            791           6         12.7          78.7       0.4X
-MILLISECONDS of timestamp                           800            814          12         12.5          80.0       0.4X
-MICROSECONDS of timestamp                           700            706           8         14.3          70.0       0.4X
-EPOCH of timestamp                                  793            819          40         12.6          79.3       0.4X
+cast to timestamp                                   292            310          16         34.3          29.2       1.0X
+YEAR of timestamp                                   847            875          26         11.8          84.7       0.3X
+YEAROFWEEK of timestamp                             964            981          24         10.4          96.4       0.3X
+QUARTER of timestamp                               1217           1219           2          8.2         121.7       0.2X
+MONTH of timestamp                                  835            844          10         12.0          83.5       0.3X
+WEEK of timestamp                                  1173           1183          15          8.5         117.3       0.2X
+DAY of timestamp                                    851            878          25         11.7          85.1       0.3X
+DAYOFWEEK of timestamp                              946            970          22         10.6          94.6       0.3X
+DOW of timestamp                                    935            959          21         10.7          93.5       0.3X
+DOW_ISO of timestamp                                947            961          13         10.6          94.7       0.3X
+DAYOFWEEK_ISO of timestamp                          965            992          26         10.4          96.5       0.3X
+DOY of timestamp                                    886            904          26         11.3          88.6       0.3X
+HOUR of timestamp                                   697            700           4         14.3          69.7       0.4X
+MINUTE of timestamp                                 654            665          10         15.3          65.4       0.4X
+SECOND of timestamp                                 770            778           8         13.0          77.0       0.4X
 
-Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke date_part for timestamp:           Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to timestamp                                   252            259           7         39.7          25.2       1.0X
-MILLENNIUM of timestamp                             917            936          18         10.9          91.7       0.3X
-CENTURY of timestamp                                943            952          12         10.6          94.3       0.3X
-DECADE of timestamp                                 863            867           5         11.6          86.3       0.3X
-YEAR of timestamp                                   839            854          16         11.9          83.9       0.3X
-ISOYEAR of timestamp                               1023           1027           4          9.8         102.3       0.2X
-QUARTER of timestamp                               1052           1060          10          9.5         105.2       0.2X
-MONTH of timestamp                                  835            846          14         12.0          83.5       0.3X
-WEEK of timestamp                                  1182           1198          14          8.5         118.2       0.2X
-DAY of timestamp                                    837            845          10         11.9          83.7       0.3X
-DAYOFWEEK of timestamp                              947            962          14         10.6          94.7       0.3X
-DOW of timestamp                                    946            964          17         10.6          94.6       0.3X
-ISODOW of timestamp                                 934            946          13         10.7          93.4       0.3X
-DOY of timestamp                                    857            859           3         11.7          85.7       0.3X
-HOUR of timestamp                                   660            661           1         15.1          66.0       0.4X
-MINUTE of timestamp                                 649            654           4         15.4          64.9       0.4X
-SECOND of timestamp                                 801            805           5         12.5          80.1       0.3X
-MILLISECONDS of timestamp                           820            825           7         12.2          82.0       0.3X
-MICROSECONDS of timestamp                           700            708          12         14.3          70.0       0.4X
-EPOCH of timestamp                                  789            797          10         12.7          78.9       0.3X
+cast to timestamp                                   233            243           9         43.0          23.3       1.0X
+YEAR of timestamp                                   810            826          15         12.3          81.0       0.3X
+YEAROFWEEK of timestamp                             996           1019          21         10.0          99.6       0.2X
+QUARTER of timestamp                               1037           1049          11          9.6         103.7       0.2X
+MONTH of timestamp                                  822            852          30         12.2          82.2       0.3X
+WEEK of timestamp                                  1179           1220          35          8.5         117.9       0.2X
+DAY of timestamp                                    822            825           3         12.2          82.2       0.3X
+DAYOFWEEK of timestamp                              937            941           3         10.7          93.7       0.2X
+DOW of timestamp                                    931            970          34         10.7          93.1       0.2X
+DOW_ISO of timestamp                                927            948          22         10.8          92.7       0.3X
+DAYOFWEEK_ISO of timestamp                          896            918          20         11.2          89.6       0.3X
+DOY of timestamp                                    863            891          25         11.6          86.3       0.3X
+HOUR of timestamp                                   639            645           6         15.7          63.9       0.4X
+MINUTE of timestamp                                 639            647          12         15.7          63.9       0.4X
+SECOND of timestamp                                 785            796          11         12.7          78.5       0.3X
 
-Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke extract for date:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to date                                        747            751           6         13.4          74.7       1.0X
-MILLENNIUM of date                                  941            947           5         10.6          94.1       0.8X
-CENTURY of date                                     952            961           8         10.5          95.2       0.8X
-DECADE of date                                      864            877          15         11.6          86.4       0.9X
-YEAR of date                                        850            862          11         11.8          85.0       0.9X
-ISOYEAR of date                                    1031           1036           8          9.7         103.1       0.7X
-QUARTER of date                                    1038           1048           9          9.6         103.8       0.7X
-MONTH of date                                       852            865          12         11.7          85.2       0.9X
-WEEK of date                                       1174           1178           5          8.5         117.4       0.6X
-DAY of date                                         830            836           5         12.0          83.0       0.9X
-DAYOFWEEK of date                                   949            972          21         10.5          94.9       0.8X
-DOW of date                                         960            969           8         10.4          96.0       0.8X
-ISODOW of date                                      918            928           9         10.9          91.8       0.8X
-DOY of date                                         858            867          11         11.7          85.8       0.9X
-HOUR of date                                       1584           1591          10          6.3         158.4       0.5X
-MINUTE of date                                     1583           1589           6          6.3         158.3       0.5X
-SECOND of date                                     1721           1728           7          5.8         172.1       0.4X
-MILLISECONDS of date                               1711           1731          17          5.8         171.1       0.4X
-MICROSECONDS of date                               1647           1656           8          6.1         164.7       0.5X
-EPOCH of date                                      1796           1805          10          5.6         179.6       0.4X
+cast to date                                        690            693           3         14.5          69.0       1.0X
+YEAR of date                                        822            841          17         12.2          82.2       0.8X
+YEAROFWEEK of date                                  967            974           7         10.3          96.7       0.7X
+QUARTER of date                                    1034           1044          11          9.7         103.4       0.7X
+MONTH of date                                       831            836           5         12.0          83.1       0.8X
+WEEK of date                                       1152           1177          34          8.7         115.2       0.6X
+DAY of date                                         836            873          34         12.0          83.6       0.8X
+DAYOFWEEK of date                                   960            992          30         10.4          96.0       0.7X
+DOW of date                                        1011           1016           4          9.9         101.1       0.7X
+DOW_ISO of date                                     969            984          16         10.3          96.9       0.7X
+DAYOFWEEK_ISO of date                               967            986          19         10.3          96.7       0.7X
+DOY of date                                         901            953          47         11.1          90.1       0.8X
+HOUR of date                                       1581           1586           5          6.3         158.1       0.4X
+MINUTE of date                                     1570           1584          13          6.4         157.0       0.4X
+SECOND of date                                     1713           1740          27          5.8         171.3       0.4X
 
-Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke date_part for date:                Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to date                                        731            740           8         13.7          73.1       1.0X
-MILLENNIUM of date                                  939            948          12         10.7          93.9       0.8X
-CENTURY of date                                     929            940          10         10.8          92.9       0.8X
-DECADE of date                                      845            861          15         11.8          84.5       0.9X
-YEAR of date                                        848            853           5         11.8          84.8       0.9X
-ISOYEAR of date                                    1011           1027          14          9.9         101.1       0.7X
-QUARTER of date                                    1038           1050          12          9.6         103.8       0.7X
-MONTH of date                                       847            872          22         11.8          84.7       0.9X
-WEEK of date                                       1143           1152          13          8.7         114.3       0.6X
-DAY of date                                         844            848           5         11.8          84.4       0.9X
-DAYOFWEEK of date                                   947            957           9         10.6          94.7       0.8X
-DOW of date                                         963            972           8         10.4          96.3       0.8X
-ISODOW of date                                      906            919          11         11.0          90.6       0.8X
-DOY of date                                         866            868           3         11.6          86.6       0.8X
-HOUR of date                                       1599           1604           7          6.3         159.9       0.5X
-MINUTE of date                                     1578           1594          13          6.3         157.8       0.5X
-SECOND of date                                     1722           1731          11          5.8         172.2       0.4X
-MILLISECONDS of date                               1699           1730          27          5.9         169.9       0.4X
-MICROSECONDS of date                               1620           1623           5          6.2         162.0       0.5X
-EPOCH of date                                      1742           1767          32          5.7         174.2       0.4X
+cast to date                                        756            762           6         13.2          75.6       1.0X
+YEAR of date                                        843            857          13         11.9          84.3       0.9X
+YEAROFWEEK of date                                 1055           1065          16          9.5         105.5       0.7X
+QUARTER of date                                    1066           1073           6          9.4         106.6       0.7X
+MONTH of date                                       856            890          44         11.7          85.6       0.9X
+WEEK of date                                       1155           1204          59          8.7         115.5       0.7X
+DAY of date                                         749            762          12         13.3          74.9       1.0X
+DAYOFWEEK of date                                   850            865          15         11.8          85.0       0.9X
+DOW of date                                         878            893          16         11.4          87.8       0.9X
+DOW_ISO of date                                     865            869           5         11.6          86.5       0.9X
+DAYOFWEEK_ISO of date                               914            967          76         10.9          91.4       0.8X
+DOY of date                                         789            792           4         12.7          78.9       1.0X
+HOUR of date                                       1558           1659         168          6.4         155.8       0.5X
+MINUTE of date                                     1581           1673          80          6.3         158.1       0.5X
+SECOND of date                                     1646           1881         319          6.1         164.6       0.5X
 
-Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke extract for interval:              Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to interval                                    920            941          18         10.9          92.0       1.0X
-YEAR of interval                                    955            967          12         10.5          95.5       1.0X
-MONTH of interval                                   954            966          11         10.5          95.4       1.0X
-DAY of interval                                     967            970           4         10.3          96.7       1.0X
-HOUR of interval                                    959            975          21         10.4          95.9       1.0X
-MINUTE of interval                                  966            981          16         10.4          96.6       1.0X
-SECOND of interval                                 1048           1066          16          9.5         104.8       0.9X
+cast to interval                                    925            941          20         10.8          92.5       1.0X
+YEAR of interval                                    903            919          14         11.1          90.3       1.0X
+MONTH of interval                                   944            958          17         10.6          94.4       1.0X
+DAY of interval                                     917            925           7         10.9          91.7       1.0X
+HOUR of interval                                    925            940          17         10.8          92.5       1.0X
+MINUTE of interval                                  951            962          12         10.5          95.1       1.0X
+SECOND of interval                                 1017           1036          19          9.8         101.7       0.9X
 
-Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.4
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_251-b08 on Mac OS X 10.15.4
 Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz
 Invoke date_part for interval:            Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------------------------------
-cast to interval                                    928            954          26         10.8          92.8       1.0X
-YEAR of interval                                    955            960           4         10.5          95.5       1.0X
-MONTH of interval                                   956            971          13         10.5          95.6       1.0X
-DAY of interval                                     964           1003          41         10.4          96.4       1.0X
-HOUR of interval                                   1057           1293         248          9.5         105.7       0.9X
-MINUTE of interval                                  975           1091         155         10.3          97.5       1.0X
-SECOND of interval                                 1104           1286         311          9.1         110.4       0.8X
+cast to interval                                    913            920           8         11.0          91.3       1.0X
+YEAR of interval                                    930            935           4         10.8          93.0       1.0X
+MONTH of interval                                   930            943          14         10.7          93.0       1.0X
+DAY of interval                                     933            946          12         10.7          93.3       1.0X
+HOUR of interval                                    951            953           3         10.5          95.1       1.0X
+MINUTE of interval                                  923            958          30         10.8          92.3       1.0X
+SECOND of interval                                  993            995           1         10.1          99.3       0.9X
 
diff --git a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql
deleted file mode 100644
index b36b98b..0000000
--- a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql
+++ /dev/null
@@ -1,111 +0,0 @@
-CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c;
-
-select date_part('millennium', c) from t;
-select date_part('millennia', c) from t;
-select date_part('mil', c) from t;
-select date_part('mils', c) from t;
-
-select date_part('century', c) from t;
-select date_part('centuries', c) from t;
-select date_part('c', c) from t;
-select date_part('cent', c) from t;
-
-select date_part('decade', c) from t;
-select date_part('decades', c) from t;
-select date_part('dec', c) from t;
-select date_part('decs', c) from t;
-
-select date_part('year', c) from t;
-select date_part('y', c) from t;
-select date_part('years', c) from t;
-select date_part('yr', c) from t;
-select date_part('yrs', c) from t;
-
-select date_part('quarter', c) from t;
-select date_part('qtr', c) from t;
-
-select date_part('month', c) from t;
-select date_part('mon', c) from t;
-select date_part('mons', c) from t;
-select date_part('months', c) from t;
-
-select date_part('week', c) from t;
-select date_part('w', c) from t;
-select date_part('weeks', c) from t;
-
-select date_part('day', c) from t;
-select date_part('d', c) from t;
-select date_part('days', c) from t;
-
-select date_part('dayofweek', c) from t;
-
-select date_part('dow', c) from t;
-
-select date_part('isodow', c) from t;
-
-select date_part('doy', c) from t;
-
-select date_part('hour', c) from t;
-select date_part('h', c) from t;
-select date_part('hours', c) from t;
-select date_part('hr', c) from t;
-select date_part('hrs', c) from t;
-
-select date_part('minute', c) from t;
-select date_part('m', c) from t;
-select date_part('min', c) from t;
-select date_part('mins', c) from t;
-select date_part('minutes', c) from t;
-
-select date_part('second', c) from t;
-select date_part('s', c) from t;
-select date_part('sec', c) from t;
-select date_part('seconds', c) from t;
-select date_part('secs', c) from t;
-
-select date_part('not_supported', c) from t;
-
-select date_part(c, c) from t;
-
-select date_part(null, c) from t;
-
-CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c;
-
-select date_part('year', c) from t2;
-select date_part('y', c) from t2;
-select date_part('years', c) from t2;
-select date_part('yr', c) from t2;
-select date_part('yrs', c) from t2;
-
-select date_part('month', c) from t2;
-select date_part('mon', c) from t2;
-select date_part('mons', c) from t2;
-select date_part('months', c) from t2;
-
-select date_part('day', c) from t2;
-select date_part('d', c) from t2;
-select date_part('days', c) from t2;
-
-select date_part('hour', c) from t2;
-select date_part('h', c) from t2;
-select date_part('hours', c) from t2;
-select date_part('hr', c) from t2;
-select date_part('hrs', c) from t2;
-
-select date_part('minute', c) from t2;
-select date_part('m', c) from t2;
-select date_part('min', c) from t2;
-select date_part('mins', c) from t2;
-select date_part('minutes', c) from t2;
-
-select date_part('second', c) from t2;
-select date_part('s', c) from t2;
-select date_part('sec', c) from t2;
-select date_part('seconds', c) from t2;
-select date_part('secs', c) from t2;
-
-select date_part('not_supported', c) from t2;
-
-select date_part(c, c) from t2;
-
-select date_part(null, c) from t2;
diff --git a/sql/core/src/test/resources/sql-tests/inputs/extract.sql b/sql/core/src/test/resources/sql-tests/inputs/extract.sql
index 12068d2..cba7fc2 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/extract.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/extract.sql
@@ -1,27 +1,12 @@
 CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, interval 10 year 20 month 30 day 40 hour 50 minute 6.7890 second as i;
 
-select extract(millennium from c) from t;
-select extract(millennia from c) from t;
-select extract(mil from c) from t;
-select extract(mils from c) from t;
-
-select extract(century from c) from t;
-select extract(centuries from c) from t;
-select extract(c from c) from t;
-select extract(cent from c)from t;
-
-select extract(decade from c) from t;
-select extract(decades from c) from t;
-select extract(dec from c) from t;
-select extract(decs from c) from t;
-
 select extract(year from c), extract(year from i) from t;
 select extract(y from c), extract(y from i) from t;
 select extract(years from c), extract(years from i) from t;
 select extract(yr from c), extract(yr from i) from t;
 select extract(yrs from c), extract(yrs from i) from t;
 
-select extract(isoyear from c) from t;
+select extract(yearofweek from c) from t;
 
 select extract(quarter from c) from t;
 select extract(qtr from c) from t;
@@ -40,10 +25,10 @@ select extract(d from c), extract(d from i) from t;
 select extract(days from c), extract(days from i) from t;
 
 select extract(dayofweek from c) from t;
-
 select extract(dow from c) from t;
 
-select extract(isodow from c) from t;
+select extract(dayofweek_iso from c) from t;
+select extract(dow_iso from c) from t;
 
 select extract(doy from c) from t;
 
@@ -65,21 +50,62 @@ select extract(sec from c), extract(sec from i) from t;
 select extract(seconds from c), extract(seconds from i) from t;
 select extract(secs from c), extract(secs from i) from t;
 
-select extract(milliseconds from c) from t;
-select extract(msec from c) from t;
-select extract(msecs from c) from t;
-select extract(millisecon from c) from t;
-select extract(mseconds from c) from t;
-select extract(ms from c) from t;
+select extract(not_supported from c) from t;
+select extract(not_supported from i) from t;
+
+select date_part('year', c), date_part('year', i) from t;
+select date_part('y', c), date_part('y', i) from t;
+select date_part('years', c), date_part('years', i) from t;
+select date_part('yr', c), date_part('yr', i) from t;
+select date_part('yrs', c), date_part('yrs', i) from t;
 
-select extract(microseconds from c) from t;
-select extract(usec from c) from t;
-select extract(usecs from c) from t;
-select extract(useconds from c) from t;
-select extract(microsecon from c) from t;
-select extract(us from c) from t;
+select date_part('yearofweek', c) from t;
 
-select extract(epoch from c) from t;
+select date_part('quarter', c) from t;
+select date_part('qtr', c) from t;
 
-select extract(not_supported from c) from t;
-select extract(not_supported from i) from t;
+select date_part('month', c), date_part('month', i) from t;
+select date_part('mon', c), date_part('mon', i) from t;
+select date_part('mons', c), date_part('mons', i) from t;
+select date_part('months', c), date_part('months', i) from t;
+
+select date_part('week', c) from t;
+select date_part('w', c) from t;
+select date_part('weeks', c) from t;
+
+select date_part('day', c), date_part('day', i) from t;
+select date_part('d', c), date_part('d', i) from t;
+select date_part('days', c), date_part('days', i) from t;
+
+select date_part('dayofweek', c) from t;
+select date_part('dow', c) from t;
+
+select date_part('dayofweek_iso', c) from t;
+select date_part('dow_iso', c) from t;
+
+select date_part('doy', c) from t;
+
+select date_part('hour', c), date_part('hour', i) from t;
+select date_part('h', c), date_part('h', i) from t;
+select date_part('hours', c), date_part('hours', i) from t;
+select date_part('hr', c), date_part('hr', i) from t;
+select date_part('hrs', c), date_part('hrs', i) from t;
+
+select date_part('minute', c), date_part('minute', i) from t;
+select date_part('m', c), date_part('m', i) from t;
+select date_part('min', c), date_part('min', i) from t;
+select date_part('mins', c), date_part('mins', i) from t;
+select date_part('minutes', c), date_part('minutes', i) from t;
+
+select date_part('second', c), date_part('second', i) from t;
+select date_part('s', c), date_part('s', i) from t;
+select date_part('sec', c), date_part('sec', i) from t;
+select date_part('seconds', c), date_part('seconds', i) from t;
+select date_part('secs', c), date_part('secs', i) from t;
+
+select date_part('not_supported', c) from t;
+select date_part(c, c) from t;
+select date_part(null, c) from t;
+
+select date_part(i, i) from t;
+select date_part(null, i) from t;
\ No newline at end of file
diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql
index 0bab2f8..6985108 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql
@@ -230,49 +230,49 @@ SELECT date 'tomorrow' - date 'yesterday' AS `Two days`;
 --
 -- epoch
 --
-SELECT EXTRACT(EPOCH FROM DATE        '1970-01-01');     --  0
-SELECT EXTRACT(EPOCH FROM TIMESTAMP   '1970-01-01');     --  0
+-- SELECT EXTRACT(EPOCH FROM DATE        '1970-01-01');     --  0
+-- SELECT EXTRACT(EPOCH FROM TIMESTAMP   '1970-01-01');     --  0
 -- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '1970-01-01+00');  --  0
 --
 -- century
 --
-SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')); -- -2
-SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')); -- -1
-SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1
-SELECT EXTRACT(CENTURY FROM DATE '0001-01-01');    --  1
-SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); --  1
-SELECT EXTRACT(CENTURY FROM DATE '1900-12-31');    -- 19
-SELECT EXTRACT(CENTURY FROM DATE '1901-01-01');    -- 20
-SELECT EXTRACT(CENTURY FROM DATE '2000-12-31');    -- 20
-SELECT EXTRACT(CENTURY FROM DATE '2001-01-01');    -- 21
-SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True;     -- true
+-- SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')); -- -2
+-- SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')); -- -1
+-- SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1
+-- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01');    --  1
+-- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); --  1
+-- SELECT EXTRACT(CENTURY FROM DATE '1900-12-31');    -- 19
+-- SELECT EXTRACT(CENTURY FROM DATE '1901-01-01');    -- 20
+-- SELECT EXTRACT(CENTURY FROM DATE '2000-12-31');    -- 20
+-- SELECT EXTRACT(CENTURY FROM DATE '2001-01-01');    -- 21
+-- SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True;     -- true
 --
 -- millennium
 --
-SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1
-SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); --  1
-SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31');    --  1
-SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01');    --  2
-SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31');    --  2
-SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01');    --  3
+-- SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1
+-- SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); --  1
+-- SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31');    --  1
+-- SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01');    --  2
+-- SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31');    --  2
+-- SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01');    --  3
 -- next test to be fixed on the turn of the next millennium;-)
-SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE);         --  3
+-- SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE);         --  3
 --
 -- decade
 --
-SELECT EXTRACT(DECADE FROM DATE '1994-12-25');    -- 199
-SELECT EXTRACT(DECADE FROM DATE '0010-01-01');    --   1
-SELECT EXTRACT(DECADE FROM DATE '0009-12-31');    --   0
-SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')); --   0
-SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); --  -1
-SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')); --  -1
-SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')); --  -2
+-- SELECT EXTRACT(DECADE FROM DATE '1994-12-25');    -- 199
+-- SELECT EXTRACT(DECADE FROM DATE '0010-01-01');    --   1
+-- SELECT EXTRACT(DECADE FROM DATE '0009-12-31');    --   0
+-- SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')); --   0
+-- SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); --  -1
+-- SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')); --  -1
+-- SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')); --  -2
 --
 -- some other types:
 --
 -- on a timestamp.
-SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True;       -- true
-SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20
+-- SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True;       -- true
+-- SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20
 -- on an interval
 -- SELECT EXTRACT(CENTURY FROM INTERVAL '100 y');  -- 1
 -- SELECT EXTRACT(CENTURY FROM INTERVAL '99 y');   -- 0
@@ -280,16 +280,16 @@ SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20
 -- SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); -- -1
 --
 -- test trunc function!
-SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001
-SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01
-SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901
-SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901
-SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01
-SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01
-SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')); -- 0100-01-01 BC
-SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01
-SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC
-SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-01-01 BC
+-- SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001
+-- SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01
+-- SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901
+-- SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901
+-- SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01
+-- SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01
+-- SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')); -- 0100-01-01 BC
+-- SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01
+-- SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC
+-- SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-01-01 BC
 
 -- [SPARK-29006] Support special date/timestamp values `infinity`/`-infinity`
 --
diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql
index bf69da2..ade29cc 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql
@@ -200,15 +200,15 @@ SELECT '' AS `54`, d1 as `timestamp`,
     date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second`
     FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01';
 
-SELECT '' AS `54`, d1 as `timestamp`,
-    date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec,
-    date_part( 'usec', d1) AS usec
-    FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01';
-
-SELECT '' AS `54`, d1 as `timestamp`,
-    date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week,
-    date_part( 'dow', d1) AS dow
-    FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01';
+-- SELECT '' AS `54`, d1 as `timestamp`,
+--     date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec,
+--     date_part( 'usec', d1) AS usec
+--     FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01';
+
+-- SELECT '' AS `54`, d1 as `timestamp`,
+--     date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week,
+--     date_part( 'dow', d1) AS dow
+--     FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01';
 
 -- [SPARK-28137] Data Type Formatting Functions
 -- TO_CHAR()
diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out
deleted file mode 100644
index 028448b..0000000
--- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out
+++ /dev/null
@@ -1,670 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- Number of queries: 83
-
-
--- !query
-CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select date_part('millennium', c) from t
--- !query schema
-struct<date_part('millennium', t.`c`):int>
--- !query output
-3
-
-
--- !query
-select date_part('millennia', c) from t
--- !query schema
-struct<date_part('millennia', t.`c`):int>
--- !query output
-3
-
-
--- !query
-select date_part('mil', c) from t
--- !query schema
-struct<date_part('mil', t.`c`):int>
--- !query output
-3
-
-
--- !query
-select date_part('mils', c) from t
--- !query schema
-struct<date_part('mils', t.`c`):int>
--- !query output
-3
-
-
--- !query
-select date_part('century', c) from t
--- !query schema
-struct<date_part('century', t.`c`):int>
--- !query output
-21
-
-
--- !query
-select date_part('centuries', c) from t
--- !query schema
-struct<date_part('centuries', t.`c`):int>
--- !query output
-21
-
-
--- !query
-select date_part('c', c) from t
--- !query schema
-struct<date_part('c', t.`c`):int>
--- !query output
-21
-
-
--- !query
-select date_part('cent', c) from t
--- !query schema
-struct<date_part('cent', t.`c`):int>
--- !query output
-21
-
-
--- !query
-select date_part('decade', c) from t
--- !query schema
-struct<date_part('decade', t.`c`):int>
--- !query output
-201
-
-
--- !query
-select date_part('decades', c) from t
--- !query schema
-struct<date_part('decades', t.`c`):int>
--- !query output
-201
-
-
--- !query
-select date_part('dec', c) from t
--- !query schema
-struct<date_part('dec', t.`c`):int>
--- !query output
-201
-
-
--- !query
-select date_part('decs', c) from t
--- !query schema
-struct<date_part('decs', t.`c`):int>
--- !query output
-201
-
-
--- !query
-select date_part('year', c) from t
--- !query schema
-struct<date_part('year', t.`c`):int>
--- !query output
-2011
-
-
--- !query
-select date_part('y', c) from t
--- !query schema
-struct<date_part('y', t.`c`):int>
--- !query output
-2011
-
-
--- !query
-select date_part('years', c) from t
--- !query schema
-struct<date_part('years', t.`c`):int>
--- !query output
-2011
-
-
--- !query
-select date_part('yr', c) from t
--- !query schema
-struct<date_part('yr', t.`c`):int>
--- !query output
-2011
-
-
--- !query
-select date_part('yrs', c) from t
--- !query schema
-struct<date_part('yrs', t.`c`):int>
--- !query output
-2011
-
-
--- !query
-select date_part('quarter', c) from t
--- !query schema
-struct<date_part('quarter', t.`c`):int>
--- !query output
-2
-
-
--- !query
-select date_part('qtr', c) from t
--- !query schema
-struct<date_part('qtr', t.`c`):int>
--- !query output
-2
-
-
--- !query
-select date_part('month', c) from t
--- !query schema
-struct<date_part('month', t.`c`):int>
--- !query output
-5
-
-
--- !query
-select date_part('mon', c) from t
--- !query schema
-struct<date_part('mon', t.`c`):int>
--- !query output
-5
-
-
--- !query
-select date_part('mons', c) from t
--- !query schema
-struct<date_part('mons', t.`c`):int>
--- !query output
-5
-
-
--- !query
-select date_part('months', c) from t
--- !query schema
-struct<date_part('months', t.`c`):int>
--- !query output
-5
-
-
--- !query
-select date_part('week', c) from t
--- !query schema
-struct<date_part('week', t.`c`):int>
--- !query output
-18
-
-
--- !query
-select date_part('w', c) from t
--- !query schema
-struct<date_part('w', t.`c`):int>
--- !query output
-18
-
-
--- !query
-select date_part('weeks', c) from t
--- !query schema
-struct<date_part('weeks', t.`c`):int>
--- !query output
-18
-
-
--- !query
-select date_part('day', c) from t
--- !query schema
-struct<date_part('day', t.`c`):int>
--- !query output
-6
-
-
--- !query
-select date_part('d', c) from t
--- !query schema
-struct<date_part('d', t.`c`):int>
--- !query output
-6
-
-
--- !query
-select date_part('days', c) from t
--- !query schema
-struct<date_part('days', t.`c`):int>
--- !query output
-6
-
-
--- !query
-select date_part('dayofweek', c) from t
--- !query schema
-struct<date_part('dayofweek', t.`c`):int>
--- !query output
-6
-
-
--- !query
-select date_part('dow', c) from t
--- !query schema
-struct<date_part('dow', t.`c`):int>
--- !query output
-6
-
-
--- !query
-select date_part('isodow', c) from t
--- !query schema
-struct<date_part('isodow', t.`c`):int>
--- !query output
-5
-
-
--- !query
-select date_part('doy', c) from t
--- !query schema
-struct<date_part('doy', t.`c`):int>
--- !query output
-126
-
-
--- !query
-select date_part('hour', c) from t
--- !query schema
-struct<date_part('hour', t.`c`):int>
--- !query output
-7
-
-
--- !query
-select date_part('h', c) from t
--- !query schema
-struct<date_part('h', t.`c`):int>
--- !query output
-7
-
-
--- !query
-select date_part('hours', c) from t
--- !query schema
-struct<date_part('hours', t.`c`):int>
--- !query output
-7
-
-
--- !query
-select date_part('hr', c) from t
--- !query schema
-struct<date_part('hr', t.`c`):int>
--- !query output
-7
-
-
--- !query
-select date_part('hrs', c) from t
--- !query schema
-struct<date_part('hrs', t.`c`):int>
--- !query output
-7
-
-
--- !query
-select date_part('minute', c) from t
--- !query schema
-struct<date_part('minute', t.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('m', c) from t
--- !query schema
-struct<date_part('m', t.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('min', c) from t
--- !query schema
-struct<date_part('min', t.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('mins', c) from t
--- !query schema
-struct<date_part('mins', t.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('minutes', c) from t
--- !query schema
-struct<date_part('minutes', t.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('second', c) from t
--- !query schema
-struct<date_part('second', t.`c`):decimal(8,6)>
--- !query output
-9.123456
-
-
--- !query
-select date_part('s', c) from t
--- !query schema
-struct<date_part('s', t.`c`):decimal(8,6)>
--- !query output
-9.123456
-
-
--- !query
-select date_part('sec', c) from t
--- !query schema
-struct<date_part('sec', t.`c`):decimal(8,6)>
--- !query output
-9.123456
-
-
--- !query
-select date_part('seconds', c) from t
--- !query schema
-struct<date_part('seconds', t.`c`):decimal(8,6)>
--- !query output
-9.123456
-
-
--- !query
-select date_part('secs', c) from t
--- !query schema
-struct<date_part('secs', t.`c`):decimal(8,6)>
--- !query output
-9.123456
-
-
--- !query
-select date_part('not_supported', c) from t
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7
-
-
--- !query
-select date_part(c, c) from t
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-The field parameter needs to be a foldable string value.;; line 1 pos 7
-
-
--- !query
-select date_part(null, c) from t
--- !query schema
-struct<date_part(NULL, t.`c`):double>
--- !query output
-NULL
-
-
--- !query
-CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-select date_part('year', c) from t2
--- !query schema
-struct<date_part('year', t2.`c`):int>
--- !query output
-1010
-
-
--- !query
-select date_part('y', c) from t2
--- !query schema
-struct<date_part('y', t2.`c`):int>
--- !query output
-1010
-
-
--- !query
-select date_part('years', c) from t2
--- !query schema
-struct<date_part('years', t2.`c`):int>
--- !query output
-1010
-
-
--- !query
-select date_part('yr', c) from t2
--- !query schema
-struct<date_part('yr', t2.`c`):int>
--- !query output
-1010
-
-
--- !query
-select date_part('yrs', c) from t2
--- !query schema
-struct<date_part('yrs', t2.`c`):int>
--- !query output
-1010
-
-
--- !query
-select date_part('month', c) from t2
--- !query schema
-struct<date_part('month', t2.`c`):tinyint>
--- !query output
-9
-
-
--- !query
-select date_part('mon', c) from t2
--- !query schema
-struct<date_part('mon', t2.`c`):tinyint>
--- !query output
-9
-
-
--- !query
-select date_part('mons', c) from t2
--- !query schema
-struct<date_part('mons', t2.`c`):tinyint>
--- !query output
-9
-
-
--- !query
-select date_part('months', c) from t2
--- !query schema
-struct<date_part('months', t2.`c`):tinyint>
--- !query output
-9
-
-
--- !query
-select date_part('day', c) from t2
--- !query schema
-struct<date_part('day', t2.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('d', c) from t2
--- !query schema
-struct<date_part('d', t2.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('days', c) from t2
--- !query schema
-struct<date_part('days', t2.`c`):int>
--- !query output
-8
-
-
--- !query
-select date_part('hour', c) from t2
--- !query schema
-struct<date_part('hour', t2.`c`):bigint>
--- !query output
-7
-
-
--- !query
-select date_part('h', c) from t2
--- !query schema
-struct<date_part('h', t2.`c`):bigint>
--- !query output
-7
-
-
--- !query
-select date_part('hours', c) from t2
--- !query schema
-struct<date_part('hours', t2.`c`):bigint>
--- !query output
-7
-
-
--- !query
-select date_part('hr', c) from t2
--- !query schema
-struct<date_part('hr', t2.`c`):bigint>
--- !query output
-7
-
-
--- !query
-select date_part('hrs', c) from t2
--- !query schema
-struct<date_part('hrs', t2.`c`):bigint>
--- !query output
-7
-
-
--- !query
-select date_part('minute', c) from t2
--- !query schema
-struct<date_part('minute', t2.`c`):tinyint>
--- !query output
-6
-
-
--- !query
-select date_part('m', c) from t2
--- !query schema
-struct<date_part('m', t2.`c`):tinyint>
--- !query output
-6
-
-
--- !query
-select date_part('min', c) from t2
--- !query schema
-struct<date_part('min', t2.`c`):tinyint>
--- !query output
-6
-
-
--- !query
-select date_part('mins', c) from t2
--- !query schema
-struct<date_part('mins', t2.`c`):tinyint>
--- !query output
-6
-
-
--- !query
-select date_part('minutes', c) from t2
--- !query schema
-struct<date_part('minutes', t2.`c`):tinyint>
--- !query output
-6
-
-
--- !query
-select date_part('second', c) from t2
--- !query schema
-struct<date_part('second', t2.`c`):decimal(8,6)>
--- !query output
-5.004003
-
-
--- !query
-select date_part('s', c) from t2
--- !query schema
-struct<date_part('s', t2.`c`):decimal(8,6)>
--- !query output
-5.004003
-
-
--- !query
-select date_part('sec', c) from t2
--- !query schema
-struct<date_part('sec', t2.`c`):decimal(8,6)>
--- !query output
-5.004003
-
-
--- !query
-select date_part('seconds', c) from t2
--- !query schema
-struct<date_part('seconds', t2.`c`):decimal(8,6)>
--- !query output
-5.004003
-
-
--- !query
-select date_part('secs', c) from t2
--- !query schema
-struct<date_part('secs', t2.`c`):decimal(8,6)>
--- !query output
-5.004003
-
-
--- !query
-select date_part('not_supported', c) from t2
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7
-
-
--- !query
-select date_part(c, c) from t2
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-The field parameter needs to be a foldable string value.;; line 1 pos 7
-
-
--- !query
-select date_part(null, c) from t2
--- !query schema
-struct<date_part(NULL, t2.`c`):double>
--- !query output
-NULL
diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out
index e6635e608..220acfb 100644
--- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 65
+-- Number of queries: 84
 
 
 -- !query
@@ -11,102 +11,6 @@ struct<>
 
 
 -- !query
-select extract(millennium from c) from t
--- !query schema
-struct<extract('millennium' FROM t.`c`):int>
--- !query output
-3
-
-
--- !query
-select extract(millennia from c) from t
--- !query schema
-struct<extract('millennia' FROM t.`c`):int>
--- !query output
-3
-
-
--- !query
-select extract(mil from c) from t
--- !query schema
-struct<extract('mil' FROM t.`c`):int>
--- !query output
-3
-
-
--- !query
-select extract(mils from c) from t
--- !query schema
-struct<extract('mils' FROM t.`c`):int>
--- !query output
-3
-
-
--- !query
-select extract(century from c) from t
--- !query schema
-struct<extract('century' FROM t.`c`):int>
--- !query output
-21
-
-
--- !query
-select extract(centuries from c) from t
--- !query schema
-struct<extract('centuries' FROM t.`c`):int>
--- !query output
-21
-
-
--- !query
-select extract(c from c) from t
--- !query schema
-struct<extract('c' FROM t.`c`):int>
--- !query output
-21
-
-
--- !query
-select extract(cent from c)from t
--- !query schema
-struct<extract('cent' FROM t.`c`):int>
--- !query output
-21
-
-
--- !query
-select extract(decade from c) from t
--- !query schema
-struct<extract('decade' FROM t.`c`):int>
--- !query output
-201
-
-
--- !query
-select extract(decades from c) from t
--- !query schema
-struct<extract('decades' FROM t.`c`):int>
--- !query output
-201
-
-
--- !query
-select extract(dec from c) from t
--- !query schema
-struct<extract('dec' FROM t.`c`):int>
--- !query output
-201
-
-
--- !query
-select extract(decs from c) from t
--- !query schema
-struct<extract('decs' FROM t.`c`):int>
--- !query output
-201
-
-
--- !query
 select extract(year from c), extract(year from i) from t
 -- !query schema
 struct<extract('year' FROM t.`c`):int,extract('year' FROM t.`i`):int>
@@ -147,9 +51,9 @@ struct<extract('yrs' FROM t.`c`):int,extract('yrs' FROM t.`i`):int>
 
 
 -- !query
-select extract(isoyear from c) from t
+select extract(yearofweek from c) from t
 -- !query schema
-struct<extract('isoyear' FROM t.`c`):int>
+struct<extract('yearofweek' FROM t.`c`):int>
 -- !query output
 2011
 
@@ -267,9 +171,17 @@ struct<extract('dow' FROM t.`c`):int>
 
 
 -- !query
-select extract(isodow from c) from t
+select extract(dayofweek_iso from c) from t
+-- !query schema
+struct<extract('dayofweek_iso' FROM t.`c`):int>
+-- !query output
+5
+
+
+-- !query
+select extract(dow_iso from c) from t
 -- !query schema
-struct<extract('isodow' FROM t.`c`):int>
+struct<extract('dow_iso' FROM t.`c`):int>
 -- !query output
 5
 
@@ -403,111 +315,329 @@ struct<extract('secs' FROM t.`c`):decimal(8,6),extract('secs' FROM t.`i`):decima
 
 
 -- !query
-select extract(milliseconds from c) from t
+select extract(not_supported from c) from t
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7
+
+
+-- !query
+select extract(not_supported from i) from t
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7
+
+
+-- !query
+select date_part('year', c), date_part('year', i) from t
 -- !query schema
-struct<extract('milliseconds' FROM t.`c`):decimal(8,3)>
+struct<date_part('year', t.`c`):int,date_part('year', t.`i`):int>
 -- !query output
-9123.456
+2011	11
 
 
 -- !query
-select extract(msec from c) from t
+select date_part('y', c), date_part('y', i) from t
 -- !query schema
-struct<extract('msec' FROM t.`c`):decimal(8,3)>
+struct<date_part('y', t.`c`):int,date_part('y', t.`i`):int>
 -- !query output
-9123.456
+2011	11
 
 
 -- !query
-select extract(msecs from c) from t
+select date_part('years', c), date_part('years', i) from t
 -- !query schema
-struct<extract('msecs' FROM t.`c`):decimal(8,3)>
+struct<date_part('years', t.`c`):int,date_part('years', t.`i`):int>
 -- !query output
-9123.456
+2011	11
 
 
 -- !query
-select extract(millisecon from c) from t
+select date_part('yr', c), date_part('yr', i) from t
 -- !query schema
-struct<extract('millisecon' FROM t.`c`):decimal(8,3)>
+struct<date_part('yr', t.`c`):int,date_part('yr', t.`i`):int>
 -- !query output
-9123.456
+2011	11
 
 
 -- !query
-select extract(mseconds from c) from t
+select date_part('yrs', c), date_part('yrs', i) from t
 -- !query schema
-struct<extract('mseconds' FROM t.`c`):decimal(8,3)>
+struct<date_part('yrs', t.`c`):int,date_part('yrs', t.`i`):int>
 -- !query output
-9123.456
+2011	11
 
 
 -- !query
-select extract(ms from c) from t
+select date_part('yearofweek', c) from t
 -- !query schema
-struct<extract('ms' FROM t.`c`):decimal(8,3)>
+struct<date_part('yearofweek', t.`c`):int>
 -- !query output
-9123.456
+2011
 
 
 -- !query
-select extract(microseconds from c) from t
+select date_part('quarter', c) from t
 -- !query schema
-struct<extract('microseconds' FROM t.`c`):int>
+struct<date_part('quarter', t.`c`):int>
 -- !query output
-9123456
+2
 
 
 -- !query
-select extract(usec from c) from t
+select date_part('qtr', c) from t
 -- !query schema
-struct<extract('usec' FROM t.`c`):int>
+struct<date_part('qtr', t.`c`):int>
 -- !query output
-9123456
+2
 
 
 -- !query
-select extract(usecs from c) from t
+select date_part('month', c), date_part('month', i) from t
 -- !query schema
-struct<extract('usecs' FROM t.`c`):int>
+struct<date_part('month', t.`c`):int,date_part('month', t.`i`):tinyint>
 -- !query output
-9123456
+5	8
 
 
 -- !query
-select extract(useconds from c) from t
+select date_part('mon', c), date_part('mon', i) from t
 -- !query schema
-struct<extract('useconds' FROM t.`c`):int>
+struct<date_part('mon', t.`c`):int,date_part('mon', t.`i`):tinyint>
 -- !query output
-9123456
+5	8
 
 
 -- !query
-select extract(microsecon from c) from t
+select date_part('mons', c), date_part('mons', i) from t
 -- !query schema
-struct<extract('microsecon' FROM t.`c`):int>
+struct<date_part('mons', t.`c`):int,date_part('mons', t.`i`):tinyint>
 -- !query output
-9123456
+5	8
 
 
 -- !query
-select extract(us from c) from t
+select date_part('months', c), date_part('months', i) from t
 -- !query schema
-struct<extract('us' FROM t.`c`):int>
+struct<date_part('months', t.`c`):int,date_part('months', t.`i`):tinyint>
 -- !query output
-9123456
+5	8
 
 
 -- !query
-select extract(epoch from c) from t
+select date_part('week', c) from t
 -- !query schema
-struct<extract('epoch' FROM t.`c`):decimal(20,6)>
+struct<date_part('week', t.`c`):int>
 -- !query output
-1304665689.123456
+18
 
 
 -- !query
-select extract(not_supported from c) from t
+select date_part('w', c) from t
+-- !query schema
+struct<date_part('w', t.`c`):int>
+-- !query output
+18
+
+
+-- !query
+select date_part('weeks', c) from t
+-- !query schema
+struct<date_part('weeks', t.`c`):int>
+-- !query output
+18
+
+
+-- !query
+select date_part('day', c), date_part('day', i) from t
+-- !query schema
+struct<date_part('day', t.`c`):int,date_part('day', t.`i`):int>
+-- !query output
+6	30
+
+
+-- !query
+select date_part('d', c), date_part('d', i) from t
+-- !query schema
+struct<date_part('d', t.`c`):int,date_part('d', t.`i`):int>
+-- !query output
+6	30
+
+
+-- !query
+select date_part('days', c), date_part('days', i) from t
+-- !query schema
+struct<date_part('days', t.`c`):int,date_part('days', t.`i`):int>
+-- !query output
+6	30
+
+
+-- !query
+select date_part('dayofweek', c) from t
+-- !query schema
+struct<date_part('dayofweek', t.`c`):int>
+-- !query output
+6
+
+
+-- !query
+select date_part('dow', c) from t
+-- !query schema
+struct<date_part('dow', t.`c`):int>
+-- !query output
+6
+
+
+-- !query
+select date_part('dayofweek_iso', c) from t
+-- !query schema
+struct<date_part('dayofweek_iso', t.`c`):int>
+-- !query output
+5
+
+
+-- !query
+select date_part('dow_iso', c) from t
+-- !query schema
+struct<date_part('dow_iso', t.`c`):int>
+-- !query output
+5
+
+
+-- !query
+select date_part('doy', c) from t
+-- !query schema
+struct<date_part('doy', t.`c`):int>
+-- !query output
+126
+
+
+-- !query
+select date_part('hour', c), date_part('hour', i) from t
+-- !query schema
+struct<date_part('hour', t.`c`):int,date_part('hour', t.`i`):bigint>
+-- !query output
+7	40
+
+
+-- !query
+select date_part('h', c), date_part('h', i) from t
+-- !query schema
+struct<date_part('h', t.`c`):int,date_part('h', t.`i`):bigint>
+-- !query output
+7	40
+
+
+-- !query
+select date_part('hours', c), date_part('hours', i) from t
+-- !query schema
+struct<date_part('hours', t.`c`):int,date_part('hours', t.`i`):bigint>
+-- !query output
+7	40
+
+
+-- !query
+select date_part('hr', c), date_part('hr', i) from t
+-- !query schema
+struct<date_part('hr', t.`c`):int,date_part('hr', t.`i`):bigint>
+-- !query output
+7	40
+
+
+-- !query
+select date_part('hrs', c), date_part('hrs', i) from t
+-- !query schema
+struct<date_part('hrs', t.`c`):int,date_part('hrs', t.`i`):bigint>
+-- !query output
+7	40
+
+
+-- !query
+select date_part('minute', c), date_part('minute', i) from t
+-- !query schema
+struct<date_part('minute', t.`c`):int,date_part('minute', t.`i`):tinyint>
+-- !query output
+8	50
+
+
+-- !query
+select date_part('m', c), date_part('m', i) from t
+-- !query schema
+struct<date_part('m', t.`c`):int,date_part('m', t.`i`):tinyint>
+-- !query output
+8	50
+
+
+-- !query
+select date_part('min', c), date_part('min', i) from t
+-- !query schema
+struct<date_part('min', t.`c`):int,date_part('min', t.`i`):tinyint>
+-- !query output
+8	50
+
+
+-- !query
+select date_part('mins', c), date_part('mins', i) from t
+-- !query schema
+struct<date_part('mins', t.`c`):int,date_part('mins', t.`i`):tinyint>
+-- !query output
+8	50
+
+
+-- !query
+select date_part('minutes', c), date_part('minutes', i) from t
+-- !query schema
+struct<date_part('minutes', t.`c`):int,date_part('minutes', t.`i`):tinyint>
+-- !query output
+8	50
+
+
+-- !query
+select date_part('second', c), date_part('second', i) from t
+-- !query schema
+struct<date_part('second', t.`c`):decimal(8,6),date_part('second', t.`i`):decimal(8,6)>
+-- !query output
+9.123456	6.789000
+
+
+-- !query
+select date_part('s', c), date_part('s', i) from t
+-- !query schema
+struct<date_part('s', t.`c`):decimal(8,6),date_part('s', t.`i`):decimal(8,6)>
+-- !query output
+9.123456	6.789000
+
+
+-- !query
+select date_part('sec', c), date_part('sec', i) from t
+-- !query schema
+struct<date_part('sec', t.`c`):decimal(8,6),date_part('sec', t.`i`):decimal(8,6)>
+-- !query output
+9.123456	6.789000
+
+
+-- !query
+select date_part('seconds', c), date_part('seconds', i) from t
+-- !query schema
+struct<date_part('seconds', t.`c`):decimal(8,6),date_part('seconds', t.`i`):decimal(8,6)>
+-- !query output
+9.123456	6.789000
+
+
+-- !query
+select date_part('secs', c), date_part('secs', i) from t
+-- !query schema
+struct<date_part('secs', t.`c`):decimal(8,6),date_part('secs', t.`i`):decimal(8,6)>
+-- !query output
+9.123456	6.789000
+
+
+-- !query
+select date_part('not_supported', c) from t
 -- !query schema
 struct<>
 -- !query output
@@ -516,9 +646,34 @@ Literals of type 'not_supported' are currently not supported for the string type
 
 
 -- !query
-select extract(not_supported from i) from t
+select date_part(c, c) from t
 -- !query schema
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7
+The field parameter needs to be a foldable string value.;; line 1 pos 7
+
+
+-- !query
+select date_part(null, c) from t
+-- !query schema
+struct<date_part(NULL, t.`c`):double>
+-- !query output
+NULL
+
+
+-- !query
+select date_part(i, i) from t
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+The field parameter needs to be a foldable string value.;; line 1 pos 7
+
+
+-- !query
+select date_part(null, i) from t
+-- !query schema
+struct<date_part(NULL, t.`i`):double>
+-- !query output
+NULL
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out
index 0dac13d..1d862ba 100755
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 98
+-- Number of queries: 60
 
 
 -- !query
@@ -572,310 +572,6 @@ struct<Two days:interval>
 
 
 -- !query
-SELECT EXTRACT(EPOCH FROM DATE        '1970-01-01')
--- !query schema
-struct<extract('EPOCH' FROM DATE '1970-01-01'):decimal(20,6)>
--- !query output
-0.000000
-
-
--- !query
-SELECT EXTRACT(EPOCH FROM TIMESTAMP   '1970-01-01')
--- !query schema
-struct<extract('EPOCH' FROM TIMESTAMP '1970-01-01 00:00:00'):decimal(20,6)>
--- !query output
-0.000000
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('CENTURY' FROM to_date('0101-12-31 BC', 'yyyy-MM-dd G')):int>
--- !query output
--2
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('CENTURY' FROM to_date('0100-12-31 BC', 'yyyy-MM-dd G')):int>
--- !query output
--1
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('CENTURY' FROM to_date('0001-12-31 BC', 'yyyy-MM-dd G')):int>
--- !query output
--1
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM DATE '0001-01-01')
--- !query schema
-struct<extract('CENTURY' FROM DATE '0001-01-01'):int>
--- !query output
-1
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD')
--- !query schema
-struct<extract('CENTURY' FROM DATE '0001-01-01'):int>
--- !query output
-1
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM DATE '1900-12-31')
--- !query schema
-struct<extract('CENTURY' FROM DATE '1900-12-31'):int>
--- !query output
-19
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM DATE '1901-01-01')
--- !query schema
-struct<extract('CENTURY' FROM DATE '1901-01-01'):int>
--- !query output
-20
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM DATE '2000-12-31')
--- !query schema
-struct<extract('CENTURY' FROM DATE '2000-12-31'):int>
--- !query output
-20
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM DATE '2001-01-01')
--- !query schema
-struct<extract('CENTURY' FROM DATE '2001-01-01'):int>
--- !query output
-21
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True
--- !query schema
-struct<True:boolean>
--- !query output
-true
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('MILLENNIUM' FROM to_date('0001-12-31 BC', 'yyyy-MM-dd G')):int>
--- !query output
--1
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD')
--- !query schema
-struct<extract('MILLENNIUM' FROM DATE '0001-01-01'):int>
--- !query output
-1
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31')
--- !query schema
-struct<extract('MILLENNIUM' FROM DATE '1000-12-31'):int>
--- !query output
-1
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01')
--- !query schema
-struct<extract('MILLENNIUM' FROM DATE '1001-01-01'):int>
--- !query output
-2
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31')
--- !query schema
-struct<extract('MILLENNIUM' FROM DATE '2000-12-31'):int>
--- !query output
-2
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01')
--- !query schema
-struct<extract('MILLENNIUM' FROM DATE '2001-01-01'):int>
--- !query output
-3
-
-
--- !query
-SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE)
--- !query schema
-struct<extract('MILLENNIUM' FROM current_date()):int>
--- !query output
-3
-
-
--- !query
-SELECT EXTRACT(DECADE FROM DATE '1994-12-25')
--- !query schema
-struct<extract('DECADE' FROM DATE '1994-12-25'):int>
--- !query output
-199
-
-
--- !query
-SELECT EXTRACT(DECADE FROM DATE '0010-01-01')
--- !query schema
-struct<extract('DECADE' FROM DATE '0010-01-01'):int>
--- !query output
-1
-
-
--- !query
-SELECT EXTRACT(DECADE FROM DATE '0009-12-31')
--- !query schema
-struct<extract('DECADE' FROM DATE '0009-12-31'):int>
--- !query output
-0
-
-
--- !query
-SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('DECADE' FROM to_date('0001-01-01 BC', 'yyyy-MM-dd G')):int>
--- !query output
-0
-
-
--- !query
-SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('DECADE' FROM to_date('0002-12-31 BC', 'yyyy-MM-dd G')):int>
--- !query output
--1
-
-
--- !query
-SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('DECADE' FROM to_date('0011-01-01 BC', 'yyyy-MM-dd G')):int>
--- !query output
--1
-
-
--- !query
-SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<extract('DECADE' FROM to_date('0012-12-31 BC', 'yyyy-MM-dd G')):int>
--- !query output
--2
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True
--- !query schema
-struct<True:boolean>
--- !query output
-true
-
-
--- !query
-SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000')
--- !query schema
-struct<extract('CENTURY' FROM TIMESTAMP '1970-03-20 04:30:00'):int>
--- !query output
-20
-
-
--- !query
-SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000')
--- !query schema
-struct<date_trunc(MILLENNIUM, TIMESTAMP '1970-03-20 04:30:00'):timestamp>
--- !query output
-1001-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20')
--- !query schema
-struct<date_trunc(MILLENNIUM, CAST(DATE '1970-03-20' AS TIMESTAMP)):timestamp>
--- !query output
-1001-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000')
--- !query schema
-struct<date_trunc(CENTURY, TIMESTAMP '1970-03-20 04:30:00'):timestamp>
--- !query output
-1901-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20')
--- !query schema
-struct<date_trunc(CENTURY, CAST(DATE '1970-03-20' AS TIMESTAMP)):timestamp>
--- !query output
-1901-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10')
--- !query schema
-struct<date_trunc(CENTURY, CAST(DATE '2004-08-10' AS TIMESTAMP)):timestamp>
--- !query output
-2001-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04')
--- !query schema
-struct<date_trunc(CENTURY, CAST(DATE '0002-02-04' AS TIMESTAMP)):timestamp>
--- !query output
-0001-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<date_trunc(CENTURY, CAST(to_date('0055-08-10 BC', 'yyyy-MM-dd G') AS TIMESTAMP)):timestamp>
--- !query output
--0099-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('DECADE', DATE '1993-12-25')
--- !query schema
-struct<date_trunc(DECADE, CAST(DATE '1993-12-25' AS TIMESTAMP)):timestamp>
--- !query output
-1990-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('DECADE', DATE '0004-12-25')
--- !query schema
-struct<date_trunc(DECADE, CAST(DATE '0004-12-25' AS TIMESTAMP)):timestamp>
--- !query output
-0000-01-01 00:00:00
-
-
--- !query
-SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G'))
--- !query schema
-struct<date_trunc(DECADE, CAST(to_date('0002-12-31 BC', 'yyyy-MM-dd G') AS TIMESTAMP)):timestamp>
--- !query output
--0010-01-01 00:00:00
-
-
--- !query
 select make_date(2013, 7, 15)
 -- !query schema
 struct<make_date(2013, 7, 15):date>
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out
index abfce91..5b0b636 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 32
+-- Number of queries: 30
 
 
 -- !query
@@ -266,36 +266,6 @@ struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:
 
 
 -- !query
-SELECT '' AS `54`, d1 as `timestamp`,
-    date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec,
-    date_part( 'usec', d1) AS usec
-    FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'
--- !query schema
-struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int>
--- !query output
-	1969-12-31 16:00:00	4	0.000	0
-	1997-01-02 00:00:00	1	0.000	0
-	1997-01-02 03:04:05	1	5000.000	5000000
-	1997-02-10 17:32:01	1	1000.000	1000000
-	2001-09-22 18:19:20	3	20000.000	20000000
-
-
--- !query
-SELECT '' AS `54`, d1 as `timestamp`,
-    date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week,
-    date_part( 'dow', d1) AS dow
-    FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'
--- !query schema
-struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int>
--- !query output
-	1969-12-31 16:00:00	1970	1	4
-	1997-01-02 00:00:00	1997	1	5
-	1997-01-02 03:04:05	1997	1	5
-	1997-02-10 17:32:01	1997	7	2
-	2001-09-22 18:19:20	2001	38	7
-
-
--- !query
 SELECT make_timestamp(2014,12,28,6,30,45.887)
 -- !query schema
 struct<make_timestamp(2014, 12, 28, 6, 30, CAST(45.887 AS DECIMAL(8,6))):timestamp>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala
index 6984d94..287854d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala
@@ -86,12 +86,8 @@ object ExtractBenchmark extends SqlBasedBenchmark {
 
   override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
     val N = 10000000L
-    val datetimeFields = Seq(
-      "MILLENNIUM", "CENTURY", "DECADE", "YEAR",
-      "ISOYEAR", "QUARTER", "MONTH", "WEEK",
-      "DAY", "DAYOFWEEK", "DOW", "ISODOW",
-      "DOY", "HOUR", "MINUTE", "SECOND",
-      "MILLISECONDS", "MICROSECONDS", "EPOCH")
+    val datetimeFields = Seq("YEAR", "YEAROFWEEK", "QUARTER", "MONTH", "WEEK", "DAY", "DAYOFWEEK",
+      "DOW", "DOW_ISO", "DAYOFWEEK_ISO", "DOY", "HOUR", "MINUTE", "SECOND")
     val intervalFields = Seq("YEAR", "MONTH", "DAY", "HOUR", "MINUTE", "SECOND")
     val settings = Map(
       "timestamp" -> datetimeFields,


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