You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/02/28 08:07:21 UTC

[GitHub] [spark] zsxwing opened a new pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

zsxwing opened a new pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732
 
 
   ### What changes were proposed in this pull request?
   
   - Add a UI test for Structured Streaming UI
   - Fix the unsafe usages of `SimpleDateFormat` by using a ThreadLocal shared object.
   - Use `start` to replace `submission` to be consistent with the API `StreamingQuery.start()`.
   
   ### Why are the changes needed?
   
   Structured Streaming UI is missing UI tests.
   
   ### Does this PR introduce any user-facing change?
   
   No
   
   ### How was this patch tested?
   
   The new test.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r386840112
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
 ##########
 @@ -82,15 +82,15 @@ object StreamingQueryListener {
    * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`.
    * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
    * @param name User-specified name of the query, null if not specified.
-   * @param submissionTime The timestamp to start a query.
+   * @param timestamp The timestamp to start a query.
    * @since 2.1.0
    */
   @Evolving
   class QueryStartedEvent private[sql](
       val id: UUID,
       val runId: UUID,
       val name: String,
-      val submissionTime: Long) extends Event
+      val timestamp: String) extends Event
 
 Review comment:
   Yep, totally agreed that `Long` is better for coding. However, `StreamingQueryProgress#timestamp` was designed to be human readable. In addition, its type cannot be changed now because that's a public API. Since the user has already had codes to parse `StreamingQueryProgress.timestamp`, it should be fine to add a field with the same format.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xuanyuanking commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387409951
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala
 ##########
 @@ -57,4 +62,16 @@ private[ui] object UIUtils {
       query.exception.map(_ => "FAILED").getOrElse("FINISHED")
     }
   }
+
+  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"))
 
 Review comment:
   I also checked this during the review, this timestamp string generated internally at
   
   https://github.com/apache/spark/blob/380e8876316d6ef5a74358be2a04ab20e8b6e7ca/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala#L91-L92
   
   So here is just parse back.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592725082
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119100/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592399816
 
 
   **[Test build #119077 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119077/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592729846
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592400358
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387209148
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala
 ##########
 @@ -57,4 +62,16 @@ private[ui] object UIUtils {
       query.exception.map(_ => "FAILED").getOrElse("FINISHED")
     }
   }
