Skip to content

Commit 56bdae4

Browse files
committed
apply with default locale
1 parent 4580c25 commit 56bdae4

File tree

7 files changed

+33
-26
lines changed

7 files changed

+33
-26
lines changed

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

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -562,7 +562,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti
562562
copy(timeZoneId = Option(timeZoneId))
563563

564564
override protected def nullSafeEval(timestamp: Any, format: Any): Any = {
565-
val df = TimestampFormatter(format.toString, timeZone, Locale.US)
565+
val df = TimestampFormatter(format.toString, timeZone)
566566
UTF8String.fromString(df.format(timestamp.asInstanceOf[Long]))
567567
}
568568

@@ -667,7 +667,7 @@ abstract class UnixTime
667667
private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String]
668668
private lazy val formatter: TimestampFormatter =
669669
try {
670-
TimestampFormatter(constFormat.toString, timeZone, Locale.US)
670+
TimestampFormatter(constFormat.toString, timeZone)
671671
} catch {
672672
case NonFatal(_) => null
673673
}
@@ -700,7 +700,7 @@ abstract class UnixTime
700700
} else {
701701
val formatString = f.asInstanceOf[UTF8String].toString
702702
try {
703-
TimestampFormatter(formatString, timeZone, Locale.US).parse(
703+
TimestampFormatter(formatString, timeZone).parse(
704704
t.asInstanceOf[UTF8String].toString) / MICROS_PER_SECOND
705705
} catch {
706706
case NonFatal(_) => null
@@ -821,7 +821,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[
821821
private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String]
822822
private lazy val formatter: TimestampFormatter =
823823
try {
824-
TimestampFormatter(constFormat.toString, timeZone, Locale.US)
824+
TimestampFormatter(constFormat.toString, timeZone)
825825
} catch {
826826
case NonFatal(_) => null
827827
}
@@ -847,7 +847,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[
847847
null
848848
} else {
849849
try {
850-
UTF8String.fromString(TimestampFormatter(f.toString, timeZone, Locale.US)
850+
UTF8String.fromString(TimestampFormatter(f.toString, timeZone)
851851
.format(time.asInstanceOf[Long] * MICROS_PER_SECOND))
852852
} catch {
853853
case NonFatal(_) => null

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -88,7 +88,8 @@ class LegacyFallbackDateFormatter(
8888
}
8989

9090
object DateFormatter {
91-
val defaultPattern = "yyyy-MM-dd"
91+
val defaultPattern: String = "yyyy-MM-dd"
92+
val defaultLocale: Locale = Locale.US
9293

9394
def apply(format: String, locale: Locale): DateFormatter = {
9495
if (SQLConf.get.legacyTimeParserEnabled) {
@@ -98,5 +99,7 @@ object DateFormatter {
9899
}
99100
}
100101

101-
def apply(): DateFormatter = apply(defaultPattern, Locale.US)
102+
def apply(format: String): DateFormatter = apply(format, defaultLocale)
103+
104+
def apply(): DateFormatter = apply(defaultPattern)
102105
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,8 @@ class LegacyFallbackTimestampFormatter(
105105
}
106106

107107
object TimestampFormatter {
108-
val defaultPattern = "yyyy-MM-dd HH:mm:ss"
108+
val defaultPattern: String = "yyyy-MM-dd HH:mm:ss"
109+
val defaultLocale: Locale = Locale.US
109110

110111
def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = {
111112
if (SQLConf.get.legacyTimeParserEnabled) {
@@ -115,5 +116,11 @@ object TimestampFormatter {
115116
}
116117
}
117118

118-
def apply(timeZone: TimeZone): TimestampFormatter = apply(defaultPattern, timeZone, Locale.US)
119+
def apply(format: String, timeZone: TimeZone): TimestampFormatter = {
120+
apply(format, timeZone, defaultLocale)
121+
}
122+
123+
def apply(timeZone: TimeZone): TimestampFormatter = {
124+
apply(defaultPattern, timeZone, defaultLocale)
125+
}
119126
}

sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper {
2929
test("parsing dates") {
3030
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
3131
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
32-
val formatter = DateFormatter("yyyy-MM-dd", Locale.US)
32+
val formatter = DateFormatter()
3333
val daysSinceEpoch = formatter.parse("2018-12-02")
3434
assert(daysSinceEpoch === 17867)
3535
}
@@ -39,7 +39,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper {
3939
test("format dates") {
4040
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
4141
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
42-
val formatter = DateFormatter("yyyy-MM-dd", Locale.US)
42+
val formatter = DateFormatter()
4343
val date = formatter.format(17867)
4444
assert(date === "2018-12-02")
4545
}
@@ -59,7 +59,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper {
5959
"5010-11-17").foreach { date =>
6060
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
6161
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
62-
val formatter = DateFormatter("yyyy-MM-dd", Locale.US)
62+
val formatter = DateFormatter()
6363
val days = formatter.parse(date)
6464
val formatted = formatter.format(days)
6565
assert(date === formatted)
@@ -82,7 +82,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper {
8282
1110657).foreach { days =>
8383
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
8484
withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) {
85-
val formatter = DateFormatter("yyyy-MM-dd", Locale.US)
85+
val formatter = DateFormatter()
8686
val date = formatter.format(days)
8787
val parsed = formatter.parse(date)
8888
assert(days === parsed)
@@ -92,7 +92,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper {
9292
}
9393

9494
test("parsing date without explicit day") {
95-
val formatter = DateFormatter("yyyy MMM", Locale.US)
95+
val formatter = DateFormatter("yyyy MMM")
9696
val daysSinceEpoch = formatter.parse("2018 Dec")
9797
assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay)
9898
}

sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala

Lines changed: 5 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -41,8 +41,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper {
4141
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
4242
val formatter = TimestampFormatter(
4343
"yyyy-MM-dd'T'HH:mm:ss.SSSSSS",
44-
TimeZone.getTimeZone(timeZone),
45-
Locale.US)
44+
TimeZone.getTimeZone(timeZone))
4645
val microsSinceEpoch = formatter.parse(localDate)
4746
assert(microsSinceEpoch === expectedMicros(timeZone))
4847
}
@@ -62,8 +61,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper {
6261
DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone =>
6362
val formatter = TimestampFormatter(
6463
"yyyy-MM-dd'T'HH:mm:ss.SSSSSS",
65-
TimeZone.getTimeZone(timeZone),
66-
Locale.US)
64+
TimeZone.getTimeZone(timeZone))
6765
val timestamp = formatter.format(microsSinceEpoch)
6866
assert(timestamp === expectedTimestamp(timeZone))
6967
}
@@ -81,7 +79,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper {
8179
2177456523456789L,
8280
11858049903010203L).foreach { micros =>
8381
DateTimeTestUtils.outstandingTimezones.foreach { timeZone =>
84-
val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone, Locale.US)
82+
val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone)
8583
val timestamp = formatter.format(micros)
8684
val parsed = formatter.parse(timestamp)
8785
assert(micros === parsed)
@@ -101,7 +99,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper {
10199
"2039-01-01T01:02:03.456789",
102100
"2345-10-07T22:45:03.010203").foreach { timestamp =>
103101
DateTimeTestUtils.outstandingTimezones.foreach { timeZone =>
104-
val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone, Locale.US)
102+
val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone)
105103
val micros = formatter.parse(timestamp)
106104
val formatted = formatter.format(micros)
107105
assert(timestamp === formatted)
@@ -112,8 +110,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper {
112110
test(" case insensitive parsing of am and pm") {
113111
val formatter = TimestampFormatter(
114112
"yyyy MMM dd hh:mm:ss a",
115-
TimeZone.getTimeZone("UTC"),
116-
Locale.US)
113+
TimeZone.getTimeZone("UTC"))
117114
val micros = formatter.parse("2009 Mar 20 11:30:01 am")
118115
assert(micros === TimeUnit.SECONDS.toMicros(
119116
LocalDateTime.of(2009, 3, 20, 11, 30, 1).toEpochSecond(ZoneOffset.UTC)))

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -124,8 +124,8 @@ object PartitioningUtils {
124124
Map.empty[String, DataType]
125125
}
126126

127-
val dateFormatter = DateFormatter(datePartitionPattern, Locale.US)
128-
val timestampFormatter = TimestampFormatter(timestampPartitionPattern, timeZone, Locale.US)
127+
val dateFormatter = DateFormatter(datePartitionPattern)
128+
val timestampFormatter = TimestampFormatter(timestampPartitionPattern, timeZone)
129129
// First, we need to parse every partition's path and see if we can find partition values.
130130
val (partitionValues, optDiscoveredBasePaths) = paths.map { path =>
131131
parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes,

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -56,8 +56,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha
5656

5757
val timeZone = TimeZone.getDefault()
5858
val timeZoneId = timeZone.getID
59-
val df = DateFormatter(datePartitionPattern, Locale.US)
60-
val tf = TimestampFormatter(timestampPartitionPattern, timeZone, Locale.US)
59+
val df = DateFormatter(datePartitionPattern)
60+
val tf = TimestampFormatter(timestampPartitionPattern, timeZone)
6161

6262
protected override def beforeAll(): Unit = {
6363
super.beforeAll()

0 commit comments

Comments
 (0)