Skip to content

Commit 06abd06

Browse files
MaxGekkcloud-fan
authored andcommitted
[SPARK-27252][SQL] Make current_date() independent from time zones
## What changes were proposed in this pull request? This makes the `CurrentDate` expression and `current_date` function independent from time zone settings. New result is number of days since epoch in `UTC` time zone. Previously, Spark shifted the current date (in `UTC` time zone) according the session time zone which violets definition of `DateType` - number of days since epoch (which is an absolute point in time, midnight of Jan 1 1970 in UTC time). The changes makes `CurrentDate` consistent to `CurrentTimestamp` which is independent from time zone too. ## How was this patch tested? The changes were tested by existing test suites like `DateExpressionsSuite`. Closes #24185 from MaxGekk/current-date. Lead-authored-by: Maxim Gekk <[email protected]> Co-authored-by: Maxim Gekk <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 50cded5 commit 06abd06

File tree

8 files changed

+39
-39
lines changed

8 files changed

+39
-39
lines changed

docs/sql-migration-guide-upgrade.md

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -103,7 +103,9 @@ displayTitle: Spark SQL Upgrading Guide
103103

104104
- In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution.
105105

106-
- In Spark version 2.4 abd earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`.
106+
- In Spark version 2.4 and earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`.
107+
108+
- In Spark version 2.4 and earlier, the `current_date` function returns the current date shifted according to the SQL config `spark.sql.session.timeZone`. Since Spark 3.0, the function always returns the current date in the `UTC` time zone.
107109

