aboutsummaryrefslogtreecommitdiff
path: root/core/src
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 /core/src
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 'core/src')
-rw-r--r--core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala5
-rwxr-xr-xcore/src/main/scala/org/apache/spark/deploy/worker/Worker.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/UIUtils.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala6
-rw-r--r--core/src/test/scala/org/apache/spark/util/UtilsSuite.scala2
12 files changed, 29 insertions, 26 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index 6550d703bc..7f75a393bf 100644
--- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -20,7 +20,7 @@ package org.apache.spark
import java.io.IOException
import java.text.NumberFormat
import java.text.SimpleDateFormat
-import java.util.Date
+import java.util.{Date, Locale}
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.fs.Path
@@ -67,12 +67,12 @@ class SparkHadoopWriter(jobConf: JobConf) extends Logging with Serializable {
def setup(jobid: Int, splitid: Int, attemptid: Int) {
setIDs(jobid, splitid, attemptid)
- HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss").format(now),
+ HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(now),
jobid, splitID, attemptID, conf.value)
}
def open() {
- val numfmt = NumberFormat.getInstance()
+ val numfmt = NumberFormat.getInstance(Locale.US)
numfmt.setMinimumIntegerDigits(5)
numfmt.setGroupingUsed(false)
@@ -162,7 +162,7 @@ class SparkHadoopWriter(jobConf: JobConf) extends Logging with Serializable {
private[spark]
object SparkHadoopWriter {
def createJobID(time: Date, id: Int): JobID = {
- val formatter = new SimpleDateFormat("yyyyMMddHHmmss")
+ val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
val jobtrackerID = formatter.format(time)
new JobID(jobtrackerID, id)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 3f54ecc17a..23156072c3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -21,7 +21,7 @@ import java.io.IOException
import java.lang.reflect.Method
import java.security.PrivilegedExceptionAction
import java.text.DateFormat
-import java.util.{Arrays, Comparator, Date}
+import java.util.{Arrays, Comparator, Date, Locale}
import scala.collection.JavaConverters._
import scala.util.control.NonFatal
@@ -357,7 +357,7 @@ class SparkHadoopUtil extends Logging {
* @return a printable string value.
*/
private[spark] def tokenToString(token: Token[_ <: TokenIdentifier]): String = {
- val df = DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.SHORT)
+ val df = DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.SHORT, Locale.US)
val buffer = new StringBuilder(128)
buffer.append(token.toString)
try {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 8c91aa1516..4618e6117a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -18,7 +18,7 @@
package org.apache.spark.deploy.master
import java.text.SimpleDateFormat
-import java.util.Date
+import java.util.{Date, Locale}
import java.util.concurrent.{ScheduledFuture, TimeUnit}
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
@@ -51,7 +51,8 @@ private[deploy] class Master(
private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
- private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
+ // For application IDs
+ private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
private val WORKER_TIMEOUT_MS = conf.getLong("spark.worker.timeout", 60) * 1000
private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 0bedd9a20a..8b1c6bf2e5 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker
import java.io.File
import java.io.IOException
import java.text.SimpleDateFormat
-import java.util.{Date, UUID}
+import java.util.{Date, Locale, UUID}
import java.util.concurrent._
import java.util.concurrent.{Future => JFuture, ScheduledFuture => JScheduledFuture}
@@ -68,7 +68,7 @@ private[deploy] class Worker(
ThreadUtils.newDaemonSingleThreadExecutor("worker-cleanup-thread"))
// For worker and executor IDs
- private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
+ private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
// Send a heartbeat every (heartbeat timeout) / 4 milliseconds
private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index e1cf3938de..36a2f5c87e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -19,7 +19,7 @@ package org.apache.spark.rdd
import java.io.IOException
import java.text.SimpleDateFormat
-import java.util.Date
+import java.util.{Date, Locale}
import scala.collection.immutable.Map
import scala.reflect.ClassTag
@@ -243,7 +243,8 @@ class HadoopRDD[K, V](
var reader: RecordReader[K, V] = null
val inputFormat = getInputFormat(jobConf)
- HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss").format(createTime),
+ HadoopRDD.addLocalConfiguration(
+ new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(createTime),
context.stageId, theSplit.index, context.attemptNumber, jobConf)
reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index baf31fb658..488e777fea 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -19,7 +19,7 @@ package org.apache.spark.rdd
import java.io.IOException
import java.text.SimpleDateFormat
-import java.util.Date
+import java.util.{Date, Locale}
import scala.reflect.ClassTag
@@ -79,7 +79,7 @@ class NewHadoopRDD[K, V](
// private val serializableConf = new SerializableWritable(_conf)
private val jobTrackerId: String = {
- val formatter = new SimpleDateFormat("yyyyMMddHHmmss")
+ val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
formatter.format(new Date())
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 068f4ed8ad..67baad1c51 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -19,7 +19,7 @@ package org.apache.spark.rdd
import java.nio.ByteBuffer
import java.text.SimpleDateFormat
-import java.util.{Date, HashMap => JHashMap}
+import java.util.{Date, HashMap => JHashMap, Locale}
import scala.collection.{mutable, Map}
import scala.collection.JavaConverters._
@@ -1079,7 +1079,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
// Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038).
val hadoopConf = conf
val job = NewAPIHadoopJob.getInstance(hadoopConf)
- val formatter = new SimpleDateFormat("yyyyMMddHHmmss")
+ val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
val jobtrackerID = formatter.format(new Date())
val stageId = self.id
val jobConfiguration = job.getConfiguration
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
index f6a9f9c557..76af33c1a1 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
@@ -21,7 +21,7 @@ import java.lang.annotation.Annotation
import java.lang.reflect.Type
import java.nio.charset.StandardCharsets
import java.text.SimpleDateFormat
-import java.util.{Calendar, SimpleTimeZone}
+import java.util.{Calendar, Locale, SimpleTimeZone}
import javax.ws.rs.Produces
import javax.ws.rs.core.{MediaType, MultivaluedMap}
import javax.ws.rs.ext.{MessageBodyWriter, Provider}
@@ -86,7 +86,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{
private[spark] object JacksonMessageWriter {
def makeISODateFormat: SimpleDateFormat = {
- val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'")
+ val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'", Locale.US)
val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT"))
iso8601.setCalendar(cal)
iso8601
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
index 0c71cd2382..d8d5e8958b 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
@@ -17,7 +17,7 @@
package org.apache.spark.status.api.v1
import java.text.{ParseException, SimpleDateFormat}
-import java.util.TimeZone
+import java.util.{Locale, TimeZone}
import javax.ws.rs.WebApplicationException
import javax.ws.rs.core.Response
import javax.ws.rs.core.Response.Status
@@ -25,12 +25,12 @@ import javax.ws.rs.core.Response.Status
private[v1] class SimpleDateParam(val originalValue: String) {
val timestamp: Long = {
- val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz")
+ val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz", Locale.US)
try {
format.parse(originalValue).getTime()
} catch {
case _: ParseException =>
- val gmtDay = new SimpleDateFormat("yyyy-MM-dd")
+ val gmtDay = new SimpleDateFormat("yyyy-MM-dd", Locale.US)
gmtDay.setTimeZone(TimeZone.getTimeZone("GMT"))
try {
gmtDay.parse(originalValue).getTime()
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index c0d1a2220f..66b097aa81 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -36,7 +36,8 @@ private[spark] object UIUtils extends Logging {
// SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use.
private val dateFormat = new ThreadLocal[SimpleDateFormat]() {
- override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
+ override def initialValue(): SimpleDateFormat =
+ new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US)
}
def formatDate(date: Date): String = dateFormat.get.format(date)
diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala
index 5c4238c038..1f263df57c 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala
@@ -18,7 +18,7 @@
package org.apache.spark.util.logging
import java.text.SimpleDateFormat
-import java.util.Calendar
+import java.util.{Calendar, Locale}
import org.apache.spark.internal.Logging
@@ -59,7 +59,7 @@ private[spark] class TimeBasedRollingPolicy(
}
@volatile private var nextRolloverTime = calculateNextRolloverTime()
- private val formatter = new SimpleDateFormat(rollingFileSuffixPattern)
+ private val formatter = new SimpleDateFormat(rollingFileSuffixPattern, Locale.US)
/** Should rollover if current time has exceeded next rollover time */
def shouldRollover(bytesToBeWritten: Long): Boolean = {
@@ -109,7 +109,7 @@ private[spark] class SizeBasedRollingPolicy(
}
@volatile private var bytesWrittenSinceRollover = 0L
- val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS")
+ val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS", Locale.US)
/** Should rollover if the next set of bytes is going to exceed the size limit */
def shouldRollover(bytesToBeWritten: Long): Boolean = {
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 15ef32f21d..feacfb7642 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -264,7 +264,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
val hour = minute * 60
def str: (Long) => String = Utils.msDurationToString(_)
- val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator()
+ val sep = new DecimalFormatSymbols(Locale.US).getDecimalSeparator
assert(str(123) === "123 ms")
assert(str(second) === "1" + sep + "0 s")