+
+  private val progressTimestampFormat = new ThreadLocal[SimpleDateFormat]() {
+    override def initialValue(): SimpleDateFormat = {
+      val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
 
 Review comment:
   This is to use the same formatter as https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala#L91

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592663765
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592665515
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592729277
 
 
   **[Test build #119103 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119103/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387042224
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
 ##########
 @@ -349,7 +349,7 @@ trait ProgressReporter extends Logging {
     result
   }
 
-  private def formatTimestamp(millis: Long): String = {
+  protected def formatTimestamp(millis: Long): String = {
     timestampFormat.format(new Date(millis))
 
 Review comment:
   Not related to this PR, just want to confirm which timezone we expect.
   
   The code here uses the JVM system timezone, while in Spark SQL we use the configured session local timezone.
   
   cc @MaxGekk as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592664902
 
 
   **[Test build #119100 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119100/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592725082
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119100/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387043021
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
 ##########
 @@ -349,7 +349,7 @@ trait ProgressReporter extends Logging {
     result
   }
 
-  private def formatTimestamp(millis: Long): String = {
+  protected def formatTimestamp(millis: Long): String = {
     timestampFormat.format(new Date(millis))
 
 Review comment:
   This is also different from the previous code which use UTC: https://github.com/apache/spark/pull/27732/files#diff-3ff3668ba061d3b690c7bfe9e8f0db32L35

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592726399
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r386804112
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
 ##########
 @@ -82,15 +82,15 @@ object StreamingQueryListener {
    * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`.
    * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
    * @param name User-specified name of the query, null if not specified.
-   * @param submissionTime The timestamp to start a query.
+   * @param timestamp The timestamp to start a query.
    * @since 2.1.0
    */
   @Evolving
   class QueryStartedEvent private[sql](
       val id: UUID,
       val runId: UUID,
       val name: String,
-      val submissionTime: Long) extends Event
+      val timestamp: String) extends Event
 
 Review comment:
   `timestamp` is not used without parsing and the parsed result is the same as [startTimestamp](https://github.com/apache/spark/pull/27732/files#diff-6532dd3b63bdab0364fbcf2303e290e4R310). So `formatTimestamp(startTimestamp)` seems a little bit meaningless.
   I understand it's for consistency but how about rather changing the type of `StreamingQueryProgress#timestamp` to `Long` for consistency?
   Almost all the use of `StreamingQueryProgress#timestamp` is with parsing so could we reduce the parsing code?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r385556396
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
 ##########
 @@ -82,15 +82,15 @@ object StreamingQueryListener {
    * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`.
    * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
    * @param name User-specified name of the query, null if not specified.
-   * @param submissionTime The timestamp to start a query.
+   * @param timestamp The timestamp to start a query.
    * @since 2.1.0
    */
   @Evolving
   class QueryStartedEvent private[sql](
       val id: UUID,
       val runId: UUID,
       val name: String,
-      val submissionTime: Long) extends Event
+      val timestamp: String) extends Event
 
 Review comment:
   Change this to `string` type to be consistent with timestamps in `StreamingQueryProgress`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387123553
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala
 ##########
 @@ -57,4 +62,16 @@ private[ui] object UIUtils {
       query.exception.map(_ => "FAILED").getOrElse("FINISHED")
     }
   }
+
+  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"))
 
 Review comment:
   I think so. It's not for parsing but formatting, and it's always UTC, so putting `Z` at the end is corrected.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387069903
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala
 ##########
 @@ -57,4 +62,16 @@ private[ui] object UIUtils {
       query.exception.map(_ => "FAILED").getOrElse("FINISHED")
     }
   }
+
+  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"))
 
 Review comment:
   This is useless settings because you require a time zone in the inputs:
   ```scala
   scala> val sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'")
   sdf: java.text.SimpleDateFormat = java.text.SimpleDateFormat@5af7aed5
   
   scala> sdf.parse
   parse   parseObject
   
   scala> sdf.parse("2020-03-03T11:12:13.123")
   java.text.ParseException: Unparseable date: "2020-03-03T11:12:13.123"
     at java.text.DateFormat.parse(DateFormat.java:366)
     ... 47 elided
   
   scala> sdf.parse("2020-03-03T11:12:13.123Z")
   res1: java.util.Date = Tue Mar 03 11:12:13 MSK 2020
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592477721
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592400370
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23821/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592729856
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23846/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592400370
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23821/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592725074
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592729846
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592725074
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592477730
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119077/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592477730
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119077/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387066658
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala
 ##########
 @@ -57,4 +62,16 @@ private[ui] object UIUtils {
       query.exception.map(_ => "FAILED").getOrElse("FINISHED")
     }
   }
+
+  private val progressTimestampFormat = new ThreadLocal[SimpleDateFormat]() {
+    override def initialValue(): SimpleDateFormat = {
+      val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
 
 Review comment:
   Any particular reasons for using SimpleDateFormat instead of DateTimeFormatter? Just in case, would it possible to use `TimestampFormatter` here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387043021
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
 ##########
 @@ -349,7 +349,7 @@ trait ProgressReporter extends Logging {
     result
   }
 
-  private def formatTimestamp(millis: Long): String = {
+  protected def formatTimestamp(millis: Long): String = {
     timestampFormat.format(new Date(millis))
 
 Review comment:
   This is also different from the previous code which use UTC: https://github.com/apache/spark/pull/27732/files#diff-3ff3668ba061d3b690c7bfe9e8f0db32L35

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387047680
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
 ##########
 @@ -82,15 +82,15 @@ object StreamingQueryListener {
    * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`.
    * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
    * @param name User-specified name of the query, null if not specified.
-   * @param submissionTime The timestamp to start a query.
+   * @param timestamp The timestamp to start a query.
 
 Review comment:
   nvm, we put `Z` at the end, to indicate it's UTC.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592781684
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119103/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592729856
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23846/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592476997
 
 
   **[Test build #119077 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119077/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).
    * This patch **fails SparkR unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592664902
 
 
   **[Test build #119100 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119100/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zsxwing commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
zsxwing commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592399594
 
 
   cc @uncleGen @xuanyuanking @cloud-fan 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] gatorsmile commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-593617657
 
 
   cc @xuanyuanking @gengliangwang @sarutak 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sarutak commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
sarutak commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r386853912
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
 ##########
 @@ -82,15 +82,15 @@ object StreamingQueryListener {
    * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`.
    * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
    * @param name User-specified name of the query, null if not specified.
-   * @param submissionTime The timestamp to start a query.
+   * @param timestamp The timestamp to start a query.
    * @since 2.1.0
    */
   @Evolving
   class QueryStartedEvent private[sql](
       val id: UUID,
       val runId: UUID,
       val name: String,
-      val submissionTime: Long) extends Event
+      val timestamp: String) extends Event
 
 Review comment:
   Ah, It's a public API. O.K, I understand that it's reasonable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387046209
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala
 ##########
 @@ -82,15 +82,15 @@ object StreamingQueryListener {
    * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`.
    * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`.
    * @param name User-specified name of the query, null if not specified.
-   * @param submissionTime The timestamp to start a query.
+   * @param timestamp The timestamp to start a query.
 
 Review comment:
   shall we mention that this is a timestamp string assuming the timezone is UTC? Or do we put the timezone at the end of the timestamp string?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592781684
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119103/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592781210
 
 
   **[Test build #119103 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119103/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592781678
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592724657
 
 
   **[Test build #119100 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119100/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592399816
 
 
   **[Test build #119077 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119077/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592781678
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387075211
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala
 ##########
 @@ -57,4 +62,16 @@ private[ui] object UIUtils {
       query.exception.map(_ => "FAILED").getOrElse("FINISHED")
     }
   }
+
+  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"))
 
 Review comment:
   Sorry, I was confused. Actually, 'Z' at the end of the pattern is not a place holder for time zone. Did you do that intentionally?  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592400358
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592665523
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23844/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592665515
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592477721
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592665523
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23844/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#discussion_r387042224
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala
 ##########
 @@ -349,7 +349,7 @@ trait ProgressReporter extends Logging {
     result
   }
 
-  private def formatTimestamp(millis: Long): String = {
+  protected def formatTimestamp(millis: Long): String = {
     timestampFormat.format(new Date(millis))
 
 Review comment:
   Not related to this PR, just want to confirm which timezone we expect.
   
   The code here uses the JVM system timezone, while in Spark SQL we use the configured session local timezone.
   
   cc @MaxGekk as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-592729277
 
 
   **[Test build #119103 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119103/testReport)** for PR 27732 at commit [`1479502`](https://github.com/apache/spark/commit/14795023e1abd1d79cc4b5a2c3ea011304cb1b4c).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27732: [SPARK-30984][SS]Add UI test for Structured Streaming UI
URL: https://github.com/apache/spark/pull/27732#issuecomment-594341571
 
 
   thanks, merging to master/3.0!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org