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 4cbc45e033503..74bb78a227657 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -21,7 +21,8 @@ import java.{util => ju} import java.lang.{Long => JLong} import java.net.URLDecoder import java.nio.charset.StandardCharsets.UTF_8 -import java.text.SimpleDateFormat +import java.time.{Instant, ZoneId} +import java.time.format.DateTimeFormatter import java.util.{Date, Locale, TimeZone} import javax.servlet.http.HttpServletRequest import javax.ws.rs.core.{MediaType, MultivaluedMap, Response} @@ -42,15 +43,14 @@ private[spark] object UIUtils extends Logging { val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" val TABLE_CLASS_STRIPED_SORTABLE = TABLE_CLASS_STRIPED + " sortable" - // 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", Locale.US) - } + private val dateTimeFormatter = DateTimeFormatter + .ofPattern("yyyy/MM/dd HH:mm:ss", Locale.US) + .withZone(ZoneId.systemDefault()) - def formatDate(date: Date): String = dateFormat.get.format(date) + def formatDate(date: Date): String = dateTimeFormatter.format(date.toInstant) - def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + def formatDate(timestamp: Long): String = + dateTimeFormatter.format(Instant.ofEpochMilli(timestamp)) def formatDuration(milliseconds: Long): String = { if (milliseconds < 100) { @@ -124,16 +124,13 @@ private[spark] object UIUtils extends Logging { } } - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val batchTimeFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) - } + private val batchTimeFormat = DateTimeFormatter + .ofPattern("yyyy/MM/dd HH:mm:ss", Locale.US) + .withZone(ZoneId.systemDefault()) - private val batchTimeFormatWithMilliseconds = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS", Locale.US) - } + private val batchTimeFormatWithMilliseconds = DateTimeFormatter + .ofPattern("yyyy/MM/dd HH:mm:ss.SSS", Locale.US) + .withZone(ZoneId.systemDefault()) /** * If `batchInterval` is less than 1 second, format `batchTime` with milliseconds. Otherwise, @@ -150,19 +147,19 @@ private[spark] object UIUtils extends Logging { batchInterval: Long, showYYYYMMSS: Boolean = true, timezone: TimeZone = null): String = { - val oldTimezones = - (batchTimeFormat.get.getTimeZone, batchTimeFormatWithMilliseconds.get.getTimeZone) + val oldTimezones = (batchTimeFormat.getZone, batchTimeFormatWithMilliseconds.getZone) if (timezone != null) { - batchTimeFormat.get.setTimeZone(timezone) - batchTimeFormatWithMilliseconds.get.setTimeZone(timezone) + val zoneId = timezone.toZoneId + batchTimeFormat.withZone(zoneId) + batchTimeFormatWithMilliseconds.withZone(zoneId) } try { val formattedBatchTime = if (batchInterval < 1000) { - batchTimeFormatWithMilliseconds.get.format(batchTime) + batchTimeFormatWithMilliseconds.format(Instant.ofEpochMilli(batchTime)) } else { // If batchInterval >= 1 second, don't show milliseconds - batchTimeFormat.get.format(batchTime) + batchTimeFormat.format(Instant.ofEpochMilli(batchTime)) } if (showYYYYMMSS) { formattedBatchTime @@ -171,8 +168,8 @@ private[spark] object UIUtils extends Logging { } } finally { if (timezone != null) { - batchTimeFormat.get.setTimeZone(oldTimezones._1) - batchTimeFormatWithMilliseconds.get.setTimeZone(oldTimezones._2) + batchTimeFormat.withZone(oldTimezones._1) + batchTimeFormatWithMilliseconds.withZone(oldTimezones._2) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala index 88a110fa9a329..c031ec39511f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.streaming.ui -import java.text.SimpleDateFormat +import java.time.ZonedDateTime +import java.time.format.DateTimeFormatter -import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId private[ui] object UIUtils { @@ -62,15 +63,12 @@ private[ui] object UIUtils { } } - private val progressTimestampFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 - format.setTimeZone(getTimeZone("UTC")) - format - } - } + private val progressTimestampFormat = DateTimeFormatter + .ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + .withZone(getZoneId("UTC")) def parseProgressTimestamp(timestamp: String): Long = { - progressTimestampFormat.get.parse(timestamp).getTime + val zonedDateTime = ZonedDateTime.parse(timestamp, progressTimestampFormat) + zonedDateTime.toInstant.toEpochMilli } }