108110
- Since Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`, and `DATE` literals are formatted using the UTC time zone. In Spark version 2.4 and earlier, both conversions use the default time zone of the Java virtual machine.
109111

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala

Lines changed: 7 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
package org.apache.spark.sql.catalyst.expressions
1919

2020
import java.sql.Timestamp
21-
import java.time.{Instant, LocalDate, ZoneId}
21+
import java.time.{Instant, LocalDate, ZoneId, ZoneOffset}
2222
import java.time.temporal.IsoFields
2323
import java.util.{Locale, TimeZone}
2424

@@ -52,30 +52,26 @@ trait TimeZoneAwareExpression extends Expression {
5252
@transient lazy val zoneId: ZoneId = DateTimeUtils.getZoneId(timeZoneId.get)
5353
}
5454

55+
// scalastyle:off line.size.limit
5556
/**
56-
* Returns the current date at the start of query evaluation.
57+
* Returns the current date in the UTC time zone at the start of query evaluation.
5758
* All calls of current_date within the same query return the same value.
5859
*
5960
* There is no code generation since this expression should get constant folded by the optimizer.
6061
*/
6162
@ExpressionDescription(
62-
usage = "_FUNC_() - Returns the current date at the start of query evaluation.",
63+
usage = "_FUNC_() - Returns the current date in the UTC time zone at the start of query evaluation.",
6364
since = "1.5.0")
64-
case class CurrentDate(timeZoneId: Option[String] = None)
65-
extends LeafExpression with TimeZoneAwareExpression with CodegenFallback {
66-
67-
def this() = this(None)
65+
// scalastyle:on line.size.limit
66+
case class CurrentDate() extends LeafExpression with CodegenFallback {
6867

6968
override def foldable: Boolean = true
7069
override def nullable: Boolean = false
7170

7271
override def dataType: DataType = DateType
7372

74-
override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
75-
copy(timeZoneId = Option(timeZoneId))
76-
7773
override def eval(input: InternalRow): Any = {
78-
DateTimeUtils.millisToDays(System.currentTimeMillis(), timeZone)
74+
LocalDate.now(ZoneOffset.UTC).toEpochDay.toInt
7975
}
8076

8177
override def prettyName: String = "current_date"

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala

Lines changed: 12 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -58,21 +58,20 @@ object ReplaceExpressions extends Rule[LogicalPlan] {
5858
*/
5959
object ComputeCurrentTime extends Rule[LogicalPlan] {
6060
def apply(plan: LogicalPlan): LogicalPlan = {
61-
val currentDates = mutable.Map.empty[String, Literal]
62-
val timeExpr = CurrentTimestamp()
63-
val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long]
64-
val currentTime = Literal.create(timestamp, timeExpr.dataType)
61+
val currentDate = {
62+
val dateExpr = CurrentDate()
63+
val date = dateExpr.eval(EmptyRow).asInstanceOf[Int]
64+
Literal.create(date, dateExpr.dataType)
65+
}
66+
val currentTimestamp = {
67+
val timeExpr = CurrentTimestamp()
68+
val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long]
69+
Literal.create(timestamp, timeExpr.dataType)
70+
}
6571

6672
plan transformAllExpressions {
67-
case CurrentDate(Some(timeZoneId)) =>
68-
currentDates.getOrElseUpdate(timeZoneId, {
69-
Literal.create(
70-
DateTimeUtils.millisToDays(
71-
MICROSECONDS.toMillis(timestamp),
72-
DateTimeUtils.getTimeZone(timeZoneId)),
73-
DateType)
74-
})
75-
case CurrentTimestamp() => currentTime
73+
case CurrentDate() => currentDate
74+
case CurrentTimestamp() => currentTimestamp
7675
}
7776
}
7877
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -54,12 +54,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
5454

5555
test("datetime function current_date") {
5656
val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT)
57-
val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int]
57+
val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int]
5858
val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT)
5959
assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1)
6060

61-
val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int]
62-
val cdpst = CurrentDate(pstId).eval(EmptyRow).asInstanceOf[Int]
61+
val cdjst = CurrentDate().eval(EmptyRow).asInstanceOf[Int]
62+
val cdpst = CurrentDate().eval(EmptyRow).asInstanceOf[Int]
6363
assert(cdpst <= cd && cd <= cdjst)
6464
}
6565

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -514,7 +514,7 @@ class MicroBatchExecution(
514514
ct.dataType, Some("Dummy TimeZoneId"))
515515
case cd: CurrentDate =>
516516
CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs,
517-
cd.dataType, cd.timeZoneId)
517+
cd.dataType, Some("UTC"))
518518
}
519519

520520
val triggerLogicalPlan = sink match {

sql/core/src/main/scala/org/apache/spark/sql/functions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2572,7 +2572,7 @@ object functions {
25722572
}
25732573

25742574
/**
2575-
* Returns the current date as a date column.
2575+
* Returns the current date in the UTC time zone as a date column.
25762576
*
25772577
* @group datetime_funcs
25782578
* @since 1.5.0

sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -720,7 +720,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
720720
assert(testData.groupBy(col("key")).toString.contains(
721721
"[grouping expressions: [key], value: [key: int, value: string], type: GroupBy]"))
722722
assert(testData.groupBy(current_date()).toString.contains(
723-
"grouping expressions: [current_date(None)], value: [key: int, value: string], " +
723+
"grouping expressions: [current_date()], value: [key: int, value: string], " +
724724
"type: GroupBy]"))
725725
}
726726

sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala

Lines changed: 11 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -19,25 +19,28 @@ package org.apache.spark.sql
1919

2020
import java.sql.{Date, Timestamp}
2121
import java.text.SimpleDateFormat
22+
import java.time.LocalDate
2223
import java.util.Locale
2324
import java.util.concurrent.TimeUnit
2425

25-
import org.apache.spark.sql.catalyst.util.DateTimeUtils
26+
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
2627
import org.apache.spark.sql.functions._
28+
import org.apache.spark.sql.internal.SQLConf
2729
import org.apache.spark.sql.test.SharedSQLContext
2830
import org.apache.spark.unsafe.types.CalendarInterval
2931

3032
class DateFunctionsSuite extends QueryTest with SharedSQLContext {
3133
import testImplicits._
3234

3335
test("function current_date") {
34-
val df1 = Seq((1, 2), (3, 1)).toDF("a", "b")
35-
val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis())
36-
val d1 = DateTimeUtils.fromJavaDate(df1.select(current_date()).collect().head.getDate(0))
37-
val d2 = DateTimeUtils.fromJavaDate(
38-
sql("""SELECT CURRENT_DATE()""").collect().head.getDate(0))
39-
val d3 = DateTimeUtils.millisToDays(System.currentTimeMillis())
40-
assert(d0 <= d1 && d1 <= d2 && d2 <= d3 && d3 - d0 <= 1)
36+
withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") {
37+
val df1 = Seq((1, 2), (3, 1)).toDF("a", "b")
38+
val d0 = System.currentTimeMillis() / MILLIS_PER_DAY
39+
val d1 = localDateToDays(df1.select(current_date()).collect().head.getAs[LocalDate](0))
40+
val d2 = localDateToDays(sql("""SELECT CURRENT_DATE()""").collect().head.getAs[LocalDate](0))
41+
val d3 = System.currentTimeMillis() / MILLIS_PER_DAY
42+
assert(d0 <= d1 && d1 <= d2 && d2 <= d3 && d3 - d0 <= 1)
43+
}
4144
}
4245

4346
test("function current_timestamp and now") {

0 commit comments

Comments
 (0)