You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by carsonwang <gi...@git.apache.org> on 2015/10/27 10:15:10 UTC

[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

GitHub user carsonwang opened a pull request:

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

    [SPARK-11206] Support SQL UI on the history server

    On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.
    
    To support SQL UI on the history server, I add an `onOtherEvent` method to the `SparkListener` trait and post all SQL related events to the same event bus. Two SQL events `SparkListenerSQLExecutionStart` and `SparkListenerSQLExecutionEnd` are added. 
    
    The history server, the standalone Master that rebuilds the web UI, and the event log listener which writes events to the storage are all in the core module. These components have to reference some necessary SQL classes like the SQL events, the SQLTab, etc. It will be best if we can make these components downstream of sql in the future. Currently I have to move these classes to core. 
    This change also make a single SQLTab for all `SQLContext` because they can now share a single `SQLListener`.

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

    $ git pull https://github.com/carsonwang/spark SqlHistoryUI

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

    https://github.com/apache/spark/pull/9297.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 #9297
    
----
commit fdf9d28362fe991a1df0d5392c9021db78fa7541
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-16T07:00:11Z

    Update SparkListener to handle other events

commit ff4075d89b480d99d0320390d00a5b10aadc9a93
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-20T06:06:07Z

    Write sql events to event log

commit b9870e6579a59628c82470e55f2cb6c4ec8fa2a7
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-21T06:29:37Z

    Move sql UI classes to core

commit 3833055b7c94247b700e6fd0565629e71611d307
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-21T06:32:28Z

    rename SqlMetricInfo class name

commit c0abfc6b7432750812a47790fe51b35dacba7429
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-22T02:22:51Z

    Update sql metric param

commit a5b1cf42a4847b2c9c9674ce0d6aa4d332498ca2
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-26T02:09:32Z

    handle accumulator updates in sql history UI

commit 7b30bc736f09600b25772e30636f8a5c19c6db5e
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-26T06:51:25Z

    Use a single SQL Tab for all SparkContext

commit d52288bb2e18a5f0e898110893a091919e13ea84
Author: Carson Wang <ca...@intel.com>
Date:   2015-10-27T08:49:30Z

    update style

----


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155389898
  
    **[Test build #45507 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45507/consoleFull)** for PR 9297 at commit [`51f913b`](https://github.com/apache/spark/commit/51f913bed91dd95029b404092e8cd7c25b02cad6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerEventRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151728574
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44493/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-152536961
  
    @vanzin  Thanks a lot for the comment. This sounds great and is very helpful. I agree it is not a good idea to move more stuff to the core. The underlying code change you mentioned sounds a lot of works to do, especially when we consider backwards compatibility. Can we apply part of them so that we can support SQL UI on the history server in this PR and also avoid moving the SQL classes to the core? I was thinking adding a `toJson` method to the `SparkListenerEvent`. As you mentioned when it is not a sealed trait, we can define sql events in the sql module. For existing events in the core, we still use the `JsonProtocol` in the `toJson` method to convert the event to json. For sql events, we can define `toJson` method in the sql module. This way we will be able to write the sql events to the log. Then we can use something you mentioned to feed events from the history server to the sql listener.
    
    I know this might still not be very scalable. But we don't make things worse once we are able to avoid moving sql classes to the core and also support SQL UI on the history server. Do you think this is doable for this ticket?


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156310081
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156336683
  
    **[Test build #45813 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45813/consoleFull)** for PR 9297 at commit [`fe5c165`](https://github.com/apache/spark/commit/fe5c16529d821098658cb61ca0f0a21d2a568270).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44622811
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/SparkUI.scala ---
    @@ -150,7 +151,14 @@ private[spark] object SparkUI {
           appName: String,
           basePath: String,
           startTime: Long): SparkUI = {
    -    create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime)
    +    val sparkUI = create(
    +      None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime)
    +    JsonProtocol.eventRegisters.foreach { eventRegister =>
    +      val listener = eventRegister.getListener()
    +      listenerBus.addListener(listener)
    +      eventRegister.attachUITab(listener, sparkUI)
    --- End diff --
    
    Yes. Ok, I changed to only attach the SQL tab when a `SparkListenerSQLExecutionStart` event is received for the first time.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44614968
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -96,6 +114,7 @@ private[spark] object JsonProtocol {
             executorMetricsUpdateToJson(metricsUpdate)
           case blockUpdated: SparkListenerBlockUpdated =>
             throw new MatchError(blockUpdated)  // TODO(ekl) implement this
    +      case _ => parse(write(event))
    --- End diff --
    
    I tried using the `JsonTypeInfo` annotation and it worked. The difference is that it need write the full class name now. The event will look like 
    ```
    "Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart"
    ```
    When deserializing the data, we use reflection to find the class so that the class list is not needed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154334960
  
    **[Test build #45203 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45203/consoleFull)** for PR 9297 at commit [`927bae8`](https://github.com/apache/spark/commit/927bae84244f900724c3865aac969e4639594760).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156289820
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45799/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r43098850
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -504,9 +542,28 @@ private[spark] object JsonProtocol {
           case `executorRemoved` => executorRemovedFromJson(json)
           case `logStart` => logStartFromJson(json)
           case `metricsUpdate` => executorMetricsUpdateFromJson(json)
    +      case `sqlExecutionStart` => sqlExecutionStartFromJson(json)
    +      case `sqlExecutionEnd` => sqlExecutionEndFromJson(json)
    --- End diff --
    
    This mach really needs `case other:String => throw new IOException(s"Unknown Event $other") message, so that when new events are added in future, the old replay code will at least display a meaningful message. Maybe that's for another pull req though.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155291520
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156049156
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45717/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154329615
  
    **[Test build #45201 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45201/consoleFull)** for PR 9297 at commit [`1954d71`](https://github.com/apache/spark/commit/1954d71d674dc35659cba6f9e3b56d42a756778e).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156296730
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151857883
  
    Hi @carsonwang,
    
    This is a really useful addition, but I'm afraid this approach is not really scalable. For example, you have the same problem with the Streaming tab in the UI, and trying to fix that with your approach would just make things worse.
    
    I actually think the underlying code needs some lower level changes before this can be implemented more cleanly. For example, a few things that I thought about in the past:
    
    - `SparkListenerEvent` should not be a sealed trait; that way streaming events and sql events can extend it without having to exist in core
    - `SparkListener` should not be the root of the listener hierarchy; `SparkFirehoseListener` should, and `SparkListener` should be a specialization of that trait that provides more type-specific method.
    - events should be written to the logs using a library like Jackson instead of having to modify `JsonProtocol` for each new event. The current approach is just not scalable. The main issue here is backwards compatibility, but I think this is a much needed change in this part of the code.
    
    With those in place, the only thing left is a protocol so that the history server can feed events to backend-specific listeners; that could be done by having the history server expose a new trait that is loaded at runtime using something like `java.util.ServiceLoader`; then there could be an implementation of that trait in sql (and another one in streaming) to process events read from the logs and add stuff to the UI.
    
    What do you think? I really would like to avoid moving more stuff to the core; it feels like this would be a pretty good opportunity to work on some of the outstanding issues with this whole part of the code.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154361230
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45201/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155260603
  
    **[Test build #45470 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45470/consoleFull)** for PR 9297 at commit [`51f913b`](https://github.com/apache/spark/commit/51f913bed91dd95029b404092e8cd7c25b02cad6).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44481472
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala ---
    @@ -0,0 +1,48 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.sql.execution.metric.SQLMetricInfo
    +import org.apache.spark.util.Utils
    +
    +/**
    + * :: DeveloperApi ::
    + * Stores information about a SQL SparkPlan.
    + */
    +@DeveloperApi
    +class SparkPlanInfo(
    +    val nodeName: String,
    +    val simpleString: String,
    +    val children: Seq[SparkPlanInfo],
    +    val metrics: Seq[SQLMetricInfo]
    +) {
    --- End diff --
    
    nit: brackets aren't necessary.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154934727
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156296867
  
    **[Test build #45808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45808/consoleFull)** for PR 9297 at commit [`fe5c165`](https://github.com/apache/spark/commit/fe5c16529d821098658cb61ca0f0a21d2a568270).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44481664
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala ---
    @@ -193,37 +224,41 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi
         }
       }
     
    -  def onExecutionStart(
    -      executionId: Long,
    -      description: String,
    -      details: String,
    -      physicalPlanDescription: String,
    -      physicalPlanGraph: SparkPlanGraph,
    -      time: Long): Unit = {
    -    val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node =>
    -      node.metrics.map(metric => metric.accumulatorId -> metric)
    -    }
    -
    -    val executionUIData = new SQLExecutionUIData(executionId, description, details,
    -      physicalPlanDescription, physicalPlanGraph, sqlPlanMetrics.toMap, time)
    -    synchronized {
    -      activeExecutions(executionId) = executionUIData
    -      _executionIdToData(executionId) = executionUIData
    -    }
    -  }
    -
    -  def onExecutionEnd(executionId: Long, time: Long): Unit = synchronized {
    -    _executionIdToData.get(executionId).foreach { executionUIData =>
    -      executionUIData.completionTime = Some(time)
    -      if (!executionUIData.hasRunningJobs) {
    -        // onExecutionEnd happens after all "onJobEnd"s
    -        // So we should update the execution lists.
    -        markExecutionFinished(executionId)
    -      } else {
    -        // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s.
    -        // Then we don't if the execution is successful, so let the last onJobEnd updates the
    -        // execution lists.
    -      }
    +  override def onOtherEvent(event: SparkListenerEvent): Unit = {
    +    event match {
    --- End diff --
    
    minor: you can move this to the previous line and avoid one level of indentation.
    
        ...: Unit = event match {



---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151735338
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156310107
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44475121
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -52,7 +55,22 @@ import org.apache.spark.storage._
     private[spark] object JsonProtocol {
       // TODO: Remove this file and put JSON serialization into each individual class.
     
    -  private implicit val format = DefaultFormats
    +  // Events defined in other modules are allowed to be registered through implementing the
    +  // SparkListenerEventRegister trait. These events are written to Json using Jackson.
    +  val eventRegisters: Iterable[SparkListenerEventRegister] = {
    +    val loader = Utils.getContextOrSparkClassLoader
    +    ServiceLoader.load(classOf[SparkListenerEventRegister], loader).asScala
    +  }
    +
    +  var eventClasses = new ListBuffer[Class[_]]()
    --- End diff --
    
    This can be a `val`. I still want to check how this stuff is used, though (haven't made it to that part of the patch yet).
    
        val eventClasses = eventRegisters.flatMap { r => eventRegister.getEventClasses() }.toSeq


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44481829
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/SparkUI.scala ---
    @@ -150,7 +151,14 @@ private[spark] object SparkUI {
           appName: String,
           basePath: String,
           startTime: Long): SparkUI = {
    -    create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime)
    +    val sparkUI = create(
    +      None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime)
    +    JsonProtocol.eventRegisters.foreach { eventRegister =>
    +      val listener = eventRegister.getListener()
    +      listenerBus.addListener(listener)
    +      eventRegister.attachUITab(listener, sparkUI)
    --- End diff --
    
    This doesn't look right. Won't this attach, e.g. the SQL tab to every application in the history server, regardless of whether the app used SQL or not?


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151728573
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154335597
  
    @vanzin , your suggestions are very helpful. I have updated the code following the ideas. I use Jackson to serialize the SQL events and also introduce a trait to register the events and feed them to the SQL listener. The SQL implementation of the trait is loaded at runtime using `java.util.ServiceLoader`. Can you please help take a look?


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151429245
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44419/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151765050
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44503/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151728071
  
    **[Test build #44493 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44493/consoleFull)** for PR 9297 at commit [`7a2aced`](https://github.com/apache/spark/commit/7a2acedfc524e5c5887bd783e6fbbe289313306a).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151730942
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154329162
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156289818
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151427044
  
    **[Test build #44419 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44419/consoleFull)** for PR 9297 at commit [`d52288b`](https://github.com/apache/spark/commit/d52288bb2e18a5f0e898110893a091919e13ea84).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156309871
  
    retest 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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156012583
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45713/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44711788
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala ---
    @@ -45,25 +46,14 @@ private[sql] object SQLExecution {
           sc.setLocalProperty(EXECUTION_ID_KEY, executionId.toString)
           val r = try {
             val callSite = Utils.getCallSite()
    -        sqlContext.listener.onExecutionStart(
    -          executionId,
    -          callSite.shortForm,
    -          callSite.longForm,
    -          queryExecution.toString,
    -          SparkPlanGraph(queryExecution.executedPlan),
    -          System.currentTimeMillis())
    +        sqlContext.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart(
    --- End diff --
    
    /cc @zsxwing @andrewor14 


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156016416
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45716/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156296483
  
    retest 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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156015418
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156287223
  
    Jenkins, retest 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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151733311
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151429238
  
    **[Test build #44419 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44419/consoleFull)** for PR 9297 at commit [`d52288b`](https://github.com/apache/spark/commit/d52288bb2e18a5f0e898110893a091919e13ea84).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156015443
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155390019
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45507/
    Test PASSed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44481407
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala ---
    @@ -45,25 +46,14 @@ private[sql] object SQLExecution {
           sc.setLocalProperty(EXECUTION_ID_KEY, executionId.toString)
           val r = try {
             val callSite = Utils.getCallSite()
    -        sqlContext.listener.onExecutionStart(
    -          executionId,
    -          callSite.shortForm,
    -          callSite.longForm,
    -          queryExecution.toString,
    -          SparkPlanGraph(queryExecution.executedPlan),
    -          System.currentTimeMillis())
    +        sqlContext.sparkContext.listenerBus.post(SparkListenerSQLExecutionStart(
    --- End diff --
    
    I'll summon @marmbrus to see whether the change in semantics here is ok. The events would now be processed asynchronously instead of in the same thread.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156012261
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155012831
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156287718
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154369514
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156049155
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151429243
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155260326
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151726946
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156336780
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45813/
    Test PASSed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154369445
  
    **[Test build #45203 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45203/consoleFull)** for PR 9297 at commit [`927bae8`](https://github.com/apache/spark/commit/927bae84244f900724c3865aac969e4639594760).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerEventRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151733309
  
    **[Test build #44499 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44499/consoleFull)** for PR 9297 at commit [`caab0ba`](https://github.com/apache/spark/commit/caab0bab0299d4eb985b2e5e68cc5813faac6dfb).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155348318
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155348070
  
    Jenkins, retest 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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151424041
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154329144
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156012580
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151423979
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154332991
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154361228
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44615239
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -96,6 +114,7 @@ private[spark] object JsonProtocol {
             executorMetricsUpdateToJson(metricsUpdate)
           case blockUpdated: SparkListenerBlockUpdated =>
             throw new MatchError(blockUpdated)  // TODO(ekl) implement this
    +      case _ => parse(write(event))
    --- End diff --
    
    Hi @steveloughran, I think currently the generic handling is only for new events added in the sub modules.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151732842
  
    **[Test build #44499 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44499/consoleFull)** for PR 9297 at commit [`caab0ba`](https://github.com/apache/spark/commit/caab0bab0299d4eb985b2e5e68cc5813faac6dfb).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155260310
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-152783164
  
    > I was thinking adding a toJson method to the SparkListenerEvent.
    
    I'm not a big fan of allowing events to write custom serialization code. I think a better approach would be to change `JsonProtocol` to use Jackson when the event doesn't match any of the currently known event types. Then later we could add a flag that makes `JsonProtocol` use Jackson for all event types.
    
    Mostly that's for backwards compatibility, so that event that are currently written to the event log don't change format. We can then document that the old format is deprecated and will be switched to the new, Jackson-based one in a future release.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155348295
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154333003
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156296702
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156311367
  
    **[Test build #45813 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45813/consoleFull)** for PR 9297 at commit [`fe5c165`](https://github.com/apache/spark/commit/fe5c16529d821098658cb61ca0f0a21d2a568270).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151728571
  
    **[Test build #44493 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44493/consoleFull)** for PR 9297 at commit [`7a2aced`](https://github.com/apache/spark/commit/7a2acedfc524e5c5887bd783e6fbbe289313306a).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151733313
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44499/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155291429
  
    **[Test build #45470 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45470/consoleFull)** for PR 9297 at commit [`51f913b`](https://github.com/apache/spark/commit/51f913bed91dd95029b404092e8cd7c25b02cad6).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156336778
  
    Merged build finished. Test PASSed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156014176
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155349864
  
    **[Test build #45507 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45507/consoleFull)** for PR 9297 at commit [`51f913b`](https://github.com/apache/spark/commit/51f913bed91dd95029b404092e8cd7c25b02cad6).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44481020
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -96,6 +114,7 @@ private[spark] object JsonProtocol {
             executorMetricsUpdateToJson(metricsUpdate)
           case blockUpdated: SparkListenerBlockUpdated =>
             throw new MatchError(blockUpdated)  // TODO(ekl) implement this
    +      case _ => parse(write(event))
    --- End diff --
    
    Hmm... I'm not sure I like this. This is why you need the class list above, when it would be much nicer to not need it. You can avoid the class list by using the Jackson API directly. For example:
    
        import com.fasterxml.jackson.annotation._
        import com.fasterxml.jackson.databind.ObjectMapper
        
        @JsonTypeInfo(use=JsonTypeInfo.Id.CLASS, include=JsonTypeInfo.As.PROPERTY, property="Event") class A { }
        
        class B(val bar: String, val baz: Option[String]) extends A
        
        object main {
          def main(args: Array[String]): Unit = {
            val mapper = new ObjectMapper()
            mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule)
        
            val b = new B("bar", None)
            val out = mapper.writeValueAsString(b)
        
            val read = mapper.readValue(out, classOf[A]).asInstanceOf[B]
            println(s"$read: ${read.bar} :: ${read.baz}")
          }
        }
        
    Basically, annotate the root event (`SparkEventListener`) with `JsonTypeInfo`, and use an `ObjectMapper` to generate and parse the JSON.
    
    I tried but couldn't make this work with `DefaultParams`. Setting just `typeHintFieldName` doesn't work, it seems you need `typeHints`, and for that you need a class list. It seems to just ignore the annotations.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r44556620
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -96,6 +114,7 @@ private[spark] object JsonProtocol {
             executorMetricsUpdateToJson(metricsUpdate)
           case blockUpdated: SparkListenerBlockUpdated =>
             throw new MatchError(blockUpdated)  // TODO(ekl) implement this
    +      case _ => parse(write(event))
    --- End diff --
    
    if some more generic handling of json marshalling is in there (nice), why is `SparkListenerBlockUpdated` being excluded? It should just be one of the types marshalled


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156287734
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156014202
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151735350
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151726937
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156296882
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151730887
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151765049
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156012575
  
    **[Test build #45713 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45713/consoleFull)** for PR 9297 at commit [`bca3f5f`](https://github.com/apache/spark/commit/bca3f5ffca9a9e6f5c56633812432892af66b4e7).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156016413
  
    Merged build finished. Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154934792
  
    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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151735661
  
    **[Test build #44503 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44503/consoleFull)** for PR 9297 at commit [`0af5afe`](https://github.com/apache/spark/commit/0af5afeafe894614e7c1cb83f343db0a0869ad77).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156012522
  
    **[Test build #45713 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45713/consoleFull)** for PR 9297 at commit [`bca3f5f`](https://github.com/apache/spark/commit/bca3f5ffca9a9e6f5c56633812432892af66b4e7).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-151764958
  
    **[Test build #44503 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44503/consoleFull)** for PR 9297 at commit [`0af5afe`](https://github.com/apache/spark/commit/0af5afeafe894614e7c1cb83f343db0a0869ad77).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154937833
  
    **[Test build #45346 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45346/consoleFull)** for PR 9297 at commit [`b03d98b`](https://github.com/apache/spark/commit/b03d98bcb6854a01b65fd5e43368580dec192482).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154369516
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45203/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#discussion_r43218812
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -504,9 +542,28 @@ private[spark] object JsonProtocol {
           case `executorRemoved` => executorRemovedFromJson(json)
           case `logStart` => logStartFromJson(json)
           case `metricsUpdate` => executorMetricsUpdateFromJson(json)
    +      case `sqlExecutionStart` => sqlExecutionStartFromJson(json)
    +      case `sqlExecutionEnd` => sqlExecutionEndFromJson(json)
    --- End diff --
    
    Added that. Thanks.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155291528
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45470/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155012699
  
    **[Test build #45346 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45346/consoleFull)** for PR 9297 at commit [`b03d98b`](https://github.com/apache/spark/commit/b03d98bcb6854a01b65fd5e43368580dec192482).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerEventRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156339821
  
    Hi @vanzin, I updated the code to address your comments. Thank you!


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-155390016
  
    Merged build finished. Test PASSed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156012250
  
     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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156296885
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/45808/
    Test FAILed.


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156049059
  
    **[Test build #45717 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45717/consoleFull)** for PR 9297 at commit [`fe5c165`](https://github.com/apache/spark/commit/fe5c16529d821098658cb61ca0f0a21d2a568270).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-156016477
  
    **[Test build #45717 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45717/consoleFull)** for PR 9297 at commit [`fe5c165`](https://github.com/apache/spark/commit/fe5c16529d821098658cb61ca0f0a21d2a568270).


---
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.
---

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


[GitHub] spark pull request: [SPARK-11206] Support SQL UI on the history se...

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

    https://github.com/apache/spark/pull/9297#issuecomment-154361012
  
    **[Test build #45201 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45201/consoleFull)** for PR 9297 at commit [`1954d71`](https://github.com/apache/spark/commit/1954d71d674dc35659cba6f9e3b56d42a756778e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait SparkListenerEvent `\n  * `trait SparkListenerEventRegister `\n  * `class SparkPlanInfo(`\n  * `class SQLMetricInfo(`\n  * `case class SparkListenerSQLExecutionStart(`\n  * `case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long)`\n


---
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.
---

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