You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by zhichao-li <gi...@git.apache.org> on 2015/02/28 07:53:24 UTC

[GitHub] spark pull request: [SPARK-6077] update listener for the existing ...

GitHub user zhichao-li opened a pull request:

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

    [SPARK-6077] update listener for the existing streamingTab instead of adding an one new one

    Currently we would create a new streaming tab for each streamingContext even if there's already one on the same sparkContext which would cause duplicate StreamingTab created and none of them is taking effect.
    snapshot: https://www.dropbox.com/s/t4gd6hqyqo0nivz/bad%20multiple%20streamings.png?dl=0
    How to reproduce:
    1)
    import org.apache.spark.SparkConf
    import org.apache.spark.streaming.
    {Seconds, StreamingContext}
    import org.apache.spark.storage.StorageLevel
    val ssc = new StreamingContext(sc, Seconds(1))
    val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER)
    val words = lines.flatMap(_.split(" "))
    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
    wordCounts.print()
    ssc.start()
    .....
    2)
    ssc.stop(false)
    val ssc = new StreamingContext(sc, Seconds(1))
    val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER)
    val words = lines.flatMap(_.split(" "))
    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
    wordCounts.print()
    ssc.start()

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

    $ git pull https://github.com/zhichao-li/spark master

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

    https://github.com/apache/spark/pull/4828.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 #4828
    
----
commit a711920518fe6c0c569d490a2c6ee33cdb1f14c0
Author: lisurprise <zh...@intel.com>
Date:   2015-02-28T06:34:31Z

    update listener for the existing streamingTab instead of adding one new tab

