[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-23 Thread nfergu
Github user nfergu commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r15312160
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
@@ -945,6 +958,8 @@ class DAGScheduler(
 // Unrecognized failure - also do nothing. If the task fails 
repeatedly, the TaskScheduler
 // will abort the job.
 }
+listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, 
event.taskInfo,
+  event.taskMetrics))
--- End diff --

@kayousterhout -- I think that this may be a practical issue. 
StatsReportListener appears to rely on all of the tasks for a stage being 
completed before the stage itself.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-22 Thread shivaram
Github user shivaram commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-49808077
  
Its worthwhile to keep in mind https://github.com/apache/spark/pull/1056 
which adds partial updates to TaskMetrics. Another reason why unifying these 
things might be useful.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-22 Thread shivaram
Github user shivaram commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-49807746
  
At a high level this is great and I think this will address my use case 
that I outline in the mailing list. I do like the idea of reusing the 
accumulator machinery we have instead of reinventing things. 

Some questions / comments
- What happens if the accumulator is used in multiple stages ? Do we have a 
start value for the accumulator when this stage starts that we will display ? 
This is again a case where TaskMetrics have a defined boundary / meaning which 
may be simpler to reason about than accumulators.

- Feature request: It would be great to have stats for these counters 
similar to task durations etc.

- I couldn't find the class AccumulableInfo -- Does it already exist in the 
tree ?




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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-22 Thread shivaram
Github user shivaram commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r15258695
  
