You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by zsxwing <gi...@git.apache.org> on 2014/03/19 14:50:07 UTC

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

GitHub user zsxwing opened a pull request:

    https://github.com/apache/spark/pull/179

    SPARK-1278: Fix improper use of SimpleDateFormat

    `SimpleDateFormat` is not immutable and thread-safe. Some places use the same SimpleDateFormat object without safeguard in the multiple threads. It will cause that the Web UI displays improper date.
    
    This PR creates `SimpleDateFormat` when it's necessary. Another solution is using `ThreadLocal` to store a `SimpleDateFormat` in each thread. If this PR impacts the performance, I can change to the latter one.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zsxwing/spark SPARK-1278

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/179.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #179
    
----
commit 5618092d6254c31629affd0107cb0f7ae5de6d5f
Author: zsxwing <zs...@gmail.com>
Date:   2014-03-19T07:40:40Z

    SPARK-1278: Fix improper use of SimpleDateFormat

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38076446
  
    Hey @zsxwing just wondering - what happens when this gets accessed from multiple threads. Is there an exception or anything?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38204176
  
    Jenkins, test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38214341
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13305/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761448
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala ---
    @@ -56,7 +56,8 @@ private[spark] class Worker(
       Utils.checkHost(host, "Expected hostname")
       assert (port > 0)
     
    -  val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For worker and executor IDs
    +  // For worker and executor IDs
    +  def dateFormat(): DateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
    --- End diff --
    
    Same as above in Master.scala


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761764
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala ---
    @@ -30,7 +31,7 @@ import org.apache.spark.util.Utils
     private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgressUI) {
     
       val listener = parent.listener
    -  val dateFmt = parent.dateFmt
    +  def dateFmt(): DateFormat = parent.dateFmt
    --- End diff --
    
    val dateFmt = WebUI.createDateFormat


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38133229
  
    > A single page/table cannot be access concurrently from multiple threads, so this is safe.
    
    I just checked `JobProgressUI`. Looks all of requests use the same page/table instance. An instance of `HttpServlet` will serve all of requests.
    
    > To make this easy we can add a utility function called createDateFormat inside of the WebUI object.
    
    Agree.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/179


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761567
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/WebUI.scala ---
    @@ -17,18 +17,18 @@
     
     package org.apache.spark.deploy
     
    -import java.text.SimpleDateFormat
    +import java.text.{DateFormat, SimpleDateFormat}
     import java.util.Date
     
     /**
      * Utilities used throughout the web UI.
      */
     private[spark] object DeployWebUI {
    -  val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    --- End diff --
    
    For this one let's add a single utility function called:
    
    ```
    def createDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    ```
    
    and let's remove the `formatDate(X)` functions. those aren't used anyways except in one place and it doesn't make much sense to have static helpers now that we have to use more localized date format objects.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10857905
  
    --- Diff: core/src/main/scala/org/apache/spark/util/FileLogger.scala ---
    @@ -111,7 +114,7 @@ class FileLogger(
       def log(msg: String, withTime: Boolean = false) {
         val writeInfo = if (!withTime) msg else {
           val date = new Date(System.currentTimeMillis())
    -      DATE_FORMAT.format(date) + ": " + msg
    +      dateFormat.get.format(date) + ": " + msg
    --- End diff --
    
    This one I don't think can be used from multiple threads. But I agree why not make it thread safe in case we ever do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10860446
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -23,12 +23,15 @@ import java.util.Date
     /**
      * Utilities used throughout the web UI.
      */
    -private[spark] object DeployWebUI {
    -  val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    +private[spark] object WebUI {
    +  // SimpleDateFormat is not thread-safe. Don't explore it to avoid improper use.
    --- End diff --
    
    Thanks for correcting the typo.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38133625
  
    Could we use `ThreadLocal` liek this:
    
    ```Scala
    private[spark] object WebUI {
    
      private val dateFormat = new ThreadLocal[SimpleDateFormat]() {
        override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
      }
    
      def getDateFormat = dateFormat.get()
    }
    ```
    Every thread will has its own `SimpleDateFormat`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761648
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/WebUI.scala ---
    @@ -17,18 +17,18 @@
     
     package org.apache.spark.deploy
     
    -import java.text.SimpleDateFormat
    +import java.text.{DateFormat, SimpleDateFormat}
     import java.util.Date
     
     /**
      * Utilities used throughout the web UI.
      */
     private[spark] object DeployWebUI {
    --- End diff --
    
    Do you mind moving this class to `org/apache/spark/ui` and renaming it to `object WebUI`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38134330
  
    @zsxwing we could do that too, but I thought it seemed a bit more complex/overkill for this. In particular because different components use different date formats and probably in the future we may want to use other date formats as well


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38208031
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761749
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala ---
    @@ -32,7 +33,7 @@ import org.apache.spark.util.{Utils, Distribution}
     /** Page showing statistics and task list for a given stage */
     private[spark] class StagePage(parent: JobProgressUI) {
       def listener = parent.listener
    -  val dateFmt = parent.dateFmt
    +  def dateFmt(): DateFormat = parent.dateFmt
    --- End diff --
    
    ```
    val dateFmt = WebUI.createDateFormat
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38141424
  
    Overrode the previous commit and updated to the latest master codes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38053337
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38116957
  
    @pwendell the thread-safety problem would manifest as just returning the wrong parsed or formatted date. The objects use instance fields as part of these processes. Have a look at `SimpleDateFormat.formatImpl` for example.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38084443
  
    Hey @zsxwing thanks for reporting this and proposing a fix!
    
    The way it is now has some performance implications because it creates a new date format every time one is needed. I'd suggest instead that each of the individual pages/tables should create a date format as a `val` and then re-use it. A single page/table cannot be access concurrently from multiple threads, so this is safe.
    
    To make this easy we can add a utility function called `createDateFormat` inside of the `WebUI` object.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10857873
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala ---
    @@ -55,7 +55,9 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener
       private val jobIdToPrintWriter = new HashMap[Int, PrintWriter]
       private val stageIdToJobId = new HashMap[Int, Int]
       private val jobIdToStageIds = new HashMap[Int, Seq[Int]]
    -  private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    +  private val dateFormat = new ThreadLocal[SimpleDateFormat]() {
    +    override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    --- End diff --
    
    think you can drop the first type declaration here. Not a big deal -I'll just clean it up when merging.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761429
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/master/Master.scala ---
    @@ -45,7 +45,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int,
     
       val conf = new SparkConf
     
    -  val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
    +  def dateFormat(): DateFormat = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
    --- End diff --
    
    I'd call this `createDateFormat` to make it clear it's generating one. Also you can drop the parens and return type:
    ```
    def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10857881
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -23,12 +23,15 @@ import java.util.Date
     /**
      * Utilities used throughout the web UI.
      */
    -private[spark] object DeployWebUI {
    -  val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    +private[spark] object WebUI {
    +  // SimpleDateFormat is not thread-safe. Don't explore it to avoid improper use.
    --- End diff --
    
    "expose" it? I can change this too


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38134429
  
    @zsxwing ah I see ya you are right about that (read your comments in reverse order). Maybe it's simplest to just use the thread local then.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38335726
  
    Thanks, I've merged this into master and did a somewhat hairy backport into 0.9


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38208029
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761323
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala ---
    @@ -320,7 +321,7 @@ private[spark] class Worker(
       }
     
       def generateWorkerId(): String = {
    -    "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), host, port)
    --- End diff --
    
    For this one I'd just create new date format inside of this function. This gets called very infrequently.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38214337
  
    Merged build finished.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761674
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala ---
    @@ -27,7 +29,7 @@ import org.apache.spark.util.Utils
     private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) {
     
       val listener = parent.listener
    -  val dateFmt = parent.dateFmt
    +  def dateFmt(): DateFormat = parent.dateFmt
    --- End diff --
    
    For this one I'd do this:
    
    ```
    val dateFmt = WebUI.createDateFormat
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/179#discussion_r10761741
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala ---
    @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils
     private[spark] class JobProgressUI(val sc: SparkContext) {
       private var _listener: Option[JobProgressListener] = None
       def listener = _listener.get
    -  val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    +  def dateFmt(): DateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
    --- End diff --
    
    I'd just remove the `dateFmt` field from this class. We should have each of the Table/Page classes create their own format object.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1278: Fix improper use of SimpleDateForm...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/179#issuecomment-38335726
  
    Thanks, I've merged this into master and did a somewhat hairy backport into 0.9


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---