----


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25979619
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -45,6 +46,7 @@ private[spark] abstract class WebUI(
     
       protected val tabs = ArrayBuffer[WebUITab]()
       protected val handlers = ArrayBuffer[ServletContextHandler]()
    +  protected val page2Handlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]]
    --- End diff --
    
    We dont use the idiom "to --> 2" in our code. `pageToHandlers` is better.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78434513
  
      [Test build #28500 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28500/consoleFull) for   PR 4828 at commit [`51e6c7f`](https://github.com/apache/spark/commit/51e6c7febc0bc63721f497712580df6f956048f2).
     * This patch **passes all 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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78428503
  
      [Test build #28500 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28500/consoleFull) for   PR 4828 at commit [`51e6c7f`](https://github.com/apache/spark/commit/51e6c7febc0bc63721f497712580df6f956048f2).
     * This patch merges cleanly.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78429427
  
    Oh wait. I just realized that there is a already a unit test in org.apace.spark.streaming.UISuite which tested whether the streaming tab works. But thats ignored too. Could you update that testsuite to use selenium and test whether the streaming tab is attached on and detached correctly. I just realized that the unit tests in the PR tests detaching for tabs, but does not test whether it works specifically for streaming (suppose the line to detach gets deleted accidentally in future, no unit test would fail). So please add the streaming's specific unit test - rename spark.streaming.UISuite to UISeleniumSuite, and then update test to use selenium.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25979694
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -60,14 +62,31 @@ private[spark] abstract class WebUI(
         tab.pages.foreach(attachPage)
         tabs += tab
       }
    +  
    +  def detachTab(tab: WebUITab) {
    +    tab.pages.foreach(detachPage)
    +    tabs -= tab
    +  }
    +  
    +  def detachPage(page: WebUIPage) {
    +    page2Handlers.remove(page).foreach(_.foreach(detachHandler))
    +  }
    +  
     
       /** Attach a page to this UI. */
       def attachPage(page: WebUIPage) {
    --- End diff --
    
    @JoshRosen Can you take a look at this code. This is your domain. 


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78423867
  
    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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-81910018
  
    Merged this in master and branch-1.3


---
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-6077] Remove streaming tab while stoppi...

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

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


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25926814
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -60,14 +61,34 @@ private[spark] abstract class WebUI(
         tab.pages.foreach(attachPage)
         tabs += tab
       }
    +  
    +  def detachTab(tab: WebUITab) {
    +    tab.pages.foreach(detachPage)
    +    tabs -= tab
    +  }
    +  
    +  def detachPage(page: WebUIPage) {
    +    page2Handlers.remove(page) match {
    --- End diff --
    
    This can be simplified as `page2Handles.remove(page).foreach(...)`


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78450659
  
      [Test build #28503 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28503/consoleFull) for   PR 4828 at commit [`59180a7`](https://github.com/apache/spark/commit/59180a76b59872f4f010ae5121d1e8e4be4f531f).
     * This patch **passes all 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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76836876
  
    Will that be stable? Not flaky? In the past we had simple web ui tests that
    uses Scala's Source class to fetch a URL to see whether a tab has been
    loaded or unloaded. Those were disabled because of flakiness. I wonder
    whether selenium tests will be more stable.
    
    On Mon, Mar 2, 2015 at 1:55 PM, Josh Rosen <no...@github.com> wrote:
    
    > @tdas <https://github.com/tdas> do you think we should add a Selenium
    > test for this?
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/4828#issuecomment-76834346>.
    >



---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-77641090
  
    Can you add unit tests for detaching pages and tabs. @JoshRosen will be able to help you out. 


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25926778
  
    --- Diff: core/src/main/scala/org/apache/spark/ui/WebUI.scala ---
    @@ -17,17 +17,17 @@
     
     package org.apache.spark.ui
     
    +import scala.collection.mutable.HashMap
    --- End diff --
    
    Change the import ordering.
    
    ```
    Java import
    
    Scala import 
    
    Third party import 
    
    Spark import 
    ```



---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78337458
  
    ok to test



---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78434518
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28500/
    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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78440716
  
      [Test build #28502 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28502/consoleFull) for   PR 4828 at commit [`4e9e4fc`](https://github.com/apache/spark/commit/4e9e4fcb9dc432dd25a1a7a8ddc2347a7822654b).
     * This patch merges cleanly.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-77985103
  
    @JoshRosen , the test for tab attach / detach has been added base on the new UISeleniumSuite framework. Could you help to review? 


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78441340
  
    @tdas UISeleniumSuite has been added for streaming and the LocalStreamingContext utilities is borrowed from LocalSparkContext .


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25926833
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -553,6 +553,10 @@ class StreamingContext private[streaming] (
        */
       def stop(stopSparkContext: Boolean = true): Unit = synchronized {
         stop(stopSparkContext, false)
    +    this.uiTab match  {
    --- End diff --
    
    The same as above.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76853339
  
    Then we should go for it!
    
    
    On Mon, Mar 2, 2015 at 2:28 PM, Josh Rosen <no...@github.com> wrote:
    
    > Poorly-written Selenium tests can be flaky if they don't account for
    > things like asynchrony (e.g. when testing Javascript interactions), but I
    > don't think that will be a problem here. We now have a bunch of Selenium
    > tests for the Spark core UI and I don't think I've ever seen them fail in
    > Jenkins:
    > https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
    >
    > —
    > Reply to this email directly or view it on GitHub
    > <https://github.com/apache/spark/pull/4828#issuecomment-76840648>.
    >



---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78449751
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28502/
    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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#discussion_r26370169
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/LocalStreamingContext.scala ---
    @@ -0,0 +1,39 @@
    +/*
    + * 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.streaming
    +
    +object LocalStreamingContext {
    +  
    +  def stop(ssc: StreamingContext) {
    +    if (ssc != null) {
    +      ssc.stop()
    +    }
    +    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  /** Runs `f` by passing in `ssc` and ensures that `ssc` is stopped. */
    +  def withSparkStreaming[T](ssc: StreamingContext)(f: StreamingContext => T) = {
    --- End diff --
    
    This is already defined in streaming.TestSuiteBase. Rather than creating this new file, please use that. 


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78451828
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28504/
    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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25582237
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala ---
    @@ -20,20 +20,26 @@ package org.apache.spark.streaming.ui
     import java.util.Calendar
     import javax.servlet.http.HttpServletRequest
     
    -import scala.xml.Node
    -
     import org.apache.spark.Logging
    -import org.apache.spark.ui._
     import org.apache.spark.ui.UIUtils._
    +import org.apache.spark.ui._
     import org.apache.spark.util.Distribution
     
    +import scala.xml.Node
    +
     /** Page for Spark Web UI that shows statistics of a streaming job */
     private[ui] class StreamingPage(parent: StreamingTab)
       extends WebUIPage("") with Logging {
     
    -  private val listener = parent.listener
    -  private val startTime = Calendar.getInstance().getTime()
    +  private var listener = parent.listener
    +  private var startTime = Calendar.getInstance().getTime()
       private val emptyCell = "-"
    +  
    +  def updateListener(listener : StreamingJobProgressListener):
    +  Unit ={
    --- End diff --
    
    The wrapping is wrong here; `Unit` should be on the previous line and no space before the brace. No space before `:`. Also, `System.currentTimeMillis()` is much more conventional for getting the 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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76834346
  
    @tdas do you think we should add a Selenium test for this?


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78408636
  
      [Test build #626 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/626/consoleFull) for   PR 4828 at commit [`31a44fa`](https://github.com/apache/spark/commit/31a44faec3b7f5d160a1345fb2f1ff6544d6e8ea).
     * This patch merges cleanly.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25979865
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala ---
    @@ -38,10 +38,13 @@ private[spark] class StreamingTab(ssc: StreamingContext)
       parent.attachTab(this)
     }
     
    -private object StreamingTab {
    +private[spark] object StreamingTab {
       def getSparkUI(ssc: StreamingContext): SparkUI = {
         ssc.sc.ui.getOrElse {
           throw new SparkException("Parent SparkUI to attach this tab to not found!")
         }
       }
    +  def detachStreamingTab(ssc: StreamingContext, tab: SparkUITab) {
    --- End diff --
    
    Need extra line. 


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78892694
  
      [Test build #28556 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28556/consoleFull) for   PR 4828 at commit [`c329806`](https://github.com/apache/spark/commit/c329806f30cf82a796a0256c2e08f774b7fdd43b).
     * This patch **passes all 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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76890176
  
    @tdas  Just updated the code with the solution you mentioned(attach-on-start-and-remove-on-stop) by adding the detachTab and detachPage function. 


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25578794
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala ---
    @@ -29,19 +29,30 @@ import StreamingTab._
      */
     private[spark] class StreamingTab(ssc: StreamingContext)
       extends SparkUITab(getSparkUI(ssc), "streaming") with Logging {
    -
       val parent = getSparkUI(ssc)
    -  val listener = ssc.progressListener
    -
    +  var listener = ssc.progressListener
       ssc.addStreamingListener(listener)
       attachPage(new StreamingPage(this))
       parent.attachTab(this)
     }
     
    -private object StreamingTab {
    +private[spark] object StreamingTab {
       def getSparkUI(ssc: StreamingContext): SparkUI = {
         ssc.sc.ui.getOrElse {
           throw new SparkException("Parent SparkUI to attach this tab to not found!")
         }
       }
    +  def updateOrCreateStreamingTab(ssc:StreamingContext):Option[StreamingTab]={
    --- End diff --
    
    @srowen , Thanks for the advice. I've just tested and refactored the code accordingly.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76673885
  
    This looks OK to me. I'd like to give @tdas at least a day to look at it.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78429099
  
    LGTM, assuming the tests pass. 


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-77646184
  
    BTW, please update the title and JIRA as we are not updating existing tab any more. 


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25583379
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala ---
    @@ -29,19 +29,29 @@ import StreamingTab._
      */
     private[spark] class StreamingTab(ssc: StreamingContext)
       extends SparkUITab(getSparkUI(ssc), "streaming") with Logging {
    -
       val parent = getSparkUI(ssc)
    -  val listener = ssc.progressListener
    -
    +  var listener = ssc.progressListener
       ssc.addStreamingListener(listener)
       attachPage(new StreamingPage(this))
       parent.attachTab(this)
     }
     
    -private object StreamingTab {
    +private[spark] object StreamingTab {
       def getSparkUI(ssc: StreamingContext): SparkUI = {
         ssc.sc.ui.getOrElse {
           throw new SparkException("Parent SparkUI to attach this tab to not found!")
         }
       }
    +  
    +  def updateOrCreateStreamingTab(ssc: StreamingContext): Option[StreamingTab] = {
    --- End diff --
    
    Updated accordingly.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-77648366
  
    Spark core's `UISuite` has an ignored test for "attaching a new tab": https://github.com/apache/spark/blob/dba0b2eadb441f41ded0f0b9706b720bcfa84881/core/src/test/scala/org/apache/spark/ui/UISuite.scala#L75
    
    In order to test the tab attach / detach functionality, I think we should re-enable and extend this test to check that we're able to remove the tab that we just attached.  It looks like this test may have been disabled due to port contention issues (#466), but I think those issues should be resolved now.  If it will be easier, you might rewriting this test using the new [UISeleniumSuite](https://github.com/apache/spark/blob/dba0b2eadb441f41ded0f0b9706b720bcfa84881/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala) framework, which will let you use Selenium CSS selectors, etc. in your test.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25583362
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala ---
    @@ -20,20 +20,26 @@ package org.apache.spark.streaming.ui
     import java.util.Calendar
     import javax.servlet.http.HttpServletRequest
     
    -import scala.xml.Node
    -
     import org.apache.spark.Logging
    -import org.apache.spark.ui._
     import org.apache.spark.ui.UIUtils._
    +import org.apache.spark.ui._
     import org.apache.spark.util.Distribution
     
    +import scala.xml.Node
    +
     /** Page for Spark Web UI that shows statistics of a streaming job */
     private[ui] class StreamingPage(parent: StreamingTab)
       extends WebUIPage("") with Logging {
     
    -  private val listener = parent.listener
    -  private val startTime = Calendar.getInstance().getTime()
    +  private var listener = parent.listener
    +  private var startTime = Calendar.getInstance().getTime()
       private val emptyCell = "-"
    +  
    +  def updateListener(listener : StreamingJobProgressListener):
    +  Unit ={
    --- End diff --
    
    Done.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25979785
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -553,6 +553,7 @@ class StreamingContext private[streaming] (
        */
       def stop(stopSparkContext: Boolean = true): Unit = synchronized {
         stop(stopSparkContext, false)
    +    this.uiTab.foreach(StreamingTab.detachStreamingTab(this, _))
    --- End diff --
    
    Why is it hear and not in the other `stop` The tab should be removed when the stremaing context is stopped, so both `stop` should detach the tab.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78450669
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28503/
    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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78451818
  
      [Test build #28504 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28504/consoleFull) for   PR 4828 at commit [`bff699b`](https://github.com/apache/spark/commit/bff699bed733a195cd7061664f88a258fabf679e).
     * This patch **passes all 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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78441268
  
      [Test build #28503 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28503/consoleFull) for   PR 4828 at commit [`59180a7`](https://github.com/apache/spark/commit/59180a76b59872f4f010ae5121d1e8e4be4f531f).
     * This patch merges cleanly.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#discussion_r26370655
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.streaming
    +
    +import org.openqa.selenium.WebDriver
    +import org.openqa.selenium.htmlunit.HtmlUnitDriver
    +import org.scalatest._
    +import org.scalatest.concurrent.Eventually._
    +import org.scalatest.selenium.WebBrowser
    +import org.scalatest.time.SpanSugar._
    +
    +import org.apache.spark._
    +import org.apache.spark.streaming.LocalStreamingContext._
    +
    +
    +/**
    + * Selenium tests for the Spark Web UI.
    + */
    +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll {
    +
    +  implicit var webDriver: WebDriver = _
    +
    +  override def beforeAll(): Unit = {
    +    webDriver = new HtmlUnitDriver
    +  }
    +
    +  override def afterAll(): Unit = {
    +    if (webDriver != null) {
    +      webDriver.quit()
    +    }
    +  }
    +
    +  /**
    +   * Create a test SparkStreamingContext with the SparkUI enabled.
    +   */
    +  private def newSparkStreamingContext(): StreamingContext = {
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test")
    +      .set("spark.ui.enabled", "true")
    +    val ssc = new StreamingContext(conf, Seconds(1))
    +    assert(ssc.sc.ui.isDefined, "Spark UI is not started!")
    +    ssc
    +  }
    +
    +  test("attaching and detaching a Streaming tab") {
    +    withSparkStreaming(newSparkStreamingContext()) { ssc =>
    +      val sparkUI = ssc.sparkContext.ui.get
    +
    +      eventually(timeout(10 seconds), interval(50 milliseconds)) {
    +        go to (sparkUI.appUIAddress.stripSuffix("/"))
    +        find(cssSelector( """ul li a[href*="jobs"]""")) should not be (None)
    --- End diff --
    
    @tdas done.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#discussion_r26272770
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala ---
    @@ -38,10 +38,14 @@ private[spark] class StreamingTab(ssc: StreamingContext)
       parent.attachTab(this)
     }
     
    -private object StreamingTab {
    +private[spark] object StreamingTab {
       def getSparkUI(ssc: StreamingContext): SparkUI = {
         ssc.sc.ui.getOrElse {
           throw new SparkException("Parent SparkUI to attach this tab to not found!")
         }
       }
    +  
    --- End diff --
    
    Can you move this code into the StreamingTab class? The code is related to the tab instance, and so should be in the class. You could argue that `getSparkUI` should also be in the class, however it needs to be called from constructor (line 31) and hence cannot be in the class. But the `detach` function should be in the class.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78449739
  
      [Test build #28502 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28502/consoleFull) for   PR 4828 at commit [`4e9e4fc`](https://github.com/apache/spark/commit/4e9e4fcb9dc432dd25a1a7a8ddc2347a7822654b).
     * This patch **passes all 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-6077] update listener for the existing ...

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

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


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

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


[GitHub] spark pull request: [SPARK-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78892715
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28556/
    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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78428351
  
    ok to test


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78313946
  
    The new attach / detach test looks fine to me.
    
    Jenkins, this is ok to test.


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25930020
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -553,6 +553,10 @@ class StreamingContext private[streaming] (
        */
       def stop(stopSparkContext: Boolean = true): Unit = synchronized {
         stop(stopSparkContext, false)
    +    this.uiTab match  {
    --- End diff --
    
    Thanks @jerryshao . Done with the changes.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#discussion_r26272817
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -578,6 +578,7 @@ class StreamingContext private[streaming] (
         // Even if we have already stopped, we still need to attempt to stop the SparkContext because
         // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true).
         if (stopSparkContext) sc.stop()
    +    uiTab.foreach(StreamingTab.detachStreamingTab(this, _))
    --- End diff --
    
    If the detach is moved, this should become `uiTab.foreach { _.detach() }`


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78415352
  
      [Test build #626 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/626/consoleFull) for   PR 4828 at commit [`31a44fa`](https://github.com/apache/spark/commit/31a44faec3b7f5d160a1345fb2f1ff6544d6e8ea).
     * This patch **passes all 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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76829945
  
    Hey this is a decent fix, but I think this is not the right one. With this fix, the new ssc will be reflected in the new streaming tab, however it will be still visible even after the earlier ssc has been stopped. The right solution is that the tab should be removed when a stream ing context is stopped. Since only one streaming context can be active on the same spark context at the same time, attach-on-start-and-remove-on-stop will fix the multiple tab problem in the right way. 
    
    Does that make sense?


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#issuecomment-76840648
  
    Poorly-written Selenium tests can be flaky if they don't account for things like asynchrony (e.g. when testing Javascript interactions), but I don't think that will be a problem here.  We now have a bunch of Selenium tests for the Spark core UI and I don't think I've ever seen them fail in Jenkins: https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala


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

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


[GitHub] spark pull request: [SPARK-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78441772
  
      [Test build #28504 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28504/consoleFull) for   PR 4828 at commit [`bff699b`](https://github.com/apache/spark/commit/bff699bed733a195cd7061664f88a258fabf679e).
     * This patch merges cleanly.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#discussion_r26370210
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.streaming
    +
    +import org.openqa.selenium.WebDriver
    +import org.openqa.selenium.htmlunit.HtmlUnitDriver
    +import org.scalatest._
    +import org.scalatest.concurrent.Eventually._
    +import org.scalatest.selenium.WebBrowser
    +import org.scalatest.time.SpanSugar._
    +
    +import org.apache.spark._
    +import org.apache.spark.streaming.LocalStreamingContext._
    +
    +
    +/**
    + * Selenium tests for the Spark Web UI.
    + */
    +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll {
    +
    +  implicit var webDriver: WebDriver = _
    +
    +  override def beforeAll(): Unit = {
    +    webDriver = new HtmlUnitDriver
    +  }
    +
    +  override def afterAll(): Unit = {
    +    if (webDriver != null) {
    +      webDriver.quit()
    +    }
    +  }
    +
    +  /**
    +   * Create a test SparkStreamingContext with the SparkUI enabled.
    +   */
    +  private def newSparkStreamingContext(): StreamingContext = {
    +    val conf = new SparkConf()
    +      .setMaster("local")
    +      .setAppName("test")
    +      .set("spark.ui.enabled", "true")
    +    val ssc = new StreamingContext(conf, Seconds(1))
    +    assert(ssc.sc.ui.isDefined, "Spark UI is not started!")
    +    ssc
    +  }
    +
    +  test("attaching and detaching a Streaming tab") {
    +    withSparkStreaming(newSparkStreamingContext()) { ssc =>
    +      val sparkUI = ssc.sparkContext.ui.get
    +
    +      eventually(timeout(10 seconds), interval(50 milliseconds)) {
    +        go to (sparkUI.appUIAddress.stripSuffix("/"))
    +        find(cssSelector( """ul li a[href*="jobs"]""")) should not be (None)
    --- End diff --
    
    I dont think the usual ones like jobs, stages, etc need to be tested. Just testing streaming tab should be fine. The test will be faster i hope. 


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78428337
  
    add to whitelist


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#discussion_r26278085
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -578,6 +578,7 @@ class StreamingContext private[streaming] (
         // Even if we have already stopped, we still need to attempt to stop the SparkContext because
         // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true).
         if (stopSparkContext) sc.stop()
    +    uiTab.foreach(StreamingTab.detachStreamingTab(this, _))
    --- End diff --
    
    @tdas the code has been updated. Thanks for your review. 


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25558151
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala ---
    @@ -29,19 +29,30 @@ import StreamingTab._
      */
     private[spark] class StreamingTab(ssc: StreamingContext)
       extends SparkUITab(getSparkUI(ssc), "streaming") with Logging {
    -
       val parent = getSparkUI(ssc)
    -  val listener = ssc.progressListener
    -
    +  var listener = ssc.progressListener
       ssc.addStreamingListener(listener)
       attachPage(new StreamingPage(this))
       parent.attachTab(this)
     }
     
    -private object StreamingTab {
    +private[spark] object StreamingTab {
       def getSparkUI(ssc: StreamingContext): SparkUI = {
         ssc.sc.ui.getOrElse {
           throw new SparkException("Parent SparkUI to attach this tab to not found!")
         }
       }
    +  def updateOrCreateStreamingTab(ssc:StreamingContext):Option[StreamingTab]={
    --- End diff --
    
    You have some style issues here, like spaces around operators, braces, and parentheses. Have a look at other code. Also we usually write e.g. `foreach { tab =>` not `foreach(tab = {`
    I think the code here can be more concise and avoid `return` with a construction built around `getTabs.find(_.isInstanceOf[StreamingTab]).map(...).getOrElse(...)


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-78865476
  
      [Test build #28556 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/28556/consoleFull) for   PR 4828 at commit [`c329806`](https://github.com/apache/spark/commit/c329806f30cf82a796a0256c2e08f774b7fdd43b).
     * This patch merges cleanly.


---
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-6077] Remove streaming tab while stoppi...

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

    https://github.com/apache/spark/pull/4828#issuecomment-79272515
  
    LGTM! Will merge asap. 


---
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-6077] update listener for the existing ...

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

    https://github.com/apache/spark/pull/4828#discussion_r25582261
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala ---
    @@ -29,19 +29,29 @@ import StreamingTab._
      */
     private[spark] class StreamingTab(ssc: StreamingContext)
       extends SparkUITab(getSparkUI(ssc), "streaming") with Logging {
    -
       val parent = getSparkUI(ssc)
    -  val listener = ssc.progressListener
    -
    +  var listener = ssc.progressListener
       ssc.addStreamingListener(listener)
       attachPage(new StreamingPage(this))
       parent.attachTab(this)
     }
     
    -private object StreamingTab {
    +private[spark] object StreamingTab {
       def getSparkUI(ssc: StreamingContext): SparkUI = {
         ssc.sc.ui.getOrElse {
           throw new SparkException("Parent SparkUI to attach this tab to not found!")
         }
       }
    +  
    +  def updateOrCreateStreamingTab(ssc: StreamingContext): Option[StreamingTab] = {
    --- End diff --
    
    This method does not need to return `Option`. It always returns `StreamingTab`.


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