You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by keypointt <gi...@git.apache.org> on 2015/12/02 19:59:17 UTC

[GitHub] spark pull request: [SPARK-11155][Web UI] Stage summary json shoul...

GitHub user keypointt opened a pull request:

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

    [SPARK-11155][Web UI] Stage summary json should include stage duration

    The json endpoint for stages doesn't include information on the stage duration that is present in the UI. This looks like a simple oversight, they should be included. eg., the metrics should be included at api/v1/applications/<appId>/stages.
    
    Metrics I've added are: submissionTime, firstTaskLaunchedTime and completionTime 

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

    $ git pull https://github.com/keypointt/spark SPARK-11155

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

    https://github.com/apache/spark/pull/10107.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 #10107
    
----
commit e46cf1568e596ca23209f090f622e98a5223d71c
Author: Xin Ren <ia...@126.com>
Date:   2015-12-01T00:56:03Z

    [SPARK-11155] add missing metrics, but still don't know where to extract from

commit 85f5be402f6b0fb772824c630443bb35fc077f1d
Author: Xin Ren <ia...@126.com>
Date:   2015-12-01T23:12:02Z

    [SPARK-11155] 'firstTaskLaunchedTime' seems working on my local machine but bad practise, should be a variable in class UiData somehow. besides, don't know how to extract 'maximumDurationOfAllTasks'

commit f3f76ba933344d97dda71e807f90d928e6fe1fa3
Author: Xin Ren <ia...@126.com>
Date:   2015-12-02T00:28:26Z

    [SPARK-11155] 'maximumDurationOfAllTasks' is for StageTable, remove this metric

commit 2cc1556584e6e9f4efe2b25253a9432601f6abad
Author: Xin Ren <ia...@126.com>
Date:   2015-12-02T00:55:52Z

    [SPARK-11155] style correction: File line length exceeds 100 characters

commit cdf9c1270a7ab585f139125a83d2801052606e9d
Author: Xin Ren <ia...@126.com>
Date:   2015-12-02T17:56:33Z

    [SPARK-11155] change from epoch time to Date() format

commit 0b6b7bf0e432cf81d07ac0c1791583dea2ff3fd0
Author: Xin Ren <ia...@126.com>
Date:   2015-12-02T18:49:46Z

    [SPARK-11155] fix test cases for HistoryServerSuite

