[SPARK-35979][SQL] Return different timestamp literals based on the default timestamp type

### What changes were proposed in this pull request?

For the timestamp literal, it should have the following behavior.
1. When `spark.sql.timestampType` is TIMESTAMP_NTZ: if there is no time zone part, return timestamp without time zone literal; otherwise, return timestamp with local time zone literal

2. When `spark.sql.timestampType` is TIMESTAMP_LTZ: return timestamp with local time zone literal

### Why are the changes needed?

When the default timestamp type is TIMESTAMP_NTZ, the result of type literal should return TIMESTAMP_NTZ when there is no time zone part in the string.

From setion 5.3 "literal" of ANSI SQL standard 2011:
```
27) The declared type of a <timestamp literal> that does not specify <time zone interval> is TIMESTAMP(P) WITHOUT TIME ZONE, where P is the number of digits in <seconds fraction>, if specified, and 0 (zero) otherwise. The declared type of a <timestamp literal> that specifies <time zone interval> is TIMESTAMP(P) WITH TIME ZONE, where P is the number of digits in <seconds fraction>, if specified, and 0 (zero) otherwise.
```
Since we don't have "timestamp with time zone", we use timestamp with local time zone instead.
### Does this PR introduce _any_ user-facing change?

No, the new timestmap type and the default timestamp configuration is not released yet.

### How was this patch tested?

Unit test

Closes #33215 from gengliangwang/tsLiteral.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
(cherry picked from commit 2fffec7de8)
Signed-off-by: Gengliang Wang <gengliang@apache.org>
This commit is contained in:
Gengliang Wang 2021-07-06 00:54:58 +08:00
parent d3e8c9c78b
commit a9947cbd71
5 changed files with 83 additions and 13 deletions

View file