--- Diff: core/src/main/scala/org/apache/spark/Accumulators.scala ---
@@ -51,6 +51,13 @@ class Accumulable[R, T] (
 
   Accumulators.register(this, true)
 
+  /** A name for this accumulator / accumulable for display in Spark's UI.
+* Note that names must be unique within a SparkContext. */
+  def name: String = s"accumulator_$id"
+
+  /** Whether to display this accumulator in the web UI. */
+  def display: Boolean = true
--- End diff --

+1 on this. It'd be good to have a drop-down like thing where you can pick 
a subset of accumulators you want to show.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-18 Thread lianhuiwang
Github user lianhuiwang commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r15113632
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala ---
@@ -217,6 +223,7 @@ private[ui] class StagePage(parent: JobProgressTab) 
extends WebUIPage("stage") {
 Summary Metrics for {numCompleted} Completed Tasks ++
 {summaryTable.getOrElse("No tasks have reported metrics 
yet.")} ++
 Aggregated Metrics by Executor ++ executorTable.toNodeSeq 
++
+Accumulators ++ accumulableTable ++
--- End diff --

in mapreduce system,it shows all counters, so i think that is ok.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-17 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-49389878
  
QA results for PR 1309:- This patch FAILED unit tests.For more 
information see test 
ouptut:https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16804/consoleFull


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-17 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-49389828
  
QA tests have started for PR 1309. This patch DID NOT merge cleanly! 
View progress: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16804/consoleFull


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-09 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14696305
  
--- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
@@ -758,6 +758,16 @@ class SparkContext(config: SparkConf) extends Logging {
 new Accumulator(initialValue, param)
 
   /**
+   * Create an [[org.apache.spark.Accumulator]] variable of a given type, 
which tasks can "add"
+   * values to using the `+=` method. Only the driver can access the 
accumulator's `value`.
+   *
+   * This version adds a custom name to the accumulator for display in the 
Spark UI.
+   */
+  def accumulator[T](initialValue: T, name: String)(implicit param: 
AccumulatorParam[T]) = {
+new Accumulator(initialValue, param, name, true)
+  }
+
+  /**
--- End diff --

Should there be (maybe you were planning to add it later since I know this 
is still WIP) a similar new accumulable method?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-09 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48437261
  
This feature looks awesome!!  Mostly cosmetic comments about how to make 
this as understandable as possible to users.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-09 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14696093
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
@@ -945,6 +958,8 @@ class DAGScheduler(
 // Unrecognized failure - also do nothing. If the task fails 
repeatedly, the TaskScheduler
 // will abort the job.
 }
+listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, 
event.taskInfo,
+  event.taskMetrics))
--- End diff --

I think this may break some current invariants w/ the listener because the 
TaskEnd can be received *after* the JobEnd but not sure whether this is a 
practical issue...have you looked at that at all?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-09 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695974
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala ---
@@ -217,6 +223,7 @@ private[ui] class StagePage(parent: JobProgressTab) 
extends WebUIPage("stage") {
 Summary Metrics for {numCompleted} Completed Tasks ++
 {summaryTable.getOrElse("No tasks have reported metrics 
yet.")} ++
 Aggregated Metrics by Executor ++ executorTable.toNodeSeq 
++
+Accumulators ++ accumulableTable ++
--- End diff --

Can we show this only if there are >0 accumulators?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-09 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695912
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -147,6 +155,11 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener {
 val info = taskEnd.taskInfo
 
 if (info != null) {
+  val accumulables = stageIdToAccumulables.getOrElseUpdate(sid, 
HashMap[String, String]())
+  for ((name, value) <- info.accumulableValues) {
+accumulables(name) = value
--- End diff --

Is this right?  Isn't this setting the value for the whole stage to be the 
partial update for the task (which could explain the discrepancy in your 
screenshot)?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695696
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala ---
@@ -42,6 +44,12 @@ class TaskInfo(
   var gettingResultTime: Long = 0
 
   /**
+   * Intermediate updates to accumulables during this task. Note that it 
is valid for the same
+   * accumulable to be updated multiple times in a single task.
--- End diff --

The second sentence in this comment is confusing because it makes it sound 
like this could have two entries for the same accumulator name...I'd just 
remove 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695670
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
@@ -809,12 +810,24 @@ class DAGScheduler(
   listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage)))
   runningStages -= stage
 }
+
 event.reason match {
   case Success =>
 logInfo("Completed " + task)
 if (event.accumUpdates != null) {
   // TODO: fail the stage if the accumulator update fails...
   Accumulators.add(event.accumUpdates) // TODO: do this only if 
task wasn't resubmitted
+  event.accumUpdates.foreach { case (id, partialValue) =>
+val acc = Accumulators.originals(id)
+val name = acc.name
+// To avoid UI cruft, ignore cases where value wasn't updated
+if (partialValue != acc.zero) {
+  val stringPartialValue = s"${partialValue}"
+  val stringValue = s"${acc.value}"
+  stageToInfos(stage).accumulatedValues(name) = stringValue
--- End diff --

Can you add a comment here saying that you update it incrementally (and not 
just at the end) so the UI always shows the correct value (assuming that is the 
reason)?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695572
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
@@ -809,12 +810,24 @@ class DAGScheduler(
   listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage)))
   runningStages -= stage
 }
+
 event.reason match {
   case Success =>
 logInfo("Completed " + task)
 if (event.accumUpdates != null) {
   // TODO: fail the stage if the accumulator update fails...
   Accumulators.add(event.accumUpdates) // TODO: do this only if 
task wasn't resubmitted
+  event.accumUpdates.foreach { case (id, partialValue) =>
--- End diff --

Can you move this code to TaskSetManager?  Where possible I think we should 
keep task-specific code out of the DAGScheduler, which should be dealing 
primarily with stages (I think @rxin may be on a mission to more clearly 
define/document 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695528
  
--- Diff: core/src/main/scala/org/apache/spark/Accumulators.scala ---
@@ -51,6 +51,13 @@ class Accumulable[R, T] (
 
   Accumulators.register(this, true)
 
+  /** A name for this accumulator / accumulable for display in Spark's UI.
+* Note that names must be unique within a SparkContext. */
+  def name: String = s"accumulator_$id"
--- End diff --

It seems weird to me that the ID and name both need to be unique.  While I 
realize that it's confusing and probably should be avoided to have two 
accumulators with the same name, can you just index accumulators by their ID in 
the UI to avoid the need for these to be unique?  It just seems like this could 
lead to hard to diagnose bugs where someone creates two accumulators with the 
same name that appear as the same in the UI (even though in practice they're 
different).


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695474
  
--- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
@@ -758,6 +758,16 @@ class SparkContext(config: SparkConf) extends Logging {
 new Accumulator(initialValue, param)
 
   /**
+   * Create an [[org.apache.spark.Accumulator]] variable of a given type, 
which tasks can "add"
+   * values to using the `+=` method. Only the driver can access the 
accumulator's `value`.
+   *
+   * This version adds a custom name to the accumulator for display in the 
Spark UI.
--- End diff --

It would be nice to have a short code example here (or somewhere in the 
documentation) showing how you can create of these and use 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48435300
  
Not sure if this is just because you took the screenshot early on, but I 
noticed that the accumulator totals in the screenshot are not correct based on 
the updates from the 4 tasks


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695421
  
--- Diff: core/src/main/scala/org/apache/spark/rdd/RDD.scala ---
@@ -1270,4 +1270,10 @@ abstract class RDD[T: ClassTag](
   def toJavaRDD() : JavaRDD[T] = {
 new JavaRDD(this)(elementClassTag)
   }
+
+  // 
===
+  // Common metrics
+  // 
===
+  // Input bytes if this RDD was read from persisted data or a filesystem
+  val inputBytes = sc.accumulator(0L, s"rdd-$id.input.bytes.persisted")
--- End diff --

is this just here as an example or is this going to stay in the code?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695401
  
--- Diff: core/src/main/scala/org/apache/spark/Accumulators.scala ---
@@ -51,6 +51,13 @@ class Accumulable[R, T] (
 
   Accumulators.register(this, true)
 
+  /** A name for this accumulator / accumulable for display in Spark's UI.
+* Note that names must be unique within a SparkContext. */
+  def name: String = s"accumulator_$id"
+
+  /** Whether to display this accumulator in the web UI. */
+  def display: Boolean = true
--- End diff --

This is probably too annoying (or a good starter task?) but it might be 
nice if you could decide on-demand, in the UI, which accumulators you want to 
show.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695308
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala ---
@@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs
 import java.util.Date
 import javax.servlet.http.HttpServletRequest
 
-import scala.xml.Node
+import scala.xml.{Unparsed, Node}
--- End diff --

nit: alphabetization


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695237
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -48,6 +48,7 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener {
 
   // TODO: Should probably consolidate all following into a single hash 
map.
   val stageIdToTime = HashMap[Int, Long]()
+  val stageIdToAccumulables = HashMap[Int, Map[String, String]]()
--- End diff --

FYI this will conflict with @rxin's change to consolidate these into a 
single hash map


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/1309#discussion_r14695146
  
--- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
@@ -809,12 +810,24 @@ class DAGScheduler(
   listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage)))
   runningStages -= stage
 }
+
 event.reason match {
   case Success =>
 logInfo("Completed " + task)
 if (event.accumUpdates != null) {
   // TODO: fail the stage if the accumulator update fails...
   Accumulators.add(event.accumUpdates) // TODO: do this only if 
task wasn't resubmitted
+  event.accumUpdates.foreach { case (id, partialValue) =>
+val acc = Accumulators.originals(id)
+val name = acc.name
+// To avoid UI cruft, ignore cases where value wasn't updated
--- End diff --

Do you think this will make things hard to debug -- e.g., if someone's 
accumulator doesn't show up in the UI and they don't realize it's because the 
value wasn't updated as opposed to because they didn't set the show-in-ui 
variable correctly?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-08 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48418871
  
@kayousterhout - do you mind taking a look at 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48161598
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16367/


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48161597
  
Merged build finished. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48161151
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16366/


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48161150
  
Merged build finished. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48157839
  
 Merged build triggered. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48157853
  
Merged build started. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48157362
  
 Merged build triggered. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48157378
  
Merged build started. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48156189
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16365/


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48156188
  
Merged build finished. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48156046
  
 Merged build triggered. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48156061
  
Merged build started. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48135601
  
How about a colon?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48129437
  
@rxin at first I just had `=`, but then I thought it could be confusing. 
Because that is not showing the total value of the accumulator, it's just 
showing the local addition from that task.


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48126407
  
This is pretty cool. Why "+="?


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48110178
  
/cc @kayousterhout


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48110157
  
Merged build finished. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48110158
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16357/


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48110113
  
 Merged build triggered. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/1309#issuecomment-48110116
  
Merged build started. 


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


[GitHub] spark pull request: SPARK-2380 [WIP]: Support displaying accumulat...

2014-07-06 Thread pwendell
GitHub user pwendell opened a pull request:

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

SPARK-2380 [WIP]: Support displaying accumulator values in the web UI

This patch adds support for giving accumulators user-visible names and 
displaying accumulator values in the web UI. This allows users to create custom 
counters that can display in the UI. The current approach displays both the 
accumulator deltas caused by each task and a "current" value of the accumulator 
totals for each stage, which gets update as tasks finish.

Currently in Spark developers have been extending the `TaskMetrics` 
functionality to provide custom instrumentation for RDD's. This provides a 
potentially nicer alternative of going through the existing accumulator 
framework (actually `TaskMetrics` and accumulators are on an awkward collision 
course as we add more features to the former). The current patch demo's how we 
can use the feature to provide instrumentation for RDD input sizes. The nice 
thing about going through accumulators is that users can actually read the 
current value of the data being tracked in their programs. This could be useful 
to e.g. decide to short-circuit a Spark stage depending on how things are going.

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

$ git pull https://github.com/pwendell/spark metrics

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

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


commit 0b72660a8da074f303ea1795af9ee1f0312877a7
Author: Patrick Wendell 
Date:   2014-07-06T04:11:15Z

Initial WIP example of supporing globally named accumulators.

commit ad85076f621df3dc688761bd189af2fd5935bd52
Author: Patrick Wendell 
Date:   2014-07-06T11:41:51Z

Example of using named accumulators for custom RDD metrics.




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