Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions docs/sql-migration-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,8 @@ license: |

- In Spark 3.2, `CREATE TABLE AS SELECT` with non-empty `LOCATION` will throw `AnalysisException`. To restore the behavior before Spark 3.2, you can set `spark.sql.legacy.allowNonEmptyLocationInCTAS` to `true`.

- In Spark 3.2, special datetime values such as `epoch`, `today`, `yesterday`, `tomorrow`, and `now` are supported in typed literals only, for instance, `select timestamp'now'`. In Spark 3.1 and 3.0, such special values are supported in any casts of strings to dates/timestamps. To keep these special values as dates/timestamps in Spark 3.1 and 3.0, you should replace them manually, e.g. `if (c in ('now', 'today'), current_date(), cast(c as date))`.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if (c in ('now', 'today'), current_date(), cast(c as date))

What does it mean?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan See the example:

scala> val df = Seq("now", "2021-01-19", "today").toDF("c")
df: org.apache.spark.sql.DataFrame = [c: string]

scala> df.selectExpr("if (c in ('now', 'today'), current_date(), cast(c as date))").show(false)
+----------------------------------------------------------+
|(IF((c IN (now, today)), current_date(), CAST(c AS DATE)))|
+----------------------------------------------------------+
|2021-06-01                                                |
|2021-01-19                                                |
|2021-06-01                                                |
+----------------------------------------------------------+


## Upgrading from Spark SQL 3.0 to 3.1

- In Spark 3.1, statistical aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis`, `covar_samp`, `corr` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.statisticalAggregate` to `true`.
Expand Down
2 changes: 1 addition & 1 deletion sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala
Original file line number Diff line number Diff line change
Expand Up @@ -549,7 +549,7 @@ trait Row extends Serializable {
require(schema != null, "JSON serialization requires a non-null schema.")

lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)
lazy val dateFormatter = DateFormatter.apply(zoneId)
lazy val dateFormatter = DateFormatter()
lazy val timestampFormatter = TimestampFormatter(zoneId)

