aboutsummaryrefslogtreecommitdiff
path: root/sql/core
diff options
context:
space:
mode:
authorSean Owen <sowen@cloudera.com>2016-11-02 09:39:15 +0000
committerSean Owen <sowen@cloudera.com>2016-11-02 09:39:15 +0000
commit9c8deef64efee20a0ddc9b612f90e77c80aede60 (patch)
treefeb6a7eeb4e10f628ff1227787c1ee430cb6195b /sql/core
parent70a5db7bbd192a4bc68bcfdc475ab221adf2fcdd (diff)
downloadspark-9c8deef64efee20a0ddc9b612f90e77c80aede60.tar.gz
spark-9c8deef64efee20a0ddc9b612f90e77c80aede60.tar.bz2
spark-9c8deef64efee20a0ddc9b612f90e77c80aede60.zip
[SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US
## What changes were proposed in this pull request? Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat` ## How was this patch tested? Existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #15610 from srowen/SPARK-18076.
Diffstat (limited to 'sql/core')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala11
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala9
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala9
7 files changed, 23 insertions, 21 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
index 3ab775c909..1981d8607c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
@@ -247,7 +247,7 @@ private[csv] object CSVTypeCast {
case options.positiveInf => Float.PositiveInfinity
case _ =>
Try(datum.toFloat)
- .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue())
+ .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).floatValue())
}
case _: DoubleType =>
datum match {
@@ -256,7 +256,7 @@ private[csv] object CSVTypeCast {
case options.positiveInf => Double.PositiveInfinity
case _ =>
Try(datum.toDouble)
- .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue())
+ .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).doubleValue())
}
case _: BooleanType => datum.toBoolean
case dt: DecimalType =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
index 014614eb99..5903729c11 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.execution.datasources.csv
import java.nio.charset.StandardCharsets
+import java.util.Locale
import org.apache.commons.lang3.time.FastDateFormat
@@ -104,11 +105,11 @@ private[csv] class CSVOptions(@transient private val parameters: Map[String, Str
// Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe.
val dateFormat: FastDateFormat =
- FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"))
+ FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), Locale.US)
val timestampFormat: FastDateFormat =
FastDateFormat.getInstance(
- parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ"))
+ parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ"), Locale.US)
val maxColumns = getInt("maxColumns", 20480)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
index 0cc1edd196..dbc27d8b23 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala
@@ -102,7 +102,7 @@ object SQLMetrics {
*/
def stringValue(metricsType: String, values: Seq[Long]): String = {
if (metricsType == SUM_METRIC) {
- val numberFormat = NumberFormat.getIntegerInstance(Locale.ENGLISH)
+ val numberFormat = NumberFormat.getIntegerInstance(Locale.US)
numberFormat.format(values.sum)
} else {
val strFormat: Long => String = if (metricsType == SIZE_METRIC) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala
index c662e7c6bc..042977f870 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala
@@ -21,7 +21,7 @@ import java.io.{BufferedReader, InputStreamReader, IOException}
import java.net.Socket
import java.sql.Timestamp
import java.text.SimpleDateFormat
-import java.util.Calendar
+import java.util.{Calendar, Locale}
import javax.annotation.concurrent.GuardedBy
import scala.collection.mutable.ListBuffer
@@ -37,7 +37,7 @@ object TextSocketSource {
val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil)
val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) ::
StructField("timestamp", TimestampType) :: Nil)
- val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
+ val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
}
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
index f7aa3b747a..e05b2252ee 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql
import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
+import java.util.Locale
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.functions._
@@ -55,8 +56,8 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
checkAnswer(sql("""SELECT CURRENT_TIMESTAMP() = NOW()"""), Row(true))
}
- val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
- val sdfDate = new SimpleDateFormat("yyyy-MM-dd")
+ val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
+ val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US)
val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime)
val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime)
@@ -395,11 +396,11 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
}
test("from_unixtime") {
- val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
+ val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS"
- val sdf2 = new SimpleDateFormat(fmt2)
+ val sdf2 = new SimpleDateFormat(fmt2, Locale.US)
val fmt3 = "yy-MM-dd HH-mm-ss"
- val sdf3 = new SimpleDateFormat(fmt3)
+ val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
val df = Seq((1000, "yyyy-MM-dd HH:mm:ss.SSS"), (-1000, "yy-MM-dd HH-mm-ss")).toDF("a", "b")
checkAnswer(
df.select(from_unixtime(col("a"))),
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index f7c22c6c93..8209b5bd7f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -21,6 +21,7 @@ import java.io.File
import java.nio.charset.UnsupportedCharsetException
import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
+import java.util.Locale
import org.apache.commons.lang3.time.FastDateFormat
import org.apache.hadoop.io.SequenceFile.CompressionType
@@ -487,7 +488,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
.select("date")
.collect()
- val dateFormat = new SimpleDateFormat("dd/MM/yyyy HH:mm")
+ val dateFormat = new SimpleDateFormat("dd/MM/yyyy HH:mm", Locale.US)
val expected =
Seq(Seq(new Timestamp(dateFormat.parse("26/08/2015 18:00").getTime)),
Seq(new Timestamp(dateFormat.parse("27/10/2014 18:30").getTime)),
@@ -509,7 +510,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
.select("date")
.collect()
- val dateFormat = new SimpleDateFormat("dd/MM/yyyy hh:mm")
+ val dateFormat = new SimpleDateFormat("dd/MM/yyyy hh:mm", Locale.US)
val expected = Seq(
new Date(dateFormat.parse("26/08/2015 18:00").getTime),
new Date(dateFormat.parse("27/10/2014 18:30").getTime),
@@ -728,7 +729,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
.option("inferSchema", "false")
.load(iso8601timestampsPath)
- val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")
+ val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.US)
val expectedTimestamps = timestamps.collect().map { r =>
// This should be ISO8601 formatted string.
Row(iso8501.format(r.toSeq.head))
@@ -761,7 +762,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
.option("inferSchema", "false")
.load(iso8601datesPath)
- val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd")
+ val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US)
val expectedDates = dates.collect().map { r =>
// This should be ISO8601 formatted string.
Row(iso8501.format(r.toSeq.head))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala
index 51832a13cf..c74406b9cb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala
@@ -144,13 +144,12 @@ class CSVTypeCastSuite extends SparkFunSuite {
DateTimeUtils.millisToDays(DateTimeUtils.stringToTime("2015-01-01").getTime))
}
- test("Float and Double Types are cast correctly with Locale") {
+ test("Float and Double Types are cast without respect to platform default Locale") {
val originalLocale = Locale.getDefault
try {
- val locale : Locale = new Locale("fr", "FR")
- Locale.setDefault(locale)
- assert(CSVTypeCast.castTo("1,00", FloatType) == 1.0)
- assert(CSVTypeCast.castTo("1,00", DoubleType) == 1.0)
+ Locale.setDefault(new Locale("fr", "FR"))
+ assert(CSVTypeCast.castTo("1,00", FloatType) == 100.0) // Would parse as 1.0 in fr-FR
+ assert(CSVTypeCast.castTo("1,00", DoubleType) == 100.0)
} finally {
Locale.setDefault(originalLocale)
}