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
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,13 @@
package org.apache.spark.sql.catalyst.plans

import java.io.File
import java.time.ZoneId

import scala.util.control.NonFatal

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -61,4 +66,21 @@ trait SQLHelper {
path.delete()
try f(path) finally Utils.deleteRecursively(path)
}


def testSpecialDatetimeValues[T](test: ZoneId => T): Unit = {
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
val zoneId = getZoneId(timeZone)
// The test can fail around midnight if it gets the reference value
// before midnight but tested code resolves special value after midnight.
// Retry can guarantee that both values were taken on the same day.
try {
test(zoneId)
} catch {
case NonFatal(_) => test(zoneId)
Copy link
Contributor

Choose a reason for hiding this comment

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

shall we set a threshold like 3? otherwise we may hit dead loop for real failures.

Copy link
Member Author

Choose a reason for hiding this comment

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

Maybe I don't understand how such loop could happen but this is not recursive code. test can be invoke maximum 2 times.

Copy link
Contributor

Choose a reason for hiding this comment

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

ah I mis-looked. It's not recursive. LGTM

}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,18 +19,19 @@ package org.apache.spark.sql.catalyst.util

import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset}
import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset}
import java.util.{Locale, TimeZone}
import java.util.concurrent.TimeUnit

import org.scalatest.Matchers

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}

class DateTimeUtilsSuite extends SparkFunSuite with Matchers {
class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper {

val TimeZonePST = TimeZone.getTimeZone("PST")
private def defaultZoneId = ZoneId.systemDefault()
Expand Down Expand Up @@ -605,27 +606,27 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers {
}

test("special timestamp values") {
DateTimeTestUtils.outstandingZoneIds.foreach { zoneId =>
testSpecialDatetimeValues { zoneId =>
val tolerance = TimeUnit.SECONDS.toMicros(30)

assert(toTimestamp("Epoch", zoneId).get === 0)
val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant)
toTimestamp("NOW", zoneId).get should be (now +- tolerance)
val now = instantToMicros(Instant.now())
toTimestamp("NOW", zoneId).get should be(now +- tolerance)
assert(toTimestamp("now UTC", zoneId) === None)
val localToday = LocalDateTime.now(zoneId)
.`with`(LocalTime.MIDNIGHT)
.atZone(zoneId)
val yesterday = instantToMicros(localToday.minusDays(1).toInstant)
toTimestamp(" Yesterday", zoneId).get should be (yesterday +- tolerance)
toTimestamp(" Yesterday", zoneId).get should be(yesterday +- tolerance)
val today = instantToMicros(localToday.toInstant)
toTimestamp("Today ", zoneId).get should be (today +- tolerance)
toTimestamp("Today ", zoneId).get should be(today +- tolerance)
val tomorrow = instantToMicros(localToday.plusDays(1).toInstant)
toTimestamp(" tomorrow CET ", zoneId).get should be (tomorrow +- tolerance)
toTimestamp(" tomorrow CET ", zoneId).get should be(tomorrow +- tolerance)
}
}

test("special date values") {
DateTimeTestUtils.outstandingZoneIds.foreach { zoneId =>
testSpecialDatetimeValues { zoneId =>
assert(toDate("epoch", zoneId).get === 0)
val today = localDateToDays(LocalDate.now(zoneId))
assert(toDate("YESTERDAY", zoneId).get === today - 1)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,18 +103,15 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper {
}

test("special date values") {
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
val zoneId = getZoneId(timeZone)
val formatter = DateFormatter(zoneId)
testSpecialDatetimeValues { zoneId =>
val formatter = DateFormatter(zoneId)

assert(formatter.parse("EPOCH") === 0)
val today = localDateToDays(LocalDate.now(zoneId))
assert(formatter.parse("Yesterday") === today - 1)
assert(formatter.parse("now") === today)
assert(formatter.parse("today ") === today)
assert(formatter.parse("tomorrow UTC") === today + 1)
}
assert(formatter.parse("EPOCH") === 0)
val today = localDateToDays(LocalDate.now(zoneId))
assert(formatter.parse("Yesterday") === today - 1)
assert(formatter.parse("now") === today)
assert(formatter.parse("today ") === today)
assert(formatter.parse("tomorrow UTC") === today + 1)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,15 @@

package org.apache.spark.sql.util

import java.time.{LocalDateTime, LocalTime, ZoneOffset}
import java.time.{Instant, LocalDateTime, LocalTime, ZoneOffset}
import java.util.concurrent.TimeUnit

import org.scalatest.Matchers

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, instantToMicros}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros

class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers {

Expand Down Expand Up @@ -137,25 +136,22 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers
}

test("special timestamp values") {
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
val zoneId = getZoneId(timeZone)
val formatter = TimestampFormatter(zoneId)
val tolerance = TimeUnit.SECONDS.toMicros(30)
testSpecialDatetimeValues { zoneId =>
val formatter = TimestampFormatter(zoneId)
val tolerance = TimeUnit.SECONDS.toMicros(30)

assert(formatter.parse("EPOCH") === 0)
val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant)
formatter.parse("now") should be (now +- tolerance)
val localToday = LocalDateTime.now(zoneId)
.`with`(LocalTime.MIDNIGHT)
.atZone(zoneId)
val yesterday = instantToMicros(localToday.minusDays(1).toInstant)
formatter.parse("yesterday CET") should be (yesterday +- tolerance)
val today = instantToMicros(localToday.toInstant)
formatter.parse(" TODAY ") should be (today +- tolerance)
val tomorrow = instantToMicros(localToday.plusDays(1).toInstant)
formatter.parse("Tomorrow ") should be (tomorrow +- tolerance)
}
assert(formatter.parse("EPOCH") === 0)
val now = instantToMicros(Instant.now())
formatter.parse("now") should be(now +- tolerance)
val localToday = LocalDateTime.now(zoneId)
.`with`(LocalTime.MIDNIGHT)
.atZone(zoneId)
val yesterday = instantToMicros(localToday.minusDays(1).toInstant)
formatter.parse("yesterday CET") should be(yesterday +- tolerance)
val today = instantToMicros(localToday.toInstant)
formatter.parse(" TODAY ") should be(today +- tolerance)
val tomorrow = instantToMicros(localToday.plusDays(1).toInstant)
formatter.parse("Tomorrow ") should be(tomorrow +- tolerance)
}
}
}