// Convert an iterator of values to a json array
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -668,7 +668,7 @@ object CatalogColumnStat extends Logging {
dataType match {
case BooleanType => s.toBoolean
case DateType if version == 1 => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s))
case DateType => DateFormatter(ZoneOffset.UTC).parse(s)
case DateType => DateFormatter().parse(s)
case TimestampType if version == 1 =>
DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s))
case TimestampType => getTimestampFormatter(isParsing = true).parse(s)
Expand All @@ -693,7 +693,7 @@ object CatalogColumnStat extends Logging {
*/
def toExternalString(v: Any, colName: String, dataType: DataType): String = {
val externalValue = dataType match {
case DateType => DateFormatter(ZoneOffset.UTC).format(v.asInstanceOf[Int])
case DateType => DateFormatter().format(v.asInstanceOf[Int])
case TimestampType => getTimestampFormatter(isParsing = false).format(v.asInstanceOf[Long])
case BooleanType | _: IntegralType | FloatType | DoubleType => v
case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@ class UnivocityGenerator(
isParsing = false)
private val dateFormatter = DateFormatter(
options.dateFormat,
options.zoneId,
options.locale,
legacyFormat = FAST_DATE_FORMAT,
isParsing = false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,6 @@ class UnivocityParser(
isParsing = true)
private lazy val dateFormatter = DateFormatter(
options.dateFormat,
options.zoneId,
options.locale,
legacyFormat = FAST_DATE_FORMAT,
isParsing = true)
Expand Down Expand Up @@ -206,7 +205,7 @@ class UnivocityParser(
// If fails to parse, then tries the way used in 2.0 and 1.x for backwards
// compatibility.
val str = DateTimeUtils.cleanLegacyTimestampStr(UTF8String.fromString(datum))
DateTimeUtils.stringToDate(str, options.zoneId).getOrElse(throw e)
DateTimeUtils.stringToDate(str).getOrElse(throw e)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -304,7 +304,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
// [[func]] assumes the input is no longer null because eval already does the null check.
@inline protected[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T])

private lazy val dateFormatter = DateFormatter(zoneId)
private lazy val dateFormatter = DateFormatter()
private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId)

private val legacyCastToStr = SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING)
Expand Down Expand Up @@ -520,9 +520,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
private[this] def castToDate(from: DataType): Any => Any = from match {
case StringType =>
if (ansiEnabled) {
buildCast[UTF8String](_, s => DateTimeUtils.stringToDateAnsi(s, zoneId))
buildCast[UTF8String](_, s => DateTimeUtils.stringToDateAnsi(s))
} else {
buildCast[UTF8String](_, s => DateTimeUtils.stringToDate(s, zoneId).orNull)
buildCast[UTF8String](_, s => DateTimeUtils.stringToDate(s).orNull)
}
case TimestampType =>
// throw valid precision more than seconds, according to Hive.
Expand Down Expand Up @@ -1167,25 +1167,18 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
private[this] def castToDateCode(
from: DataType,
ctx: CodegenContext): CastFunction = {
def getZoneId() = {
val zoneIdClass = classOf[ZoneId]
JavaCode.global(
ctx.addReferenceObj("zoneId", zoneId, zoneIdClass.getName),
zoneIdClass)
}
from match {
case StringType =>
val intOpt = ctx.freshVariable("intOpt", classOf[Option[Integer]])
val zid = getZoneId()
(c, evPrim, evNull) =>
if (ansiEnabled) {
code"""
$evPrim = org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDateAnsi($c, $zid);
$evPrim = org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDateAnsi($c);
"""
} else {
code"""
scala.Option<Integer> $intOpt =
org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDate($c, $zid);
org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDate($c);
if ($intOpt.isDefined()) {
$evPrim = ((Integer) $intOpt.get()).intValue();
} else {
Expand All @@ -1195,7 +1188,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit
}

case TimestampType =>
val zid = getZoneId()
val zidClass = classOf[ZoneId]
val zid = JavaCode.global(ctx.addReferenceObj("zoneId", zoneId, zidClass.getName), zidClass)
(c, evPrim, evNull) =>
code"""$evPrim =
org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToDays($c, $zid);"""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -339,7 +339,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression {
case other =>
dataType match {
case DateType =>
DateFormatter(timeZoneId).format(value.asInstanceOf[Int])
DateFormatter().format(value.asInstanceOf[Int])
case TimestampType =>
TimestampFormatter.getFractionFormatter(timeZoneId).format(value.asInstanceOf[Long])
case DayTimeIntervalType => toDayTimeIntervalString(value.asInstanceOf[Long], ANSI_STYLE)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,6 @@ private[sql] class JacksonGenerator(
isParsing = false)
private val dateFormatter = DateFormatter(
options.dateFormat,
options.zoneId,
options.locale,
legacyFormat = FAST_DATE_FORMAT,
isParsing = false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,6 @@ class JacksonParser(
isParsing = true)
private lazy val dateFormatter = DateFormatter(
options.dateFormat,
options.zoneId,
options.locale,
legacyFormat = FAST_DATE_FORMAT,
isParsing = true)
Expand Down Expand Up @@ -264,7 +263,7 @@ class JacksonParser(
// If fails to parse, then tries the way used in 2.0 and 1.x for backwards
// compatibility.
val str = DateTimeUtils.cleanLegacyTimestampStr(UTF8String.fromString(parser.getText))
DateTimeUtils.stringToDate(str, options.zoneId).getOrElse {
DateTimeUtils.stringToDate(str).getOrElse {
// In Spark 1.5.0, we store the data as number of days since epoch in string.
// So, we just convert it to Int.
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ 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.{getZoneId, stringToDate, stringToTimestamp}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, convertSpecialTimestamp, getZoneId, stringToDate, stringToTimestamp}
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
Expand Down Expand Up @@ -2094,10 +2094,13 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
try {
valueType match {
case "DATE" =>
toLiteral(stringToDate(_, getZoneId(conf.sessionLocalTimeZone)), DateType)
val zoneId = getZoneId(conf.sessionLocalTimeZone)
val specialDate = convertSpecialDate(value, zoneId).map(Literal(_, DateType))
specialDate.getOrElse(toLiteral(stringToDate, DateType))
case "TIMESTAMP" =>
val zoneId = getZoneId(conf.sessionLocalTimeZone)
toLiteral(stringToTimestamp(_, zoneId), TimestampType)
val specialTs = convertSpecialTimestamp(value, zoneId).map(Literal(_, TimestampType))
specialTs.getOrElse(toLiteral(stringToTimestamp(_, zoneId), TimestampType))
case "INTERVAL" =>
val interval = try {
IntervalUtils.stringToInterval(UTF8String.fromString(value))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.util

import java.text.SimpleDateFormat
import java.time.{LocalDate, ZoneId}
import java.time.LocalDate
import java.util.{Date, Locale}

import org.apache.commons.lang3.time.FastDateFormat
Expand All @@ -39,7 +39,6 @@ sealed trait DateFormatter extends Serializable {

class Iso8601DateFormatter(
pattern: String,
zoneId: ZoneId,
locale: Locale,
legacyFormat: LegacyDateFormats.LegacyDateFormat,
isParsing: Boolean)
Expand All @@ -49,17 +48,13 @@ class Iso8601DateFormatter(
private lazy val formatter = getOrCreateFormatter(pattern, locale, isParsing)

@transient
private lazy val legacyFormatter = DateFormatter.getLegacyFormatter(
pattern, zoneId, locale, legacyFormat)
private lazy val legacyFormatter = DateFormatter.getLegacyFormatter(pattern, locale, legacyFormat)

override def parse(s: String): Int = {
val specialDate = convertSpecialDate(s.trim, zoneId)
specialDate.getOrElse {
try {
val localDate = toLocalDate(formatter.parse(s))
localDateToDays(localDate)
} catch checkParsedDiff(s, legacyFormatter.parse)
}
try {
val localDate = toLocalDate(formatter.parse(s))
localDateToDays(localDate)
} catch checkParsedDiff(s, legacyFormatter.parse)
}

override def format(localDate: LocalDate): String = {
Expand Down Expand Up @@ -153,23 +148,21 @@ object DateFormatter {

private def getFormatter(
format: Option[String],
zoneId: ZoneId,
locale: Locale = defaultLocale,
legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT,
isParsing: Boolean): DateFormatter = {
val pattern = format.getOrElse(defaultPattern)
if (SQLConf.get.legacyTimeParserPolicy == LEGACY) {
getLegacyFormatter(pattern, zoneId, locale, legacyFormat)
getLegacyFormatter(pattern, locale, legacyFormat)
} else {
val df = new Iso8601DateFormatter(pattern, zoneId, locale, legacyFormat, isParsing)
val df = new Iso8601DateFormatter(pattern, locale, legacyFormat, isParsing)
df.validatePatternString()
df
}
}

def getLegacyFormatter(
pattern: String,
zoneId: ZoneId,
locale: Locale,
legacyFormat: LegacyDateFormat): DateFormatter = {
legacyFormat match {
Expand All @@ -182,18 +175,17 @@ object DateFormatter {

def apply(
format: String,
zoneId: ZoneId,
locale: Locale,
legacyFormat: LegacyDateFormat,
isParsing: Boolean): DateFormatter = {
getFormatter(Some(format), zoneId, locale, legacyFormat, isParsing)
getFormatter(Some(format), locale, legacyFormat, isParsing)
}

def apply(format: String, zoneId: ZoneId, isParsing: Boolean = false): DateFormatter = {
getFormatter(Some(format), zoneId, isParsing = isParsing)
def apply(format: String, isParsing: Boolean = false): DateFormatter = {
getFormatter(Some(format), isParsing = isParsing)
}

def apply(zoneId: ZoneId): DateFormatter = {
getFormatter(None, zoneId, isParsing = false)
def apply(): DateFormatter = {
getFormatter(None, isParsing = false)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.sql.catalyst.util

import java.nio.charset.StandardCharsets
import java.sql.{Date, Timestamp}
import java.time._
import java.time.temporal.{ChronoField, ChronoUnit, IsoFields}
Expand Down Expand Up @@ -246,8 +245,6 @@ object DateTimeUtils {
var i = 0
var currentSegmentValue = 0
val bytes = s.trimAll().getBytes
val specialTimestamp = convertSpecialTimestamp(bytes, timeZoneId)
if (specialTimestamp.isDefined) return specialTimestamp
var j = 0
var digitsMilli = 0
var justTime = false
Expand Down Expand Up @@ -419,16 +416,14 @@ object DateTimeUtils {
* `yyyy-[m]m-[d]d *`
* `yyyy-[m]m-[d]dT*`
*/
def stringToDate(s: UTF8String, zoneId: ZoneId): Option[Int] = {
def stringToDate(s: UTF8String): Option[Int] = {
if (s == null) {
return None
}
val segments: Array[Int] = Array[Int](1, 1, 1)
var i = 0
var currentSegmentValue = 0
val bytes = s.trimAll().getBytes
val specialDate = convertSpecialDate(bytes, zoneId)
if (specialDate.isDefined) return specialDate
var j = 0
while (j < bytes.length && (i < 3 && !(bytes(j) == ' ' || bytes(j) == 'T'))) {
val b = bytes(j)
Expand Down Expand Up @@ -467,8 +462,8 @@ object DateTimeUtils {
}
}

def stringToDateAnsi(s: UTF8String, zoneId: ZoneId): Int = {
stringToDate(s, zoneId).getOrElse {
def stringToDateAnsi(s: UTF8String): Int = {
stringToDate(s).getOrElse {
throw QueryExecutionErrors.cannotCastUTF8StringToDataTypeError(s, DateType)
}
}
Expand Down Expand Up @@ -908,13 +903,13 @@ object DateTimeUtils {
/**
* Converts notational shorthands that are converted to ordinary timestamps.
*
* @param input A trimmed string
* @param input A string to parse. It can contain trailing or leading whitespaces.
* @param zoneId Zone identifier used to get the current date.
* @return Some of microseconds since the epoch if the conversion completed
* successfully otherwise None.
*/
def convertSpecialTimestamp(input: String, zoneId: ZoneId): Option[Long] = {
extractSpecialValue(input, zoneId).flatMap {
extractSpecialValue(input.trim, zoneId).flatMap {
case "epoch" => Some(0)
case "now" => Some(currentTimestamp())
case "today" => Some(instantToMicros(today(zoneId).toInstant))
Expand All @@ -924,23 +919,15 @@ object DateTimeUtils {
}
}

private def convertSpecialTimestamp(bytes: Array[Byte], zoneId: ZoneId): Option[Long] = {
if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) {
convertSpecialTimestamp(new String(bytes, StandardCharsets.UTF_8), zoneId)
} else {
None
}
}

/**
* Converts notational shorthands that are converted to ordinary dates.
*
* @param input A trimmed string
* @param input A string to parse. It can contain trailing or leading whitespaces.
* @param zoneId Zone identifier used to get the current date.
* @return Some of days since the epoch if the conversion completed successfully otherwise None.
*/
def convertSpecialDate(input: String, zoneId: ZoneId): Option[Int] = {
extractSpecialValue(input, zoneId).flatMap {
extractSpecialValue(input.trim, zoneId).flatMap {
case "epoch" => Some(0)
case "now" | "today" => Some(currentDate(zoneId))
case "tomorrow" => Some(Math.addExact(currentDate(zoneId), 1))
Expand All @@ -949,14 +936,6 @@ object DateTimeUtils {
}
}

private def convertSpecialDate(bytes: Array[Byte], zoneId: ZoneId): Option[Int] = {
if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) {
convertSpecialDate(new String(bytes, StandardCharsets.UTF_8), zoneId)
} else {
None
}
}

/**
* Subtracts two dates expressed as days since 1970-01-01.
*
Expand Down
Loading