aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala17
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala6
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala24
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala6
-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
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala9
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala4
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala3
16 files changed, 62 insertions, 59 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 7ab68a13e0..67c078ae5e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import java.text.SimpleDateFormat
-import java.util.{Calendar, TimeZone}
+import java.util.{Calendar, Locale, TimeZone}
import scala.util.Try
@@ -331,7 +331,7 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx
override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType)
override protected def nullSafeEval(timestamp: Any, format: Any): Any = {
- val sdf = new SimpleDateFormat(format.toString)
+ val sdf = new SimpleDateFormat(format.toString, Locale.US)
UTF8String.fromString(sdf.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000)))
}
@@ -400,7 +400,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes {
private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String]
private lazy val formatter: SimpleDateFormat =
- Try(new SimpleDateFormat(constFormat.toString)).getOrElse(null)
+ Try(new SimpleDateFormat(constFormat.toString, Locale.US)).getOrElse(null)
override def eval(input: InternalRow): Any = {
val t = left.eval(input)
@@ -425,7 +425,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes {
null
} else {
val formatString = f.asInstanceOf[UTF8String].toString
- Try(new SimpleDateFormat(formatString).parse(
+ Try(new SimpleDateFormat(formatString, Locale.US).parse(
t.asInstanceOf[UTF8String].toString).getTime / 1000L).getOrElse(null)
}
}
@@ -520,7 +520,7 @@ case class FromUnixTime(sec: Expression, format: Expression)
private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String]
private lazy val formatter: SimpleDateFormat =
- Try(new SimpleDateFormat(constFormat.toString)).getOrElse(null)
+ Try(new SimpleDateFormat(constFormat.toString, Locale.US)).getOrElse(null)
override def eval(input: InternalRow): Any = {
val time = left.eval(input)
@@ -539,9 +539,10 @@ case class FromUnixTime(sec: Expression, format: Expression)
if (f == null) {
null
} else {
- Try(UTF8String.fromString(new SimpleDateFormat(
- f.asInstanceOf[UTF8String].toString).format(new java.util.Date(
- time.asInstanceOf[Long] * 1000L)))).getOrElse(null)
+ Try(
+ UTF8String.fromString(new SimpleDateFormat(f.toString, Locale.US).
+ format(new java.util.Date(time.asInstanceOf[Long] * 1000L)))
+ ).getOrElse(null)
}
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 1bcbb6cfc9..25a5e3fd7d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -1415,7 +1415,7 @@ case class Sentences(
val locale = if (languageStr != null && countryStr != null) {
new Locale(languageStr.toString, countryStr.toString)
} else {
- Locale.getDefault
+ Locale.US
}
getSentences(string.asInstanceOf[UTF8String].toString, locale)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
index aec18922ea..c45970658c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.json
+import java.util.Locale
+
import com.fasterxml.jackson.core.{JsonFactory, JsonParser}
import org.apache.commons.lang3.time.FastDateFormat
@@ -56,11 +58,11 @@ private[sql] class JSONOptions(
// 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)
// Parse mode flags
if (!ParseModes.isValidMode(parseMode)) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 0b643a5b84..235ca8d263 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util
import java.sql.{Date, Timestamp}
import java.text.{DateFormat, SimpleDateFormat}
-import java.util.{Calendar, TimeZone}
+import java.util.{Calendar, Locale, TimeZone}
import javax.xml.bind.DatatypeConverter
import scala.annotation.tailrec
@@ -79,14 +79,14 @@ object DateTimeUtils {
// `SimpleDateFormat` is not thread-safe.
val threadLocalTimestampFormat = new ThreadLocal[DateFormat] {
override def initialValue(): SimpleDateFormat = {
- new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
+ new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
}
}
// `SimpleDateFormat` is not thread-safe.
private val threadLocalDateFormat = new ThreadLocal[DateFormat] {
override def initialValue(): SimpleDateFormat = {
- new SimpleDateFormat("yyyy-MM-dd")
+ new SimpleDateFormat("yyyy-MM-dd", Locale.US)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 6118a34d29..35cea25ba0 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
-import java.util.Calendar
+import java.util.{Calendar, Locale}
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.util.DateTimeUtils
@@ -30,8 +30,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
import IntegralLiteralTestUtils._
- 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-11-08 13:10:15").getTime)
@@ -49,7 +49,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("DayOfYear") {
- val sdfDay = new SimpleDateFormat("D")
+ val sdfDay = new SimpleDateFormat("D", Locale.US)
(0 to 3).foreach { m =>
(0 to 5).foreach { i =>
val c = Calendar.getInstance()
@@ -411,9 +411,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
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)
checkEvaluation(
FromUnixTime(Literal(0L), Literal("yyyy-MM-dd HH:mm:ss")), sdf1.format(new Timestamp(0)))
checkEvaluation(FromUnixTime(
@@ -430,11 +430,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("unix_timestamp") {
- 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"
- val sdf3 = new SimpleDateFormat(fmt3)
+ val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
val date1 = Date.valueOf("2015-07-24")
checkEvaluation(
UnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss")), 0L)
@@ -466,11 +466,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
test("to_unix_timestamp") {
- 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"
- val sdf3 = new SimpleDateFormat(fmt3)
+ val sdf3 = new SimpleDateFormat(fmt3, Locale.US)
val date1 = Date.valueOf("2015-07-24")
checkEvaluation(
ToUnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss")), 0L)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
index 4f516d0064..e0a9a0c3d5 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util
import java.sql.{Date, Timestamp}
import java.text.SimpleDateFormat
-import java.util.{Calendar, TimeZone}
+import java.util.{Calendar, Locale, TimeZone}
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.util.DateTimeUtils._
@@ -68,8 +68,8 @@ class DateTimeUtilsSuite extends SparkFunSuite {
assert(d2.toString === d1.toString)
}
- val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss")
- val df2 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z")
+ val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
+ val df2 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z", Locale.US)
checkFromToJavaDate(new Date(100))
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)
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index 2843100fb3..05164d774c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -20,9 +20,7 @@ package org.apache.spark.sql.hive.execution
import java.io.IOException
import java.net.URI
import java.text.SimpleDateFormat
-import java.util.{Date, Random}
-
-import scala.collection.JavaConverters._
+import java.util.{Date, Locale, Random}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
@@ -60,9 +58,8 @@ case class InsertIntoHiveTable(
private def executionId: String = {
val rand: Random = new Random
- val format: SimpleDateFormat = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS")
- val executionId: String = "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong)
- return executionId
+ val format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS", Locale.US)
+ "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong)
}
private def getStagingDir(inputPath: Path, hadoopConf: Configuration): Path = {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
index ea88276bb9..e53c3e4d48 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.hive
import java.text.NumberFormat
-import java.util.Date
+import java.util.{Date, Locale}
import scala.collection.JavaConverters._
@@ -95,7 +95,7 @@ private[hive] class SparkHiveWriterContainer(
}
protected def getOutputName: String = {
- val numberFormat = NumberFormat.getInstance()
+ val numberFormat = NumberFormat.getInstance(Locale.US)
numberFormat.setMinimumIntegerDigits(5)
numberFormat.setGroupingUsed(false)
val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
index 64d0ecbeef..cecfd99098 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.sources
import java.text.NumberFormat
+import java.util.Locale
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
@@ -141,7 +142,7 @@ class SimpleTextOutputWriter(path: String, context: TaskAttemptContext)
class AppendingTextOutputFormat(path: String) extends TextOutputFormat[NullWritable, Text] {
- val numberFormat = NumberFormat.getInstance()
+ val numberFormat = NumberFormat.getInstance(Locale.US)
numberFormat.setMinimumIntegerDigits(5)
numberFormat.setGroupingUsed(false)