----


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161480398
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161868423
  
    **[Test build #47175 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47175/consoleFull)** for PR 10107 at commit [`baeaeee`](https://github.com/apache/spark/commit/baeaeeec37f6c980e62591dd74c2595a7918bc14).
     * 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162740014
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162118393
  
    **[Test build #47210 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47210/consoleFull)** for PR 10107 at commit [`475c2bd`](https://github.com/apache/spark/commit/475c2bd41db1b6b5ffd70f7b25a030b18c889ae5).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46743512
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.status.api.v1
    +
    +import java.util.Date
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality}
    +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
    +
    +
    +class AllStagesResourceSuite extends SparkFunSuite {
    +
    +  test("test firstTaskLaunchedTime, there are no tasks") {
    +
    +    val status = StageStatus.PENDING
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val includeDetails = false
    +
    +    val noTasks = new StageUIData()
    +
    +    var actual = AllStagesResource.stageUiToStageData(
    +      status, stageInfo, noTasks, includeDetails)
    +
    +    assert(actual.firstTaskLaunchedTime == None)
    +  }
    +
    +  test("test firstTaskLaunchedTime, there are tasks but none launched") {
    +
    +    val status = StageStatus.ACTIVE
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val includeDetails = false
    +
    +    // generate some tasks, launched time is minus
    +    val taskNoLaunched1 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -100, "", "", TaskLocality.ANY, false), None, None)
    +    val taskNoLaunched2 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -200, "", "", TaskLocality.ANY, false), None, None)
    +    val taskNoLaunched3 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -300, "", "", TaskLocality.ANY, false), None, None)
    +
    +    // construct hashmap
    +    var taskDataNoLaunched = new HashMap[Long, TaskUIData]
    --- End diff --
    
    could be a `val`


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162687748
  
    **[Test build #47272 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47272/consoleFull)** for PR 10107 at commit [`b6927aa`](https://github.com/apache/spark/commit/b6927aa495b7f47ba9aa8300241c346dc7854af6).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161788181
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161444023
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161437286
  
    Jenkins, 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162132697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47207/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162741406
  
    **[Test build #47302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47302/consoleFull)** for PR 10107 at commit [`b6927aa`](https://github.com/apache/spark/commit/b6927aa495b7f47ba9aa8300241c346dc7854af6).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162132736
  
    **[Test build #47210 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47210/consoleFull)** for PR 10107 at commit [`475c2bd`](https://github.com/apache/spark/commit/475c2bd41db1b6b5ffd70f7b25a030b18c889ae5).
     * 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162132696
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161759606
  
    for test suit "AllStageResourceSuite", I should add it to "core/src/test/scala/org/apache/spark/api/scala/AllStageResourceSuite.scala"? since I found under "core/src/test/scala/org/apache/spark/api/" there is only one folder "python".
    
    And also add "core/src/test/resources/AllStageResourceSuiteExpectations/all_stages_expectation.json"? I'm not quite understand here, since in "HistoryServerSuite" these expected results have already been tested.
    
    Thank you very much :smile: 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162687905
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47272/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162132770
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47210/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161449989
  
    hmm, that seems strange that the mima checks passed when run locally.  If you click through to the build results, you'll see this failure:
    
    ```
    [error]  * method this(org.apache.spark.status.api.v1.StageStatus,Int,Int,Int,Int,Int,Long,Long,Long,Long,Long,Long,Long,Long,Long,Long,Long,java.lang.String,java.lang.String,java.lang.String,scala.collection.Seq,scala.Option,scala.Option)Unit in class org.apache.spark.status.api.v1.StageData does not have a correspondent in new version
    [error]    filter with: ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this")
    ```
    
    which is a false positive (that constructor is marked `private[spark]`, so its ok for us to change it), so you can add the given `ProblemFilters` line to `project/MimaExcludes.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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46826421
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.status.api.v1
    +
    +import java.util.Date
    +
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality}
    +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
    +
    +
    +class AllStagesResourceSuite extends SparkFunSuite {
    +
    +  def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = {
    +    val tasks = taskLaunchTimes.zipWithIndex.map { case (time, idx) =>
    +      idx.toLong -> new TaskUIData(
    +        new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None)
    +    }.toMap
    +    val stageUiData = new StageUIData()
    +    stageUiData.taskData = mapToHashmap(tasks)
    +    val status = StageStatus.ACTIVE
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false)
    +
    +    stageData.firstTaskLaunchedTime
    +  }
    +
    +  def mapToHashmap(original: Map[Long, TaskUIData]): HashMap[Long, TaskUIData] = {
    +    val map = new HashMap[Long, TaskUIData]
    +    original.foreach { e => map.put(e._1, e._2) }
    +
    +    return map
    +  }
    +
    +  test("test firstTaskLaunchedTime, there are no tasks") {
    --- End diff --
    
    you don't need to put "test" in the test name, its understood


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162960042
  
    merged to master (I made some very tiny style changes).  Thanks for working on this @keypointt!


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162132675
  
    **[Test build #47207 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47207/consoleFull)** for PR 10107 at commit [`a341796`](https://github.com/apache/spark/commit/a3417962ebb91d85614fdee73d5ef27a43877528).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162117662
  
    Thank you for your beautiful elegant functional code, I just copied it :stuck_out_tongue:
    How do you like this commit? I added another method convert Map to HashMap to fit the "taskData" declaration 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46695798
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,92 @@
    +/*
    + * 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.status.api.v1
    +
    +import org.scalatest.Matchers
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.scheduler.StageInfo
    +import org.apache.spark.ui.jobs.UIData.StageUIData
    +
    +class AllStagesResourceSuite extends SparkFunSuite with Matchers {
    +
    +  test("test test") {
    +    val s = "abc"
    +    assert(s === "abc")
    +  }
    +
    +  test("test class instantiation") {
    +    val status = StageStatus.COMPLETE
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val stageUiData = new StageUIData()
    +    val includeDetails = false
    +
    +    val actual = AllStagesResource.stageUiToStageData(
    +      status, stageInfo, stageUiData, includeDetails)
    +    val expected = new StageData(status, 1, 1, 0, 0, 0, 0, None, None, None,
    +      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, "stage 1", "details abc", "", Seq.empty, None, None)
    +
    +    assert(actual.status == expected.status)
    --- End diff --
    
    Hi @keypointt thanks for doing this, but I guess I was thinking something much simpler, sorry for the extra work you had to do.  I was thinking a test case which really just focuses on `firstTaskLaunchTime` -- so call `AllStagesResource.stageUiToStageData` as you are doing, but just check the `firstTaskLaunchTime` part of the results.  And try it with 3 different versions:
    
    1. No tasks --> `firstTaskLaunchTime == None`
    2. Several tasks, but none launched --> `firstTaskLaunchTime == None`
    3. Several tasks, some launched --> `firstTaskLaunchTime == Some(...)`
    
    does that make sense?  I dont' see much value in checking every single field since that is mostly covered by the other tests.  Then I'd also rename the test to "firstTaskLaunchTime" as well so its clear that is what its focusing on.


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161763853
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161480295
  
    **[Test build #47093 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47093/consoleFull)** for PR 10107 at commit [`e9d78db`](https://github.com/apache/spark/commit/e9d78dbf58b54cf995e43bb00abf519f7235dbd8).
     * 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162095292
  
    Thanks a lot for the explanation, how do you like this change? seems not very "functional" when I create those tasks objects :smile: 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162759742
  
    **[Test build #47302 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47302/consoleFull)** for PR 10107 at commit [`b6927aa`](https://github.com/apache/spark/commit/b6927aa495b7f47ba9aa8300241c346dc7854af6).
     * 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161852381
  
    **[Test build #47175 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47175/consoleFull)** for PR 10107 at commit [`baeaeee`](https://github.com/apache/spark/commit/baeaeeec37f6c980e62591dd74c2595a7918bc14).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161788283
  
    you should mimic the folder structure of the main class, just w/ "test" instead of main".  So for "core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala", you would make "core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala".
    
    I'm not suggesting you add another expecation file at all.  HistoryServerSuite is a little unusual in that the tests reference an external resource for what the results should be. You can write a much more straightforward test, it will just call the relevant method in `AllStagesResource`, and do some checks on the result.  for a really basic example consider ByteArrayChunkOutputStreamSuite.
    
    Hopefully that helps, I can also write a skeleton if you are stuck


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46744776
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
    @@ -59,6 +59,16 @@ private[v1] object AllStagesResource {
           stageUiData: StageUIData,
           includeDetails: Boolean): StageData = {
     
    +    val taskLaunchTimes = stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0)
    +
    +    val firstTaskLaunchedTime: Option[Date] =
    +      if (taskLaunchTimes.nonEmpty) {
    +        val minTaskLauchTime = taskLaunchTimes.min
    +        Some(new Date(minTaskLauchTime))
    --- End diff --
    
    super minor, but you don't really need `minTaskLaunchTime`, could just combine it into one line: `Some(new Date(taskLaunchTimes.min))`


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161789850
  
    I got what you meant, just to test these methods for corner cases. I think I can do the same thing referring to "ByteArrayChunkOutputStreamSuite". I'll try it out myself first, and let you know if I'm stuck. Thank you very much :smile: 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161462373
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46826348
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.status.api.v1
    +
    +import java.util.Date
    +
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality}
    +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
    +
    +
    +class AllStagesResourceSuite extends SparkFunSuite {
    +
    +  def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = {
    +    val tasks = taskLaunchTimes.zipWithIndex.map { case (time, idx) =>
    +      idx.toLong -> new TaskUIData(
    +        new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None)
    +    }.toMap
    +    val stageUiData = new StageUIData()
    +    stageUiData.taskData = mapToHashmap(tasks)
    --- End diff --
    
    sorry I hadn't realized that it needed to be a hashmap, I though any map would do.  this is really super minor, but rather than having a helper method, I'd just build the hashmap in the first place, the same way you are doing now:
    
    ```scala
    val tasks = new HashMap[Long, TaskUIData]
    taskLaunchTimes.zipWithIndex.foreach { case (time, idx) =>
      tasks(idx.toLong) = new TaskUIData(
        new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None)
    }
    ``` 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161443973
  
    **[Test build #47088 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47088/consoleFull)** for PR 10107 at commit [`0b6b7bf`](https://github.com/apache/spark/commit/0b6b7bf0e432cf81d07ac0c1791583dea2ff3fd0).
     * This patch **fails MiMa 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161448914
  
    I've run "./dev/mima" on my local machine and it's passed, I'm not sure is there anything else I should change to pass the SparkQA 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161462375
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47092/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162097674
  
    **[Test build #47207 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47207/consoleFull)** for PR 10107 at commit [`a341796`](https://github.com/apache/spark/commit/a3417962ebb91d85614fdee73d5ef27a43877528).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162689241
  
    hi @squito could you please have a look at this build? "fails from timeout", seems halted at pyspark testing for 2.5 hours, not sure what happened since I didn't touch python part


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46744734
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.status.api.v1
    +
    +import java.util.Date
    +import scala.collection.mutable.HashMap
    --- End diff --
    
    nit: blank line between `java` and `scala` imports.


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161868469
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47175/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46485011
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
    @@ -59,6 +59,9 @@ private[v1] object AllStagesResource {
           stageUiData: StageUIData,
           includeDetails: Boolean): StageData = {
     
    +    val firstTaskLaunchedTime =
    +      new Date(stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0).min)
    --- End diff --
    
    probably worth adding a unit test too, since its easy to forget about this in the future.


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161461176
  
    **[Test build #47093 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47093/consoleFull)** for PR 10107 at commit [`e9d78db`](https://github.com/apache/spark/commit/e9d78dbf58b54cf995e43bb00abf519f7235dbd8).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161684523
  
    You could add a case in HistoryServerSuite, but then you'd need to also create more event log data with tasks that hadn't been submitted.  I think it would be much easier to just add a new test suite, AllStageResourceSuite, just with one simple 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162759804
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161399322
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46743675
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.status.api.v1
    +
    +import java.util.Date
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality}
    +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
    +
    +
    +class AllStagesResourceSuite extends SparkFunSuite {
    +
    +  test("test firstTaskLaunchedTime, there are no tasks") {
    +
    +    val status = StageStatus.PENDING
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val includeDetails = false
    +
    +    val noTasks = new StageUIData()
    +
    +    var actual = AllStagesResource.stageUiToStageData(
    +      status, stageInfo, noTasks, includeDetails)
    +
    +    assert(actual.firstTaskLaunchedTime == None)
    +  }
    +
    +  test("test firstTaskLaunchedTime, there are tasks but none launched") {
    +
    +    val status = StageStatus.ACTIVE
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val includeDetails = false
    +
    +    // generate some tasks, launched time is minus
    +    val taskNoLaunched1 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -100, "", "", TaskLocality.ANY, false), None, None)
    +    val taskNoLaunched2 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -200, "", "", TaskLocality.ANY, false), None, None)
    +    val taskNoLaunched3 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -300, "", "", TaskLocality.ANY, false), None, None)
    +
    +    // construct hashmap
    +    var taskDataNoLaunched = new HashMap[Long, TaskUIData]
    +    taskDataNoLaunched.put(1, taskNoLaunched1)
    +    taskDataNoLaunched.put(2, taskNoLaunched2)
    +    taskDataNoLaunched.put(3, taskNoLaunched3)
    +
    +    val tasksNoLaunched = new StageUIData()
    +    tasksNoLaunched.taskData = taskDataNoLaunched
    --- End diff --
    
    you can clean this up with a helper method:
    
    ```scala
    def buildStageUiData(taskLaunchTimes: Seq[Long]): StageUIData = {
      val tasks = taskLaunchTime.zipWithIndex.map { case (time, idx) =>
        idx.toLong -> new TaskUIData(new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None)
      }.toMap
      val stageUiData = new StageUIData()
      stageUiData.taskData = tasks
      stageUiData
    }
    ```


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46480783
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -120,6 +120,9 @@ class StageData private[spark](
         val numFailedTasks: Int,
     
         val executorRunTime: Long,
    +    var submissionTime: Date,
    --- End diff --
    
    this can be a `val` too, right?


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46568902
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
    @@ -59,6 +59,10 @@ private[v1] object AllStagesResource {
           stageUiData: StageUIData,
           includeDetails: Boolean): StageData = {
     
    +    val firstTaskLaunchedTimeOption =
    +      Option(stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0).min)
    +    val firstTaskLaunchedTime = new Date(firstTaskLaunchedTimeOption.get)
    --- End diff --
    
    this isn't enough -- if the list of tasks with launchTime > 0 is empty, this will throw an exception:
    
    ```scala
    scala> Option(Seq[Int]().min)
    java.lang.UnsupportedOperationException: empty.min
    ```
    
    also, I think the final type of `StageData#firstTaskLaunchTime` actually needs to be a `Option[Date]`, since it might be empty.


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162957856
  
    hi @squito now tests passed, what should I do next? 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161459145
  
    I've added the "ProblemFilters" line. And how to add a unit test? I'm not sure I know the correct way to do it, I mean, add another test case in "HistoryServerSuite.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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161444026
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47088/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162003422
  
    Also to respond to your comment on mockito -- that is a very useful tool, we use it elsewhere in spark, but there isn't any need to use it here.  there is nothing to mock in this case.  without going into details here, you can probably read more about the use cases or look at where its used elsewhere in spark to get an idea about 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46482857
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
    @@ -59,6 +59,9 @@ private[v1] object AllStagesResource {
           stageUiData: StageUIData,
           includeDetails: Boolean): StageData = {
     
    +    val firstTaskLaunchedTime =
    +      new Date(stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0).min)
    --- End diff --
    
    ok I'll modify it too, thanks a lot :+1: 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162759805
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47302/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161440103
  
    **[Test build #47088 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47088/consoleFull)** for PR 10107 at commit [`0b6b7bf`](https://github.com/apache/spark/commit/0b6b7bf0e432cf81d07ac0c1791583dea2ff3fd0).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161851408
  
    Hi @squito could you please have a look at https://github.com/keypointt/spark/commit/baeaeeec37f6c980e62591dd74c2595a7918bc14, I'm trying to make sure that I'm on the right path?
    
    Should I use mockito? I googled and found it's a good testing mocking tool.
    
    Thanks a lot 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162612772
  
    **[Test build #47272 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47272/consoleFull)** for PR 10107 at commit [`b6927aa`](https://github.com/apache/spark/commit/b6927aa495b7f47ba9aa8300241c346dc7854af6).


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46482655
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -120,6 +120,9 @@ class StageData private[spark](
         val numFailedTasks: Int,
     
         val executorRunTime: Long,
    +    var submissionTime: Date,
    --- End diff --
    
    oh right, I was testing it as "Long" at the beginning, and then changed to "Date" but forgot to change it to be val. I'll change 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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161763857
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47159/
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162961226
  
    @squito yeah! my first code change, and I've learnt a lot from you :+1: 
    Thank you so much Imran! 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162610193
  
    I've changed the test name, and the hashmap generation :smile: 


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46744245
  
    --- Diff: core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.status.api.v1
    +
    +import java.util.Date
    +import scala.collection.mutable.HashMap
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality}
    +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
    +
    +
    +class AllStagesResourceSuite extends SparkFunSuite {
    +
    +  test("test firstTaskLaunchedTime, there are no tasks") {
    +
    +    val status = StageStatus.PENDING
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val includeDetails = false
    +
    +    val noTasks = new StageUIData()
    +
    +    var actual = AllStagesResource.stageUiToStageData(
    +      status, stageInfo, noTasks, includeDetails)
    +
    +    assert(actual.firstTaskLaunchedTime == None)
    +  }
    +
    +  test("test firstTaskLaunchedTime, there are tasks but none launched") {
    +
    +    val status = StageStatus.ACTIVE
    +    val stageInfo = new StageInfo(
    +      1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
    +    val includeDetails = false
    +
    +    // generate some tasks, launched time is minus
    +    val taskNoLaunched1 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -100, "", "", TaskLocality.ANY, false), None, None)
    +    val taskNoLaunched2 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -200, "", "", TaskLocality.ANY, false), None, None)
    +    val taskNoLaunched3 = new TaskUIData(
    +      new TaskInfo(1, 1, 1, -300, "", "", TaskLocality.ANY, false), None, None)
    +
    +    // construct hashmap
    +    var taskDataNoLaunched = new HashMap[Long, TaskUIData]
    +    taskDataNoLaunched.put(1, taskNoLaunched1)
    +    taskDataNoLaunched.put(2, taskNoLaunched2)
    +    taskDataNoLaunched.put(3, taskNoLaunched3)
    +
    +    val tasksNoLaunched = new StageUIData()
    +    tasksNoLaunched.taskData = taskDataNoLaunched
    --- End diff --
    
    actually, take it a step further, have the helper method even call the `stageUiToStageData`:
    
    ```scala
    def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = {
      val tasks = taskLaunchTime.zipWithIndex.map { case (time, idx) =>
        idx.toLong -> new TaskUIData(new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false), None, None)
      }.toMap
      val stageUiData = new StageUIData()
      stageUiData.taskData = tasks
      val status = StageStatus.ACTIVE
      val stageInfo = new StageInfo(
        1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty)
      val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false)
      stageData.firstTaskLaunchTime
    }
    ```
    
    then each of the three cases becomes super simple.


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161480399
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47093/
    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-11155][Web UI] Stage summary json shoul...

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

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


---
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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-161868467
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162132768
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#issuecomment-162687902
  
    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-11155][Web UI] Stage summary json shoul...

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

    https://github.com/apache/spark/pull/10107#discussion_r46476993
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala ---
    @@ -59,6 +59,9 @@ private[v1] object AllStagesResource {
           stageUiData: StageUIData,
           includeDetails: Boolean): StageData = {
     
    +    val firstTaskLaunchedTime =
    +      new Date(stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0).min)
    --- End diff --
    
    I think you need to guard against calling min on an empty list, eg. if no tasks have been launched yet.  So the type should probably be an Option


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