@ -38,8 +38,8 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, IntervalUtils}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, getZoneId, stringToDate, stringToTimestamp}
import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, stringToTimestamp, stringToTimestampWithoutTimeZone}
import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit
import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, TableCatalog}
import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
@ -2126,9 +2126,31 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
val specialDate = convertSpecialDate(value, zoneId).map(Literal(_, DateType))
specialDate.getOrElse(toLiteral(stringToDate, DateType))
case "TIMESTAMP" =>
def constructTimestampLTZLiteral(value: String): Literal = {
val zoneId = getZoneId(conf.sessionLocalTimeZone)
val specialTs = convertSpecialTimestamp(value, zoneId).map(Literal(_, TimestampType))
specialTs.getOrElse(toLiteral(stringToTimestamp(_, zoneId), TimestampType))
}
SQLConf.get.timestampType match {
case TimestampNTZType =>
val specialTs = convertSpecialTimestampNTZ(value).map(Literal(_, TimestampNTZType))
specialTs.getOrElse {
val containsTimeZonePart =
DateTimeUtils.parseTimestampString(UTF8String.fromString(value))._2.isDefined
// If the input string contains time zone part, return a timestamp with local time
// zone literal.
if (containsTimeZonePart) {
constructTimestampLTZLiteral(value)
} else {
toLiteral(stringToTimestampWithoutTimeZone, TimestampNTZType)
}
}
case TimestampType =>
constructTimestampLTZLiteral(value)
}
case "INTERVAL" =>
val interval = try {
IntervalUtils.stringToInterval(UTF8String.fromString(value))

View file

@ -248,7 +248,7 @@ object DateTimeUtils {
* @return timestamp segments, time zone id and whether the input is just time without a date. If
* the input string can't be parsed as timestamp, the result timestamp segments are empty.
*/
private def parseTimestampString(s: UTF8String): (Array[Int], Option[ZoneId], Boolean) = {
def parseTimestampString(s: UTF8String): (Array[Int], Option[ZoneId], Boolean) = {
if (s == null) {
return (Array.empty, None, false)
}
@ -983,10 +983,9 @@ object DateTimeUtils {
* Extracts special values from an input string ignoring case.
*
* @param input A trimmed string
* @param zoneId Zone identifier used to get the current date.
* @return Some special value in lower case or None.
*/
private def extractSpecialValue(input: String, zoneId: ZoneId): Option[String] = {
private def extractSpecialValue(input: String): Option[String] = {
def isValid(value: String, timeZoneId: String): Boolean = {
// Special value can be without any time zone
if (timeZoneId.isEmpty) return true
@ -1018,7 +1017,7 @@ object DateTimeUtils {
* successfully otherwise None.
*/
def convertSpecialTimestamp(input: String, zoneId: ZoneId): Option[Long] = {
extractSpecialValue(input.trim, zoneId).flatMap {
extractSpecialValue(input.trim).flatMap {
case "epoch" => Some(0)
case "now" => Some(currentTimestamp())
case "today" => Some(instantToMicros(today(zoneId).toInstant))
@ -1028,6 +1027,28 @@ object DateTimeUtils {
}
}
/**
* Converts notational shorthands that are converted to ordinary timestamps without time zone.
*
* @param input A string to parse. It can contain trailing or leading whitespaces.
* @return Some of microseconds since the epoch if the conversion completed
* successfully otherwise None.
*/
def convertSpecialTimestampNTZ(input: String): Option[Long] = {
val localDateTime = extractSpecialValue(input.trim).flatMap {
case "epoch" => Some(LocalDateTime.of(1970, 1, 1, 0, 0))
case "now" => Some(LocalDateTime.now())
case "today" => Some(LocalDateTime.now().`with`(LocalTime.MIDNIGHT))
case "tomorrow" =>
Some(LocalDateTime.now().`with`(LocalTime.MIDNIGHT).plusDays(1))
case "yesterday" =>
Some(LocalDateTime.now().`with`(LocalTime.MIDNIGHT).minusDays(1))
case _ => None
}
localDateTime.map(localDateTimeToMicros)
}
/**
* Converts notational shorthands that are converted to ordinary dates.
*
@ -1036,7 +1057,7 @@ object DateTimeUtils {
* @return Some of days since the epoch if the conversion completed successfully otherwise None.
*/
def convertSpecialDate(input: String, zoneId: ZoneId): Option[Int] = {
extractSpecialValue(input.trim, zoneId).flatMap {
extractSpecialValue(input.trim).flatMap {
case "epoch" => Some(0)
case "now" | "today" => Some(currentDate(zoneId))
case "tomorrow" => Some(Math.addExact(currentDate(zoneId), 1))

View file

@ -2860,10 +2860,10 @@ object SQLConf {
val TIMESTAMP_TYPE =
buildConf("spark.sql.timestampType")
.doc("Configures the default timestamp type of Spark SQL, including SQL DDL and Cast " +
s"clause. Setting the configuration as ${TimestampTypes.TIMESTAMP_NTZ.toString} will " +
.doc("Configures the default timestamp type of Spark SQL, including SQL DDL, Cast clause " +
s"and type literal. Setting the configuration as ${TimestampTypes.TIMESTAMP_NTZ} will " +
"use TIMESTAMP WITHOUT TIME ZONE as the default type while putting it as " +
s"${TimestampTypes.TIMESTAMP_LTZ.toString} will use TIMESTAMP WITH LOCAL TIME ZONE. " +
s"${TimestampTypes.TIMESTAMP_LTZ} will use TIMESTAMP WITH LOCAL TIME ZONE. " +
"Before the 3.2.0 release, Spark only supports the TIMESTAMP WITH " +
"LOCAL TIME ZONE type.")
.version("3.2.0")

View file

@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils}
import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.TimestampTypes
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
@ -463,6 +464,17 @@ class ExpressionParserSuite extends AnalysisTest {
Literal(Timestamp.valueOf("2016-03-11 20:54:00.000")))
intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value")
// Timestamp without time zone
withSQLConf(SQLConf.TIMESTAMP_TYPE.key -> TimestampTypes.TIMESTAMP_NTZ.toString) {
assertEqual("tImEstAmp '2016-03-11 20:54:00.000'",
Literal(LocalDateTime.parse("2016-03-11T20:54:00.000")))
intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value")
// If the timestamp string contains time zone, return a timestamp with local time zone literal
assertEqual("tImEstAmp '1970-01-01 00:00:00.000 +01:00'",
Literal(-3600000000L, TimestampType))
}
// Interval.
val intervalLiteral = Literal(IntervalUtils.stringToInterval("interval 3 month 1 hour"))
assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral)

View file

@ -726,6 +726,21 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {
}
}
test("SPARK-35979: special timestamp without time zone values") {
val tolerance = TimeUnit.SECONDS.toMicros(30)
assert(convertSpecialTimestampNTZ("Epoch").get === 0)
val now = DateTimeUtils.localDateTimeToMicros(LocalDateTime.now())
convertSpecialTimestampNTZ("NOW").get should be(now +- tolerance)
val localToday = LocalDateTime.now().`with`(LocalTime.MIDNIGHT)
val yesterday = DateTimeUtils.localDateTimeToMicros(localToday.minusDays(1))
convertSpecialTimestampNTZ(" Yesterday").get should be(yesterday)
val today = DateTimeUtils.localDateTimeToMicros(localToday)
convertSpecialTimestampNTZ("Today ").get should be(today)
val tomorrow = DateTimeUtils.localDateTimeToMicros(localToday.plusDays(1))
convertSpecialTimestampNTZ(" tomorrow ").get should be(tomorrow)
}
test("SPARK-28141: special date values") {
testSpecialDatetimeValues { zoneId =>
assert(convertSpecialDate("epoch", zoneId).get === 0)