Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

SPARK-2380: Support displaying accumulator values in the web UI #1309

Closed
wants to merge 19 commits into from
Closed
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 13 additions & 2 deletions core/src/main/scala/org/apache/spark/Accumulators.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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


/** Whether to display this accumulator in the web UI. */
def display: Boolean = true
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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


/**
* Add more data to this accumulator / accumulable
* @param term the data to add
Expand Down Expand Up @@ -219,8 +226,12 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa
* @param param helper object defining how to add elements of type `T`
* @tparam T result type
*/
class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T])
extends Accumulable[T,T](initialValue, param)
class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], _name: String,
_display: Boolean) extends Accumulable[T,T](initialValue, param) {
override def name = if (_name.eq(null)) s"accumulator_$id" else _name
override def display = _display
def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, null, true)
}

/**
* A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add
Expand Down
3 changes: 3 additions & 0 deletions core/src/main/scala/org/apache/spark/CacheManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
case Some(blockResult) =>
// Partition is already materialized, so just return its values
context.taskMetrics.inputMetrics = Some(blockResult.inputMetrics)
if (blockResult.inputMetrics.bytesRead > 0) {
rdd.inputBytes += blockResult.inputMetrics.bytesRead
}
new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]])

case None =>
Expand Down
10 changes: 10 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -757,6 +757,16 @@ class SparkContext(config: SparkConf) extends Logging {
def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
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)
}

/**
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

* Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values
* with `+=`. Only the driver can access the accumuable's `value`.
Expand Down
4 changes: 4 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,9 @@ class HadoopRDD[K, V](
minPartitions)
}

private val accName = s"rdd-$id.input.bytes.hadoop"
val hadoopInputBytes = sc.accumulator(0L, accName)(SparkContext.LongAccumulatorParam)

protected val jobConfCacheKey = "rdd_%d_job_conf".format(id)

protected val inputFormatCacheKey = "rdd_%d_input_format".format(id)
Expand Down Expand Up @@ -205,6 +208,7 @@ class HadoopRDD[K, V](
* always at record boundaries, so tasks may need to read into other splits to complete
* a record. */
inputMetrics.bytesRead = split.inputSplit.value.getLength()
hadoopInputBytes += split.inputSplit.value.getLength()
} catch {
case e: java.io.IOException =>
logWarning("Unable to get input size to set InputMetrics for task", e)
Expand Down
6 changes: 6 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}
19 changes: 17 additions & 2 deletions core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
Original file line number Diff line number Diff line change
Expand Up @@ -791,9 +791,10 @@ class DAGScheduler(
val task = event.task
val stageId = task.stageId
val taskType = Utils.getFormattedClassName(task)
listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo,
event.taskMetrics))

if (!stageIdToStage.contains(task.stageId)) {
listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo,
event.taskMetrics))
// Skip all the actions if the stage has been cancelled.
return
}
Expand All @@ -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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 (partialValue != acc.zero) {
val stringPartialValue = s"${partialValue}"
val stringValue = s"${acc.value}"
stageToInfos(stage).accumulatedValues(name) = stringValue
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

event.taskInfo.accumulableValues += ((name, stringPartialValue))
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can this be moved to a method on the Accumulators companion object or something? These details about AccumulableInfo, prettyPartialValues, etc. aren't things that need to appear in the DAGScheduler.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@markhamstra yeah we can move these elsewhere, good idea.

}
pendingTasks(stage) -= task
task match {
Expand Down Expand Up @@ -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))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

submitWaitingStages()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@

package org.apache.spark.scheduler

import scala.collection.mutable.HashMap
import scala.collection.mutable.Map

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.storage.RDDInfo

Expand All @@ -37,6 +40,8 @@ class StageInfo(
var completionTime: Option[Long] = None
/** If the stage failed, the reason why. */
var failureReason: Option[String] = None
/** Terminal values of accumulables updated during this stage. */
val accumulatedValues: Map[String, String] = HashMap[String, String]()

def stageFailed(reason: String) {
failureReason = Some(reason)
Expand Down
8 changes: 8 additions & 0 deletions core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.scheduler

import scala.collection.mutable.ListBuffer

import org.apache.spark.annotation.DeveloperApi

/**
Expand All @@ -41,6 +43,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.
*/
val accumulableValues = ListBuffer[(String, String)]()

/**
* The time when the task has completed successfully (including the time to remotely fetch
* results, if necessary).
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.ui.jobs

import scala.collection.mutable.{HashMap, ListBuffer}
import scala.collection.mutable.{HashMap, ListBuffer, Map}

import org.apache.spark._
import org.apache.spark.annotation.DeveloperApi
Expand Down Expand Up @@ -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]]()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

val stageIdToInputBytes = HashMap[Int, Long]()
val stageIdToShuffleRead = HashMap[Int, Long]()
val stageIdToShuffleWrite = HashMap[Int, Long]()
Expand All @@ -73,6 +74,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener {
val stageId = stage.stageId
// Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage
poolToActiveStages(stageIdToPool(stageId)).remove(stageId)

val accumulables = stageIdToAccumulables.getOrElseUpdate(stageId, HashMap[String, String]())
for ((name, value) <- stageCompleted.stageInfo.accumulatedValues) {
accumulables(name) = value
}

activeStages.remove(stageId)
if (stage.failureReason.isEmpty) {
completedStages += stage
Expand All @@ -89,6 +96,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener {
val toRemove = math.max(retainedStages / 10, 1)
stages.take(toRemove).foreach { s =>
stageIdToTime.remove(s.stageId)
stageIdToAccumulables.remove(s.stageId)
stageIdToInputBytes.remove(s.stageId)
stageIdToShuffleRead.remove(s.stageId)
stageIdToShuffleWrite.remove(s.stageId)
Expand Down Expand Up @@ -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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}

// create executor summary map if necessary
val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid,
op = new HashMap[String, ExecutorSummary]())
Expand Down
14 changes: 12 additions & 2 deletions core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: alphabetization


import org.apache.spark.ui.{WebUIPage, UIUtils}
import org.apache.spark.util.{Utils, Distribution}
Expand Down Expand Up @@ -57,6 +57,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
val memoryBytesSpilled = listener.stageIdToMemoryBytesSpilled.getOrElse(stageId, 0L)
val diskBytesSpilled = listener.stageIdToDiskBytesSpilled.getOrElse(stageId, 0L)
val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0
val accumulables = listener.stageIdToAccumulables(stageId)

var activeTime = 0L
val now = System.currentTimeMillis
Expand Down Expand Up @@ -102,10 +103,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
</ul>
</div>
// scalastyle:on
val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
def accumulableRow(acc: (String, String)) = <tr><td>{acc._1}</td><td>{acc._2}</td></tr>
val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow,
accumulables.toSeq)

val taskHeaders: Seq[String] =
Seq(
"Index", "ID", "Attempt", "Status", "Locality Level", "Executor",
"Launch Time", "Duration", "GC Time") ++
"Launch Time", "Duration", "GC Time", "Accumulators") ++
{if (hasInput) Seq("Input") else Nil} ++
{if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++
{if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++
Expand Down Expand Up @@ -217,6 +223,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
<h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
<div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
<h4>Aggregated Metrics by Executor</h4> ++ executorTable.toNodeSeq ++
<h4>Accumulators</h4> ++ accumulableTable ++
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kayousterhout yea -good idea

<h4>Tasks</h4> ++ taskTable

UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId),
Expand Down Expand Up @@ -283,6 +290,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
<td sorttable_customkey={gcTime.toString}>
{if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""}
</td>
<td>
{Unparsed(info.accumulableValues.map{ case (k, v) => s"$k: $v" }.mkString("<br/>"))}
</td>
<!--
TODO: Add this back after we add support to hide certain columns.
<td sorttable_customkey={serializationTime.toString}>
Expand Down
22 changes: 20 additions & 2 deletions core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
Original file line number Diff line number Diff line change
Expand Up @@ -190,10 +190,14 @@ private[spark] object JsonProtocol {
("Details" -> stageInfo.details) ~
("Submission Time" -> submissionTime) ~
("Completion Time" -> completionTime) ~
("Failure Reason" -> failureReason)
("Failure Reason" -> failureReason) ~
("Accumulated Values" -> mapToJson(stageInfo.accumulatedValues))
}

def taskInfoToJson(taskInfo: TaskInfo): JValue = {
val accumUpdateMap = taskInfo.accumulableValues.map { case (k, v) =>
mapToJson(Map(k -> v))
}.toList
("Task ID" -> taskInfo.taskId) ~
("Index" -> taskInfo.index) ~
("Attempt" -> taskInfo.attempt) ~
Expand All @@ -204,7 +208,8 @@ private[spark] object JsonProtocol {
("Speculative" -> taskInfo.speculative) ~
("Getting Result Time" -> taskInfo.gettingResultTime) ~
("Finish Time" -> taskInfo.finishTime) ~
("Failed" -> taskInfo.failed)
("Failed" -> taskInfo.failed) ~
("Accumulable Updates" -> JArray(accumUpdateMap))
}

def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = {
Expand Down Expand Up @@ -485,11 +490,17 @@ private[spark] object JsonProtocol {
val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long])
val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long])
val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String])
val accumulatedValues = (json \ "Accumulated Values").extractOpt[JObject].map(mapFromJson(_))

val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos, details)
stageInfo.submissionTime = submissionTime
stageInfo.completionTime = completionTime
stageInfo.failureReason = failureReason
accumulatedValues.foreach { values =>
for ((k, v) <- values) {
stageInfo.accumulatedValues(k) = v
}
}
stageInfo
}

Expand All @@ -505,12 +516,19 @@ private[spark] object JsonProtocol {
val gettingResultTime = (json \ "Getting Result Time").extract[Long]
val finishTime = (json \ "Finish Time").extract[Long]
val failed = (json \ "Failed").extract[Boolean]
val accumulableUpdates = (json \ "Accumulable Updates").extractOpt[Seq[JValue]].map(
updates => updates.map(mapFromJson(_)))

val taskInfo =
new TaskInfo(taskId, index, attempt, launchTime, executorId, host, taskLocality, speculative)
taskInfo.gettingResultTime = gettingResultTime
taskInfo.finishTime = finishTime
taskInfo.failed = failed
accumulableUpdates.foreach { maps =>
for (m <- maps) {
taskInfo.accumulableValues += m.head
}
}
taskInfo
}

Expand Down
Loading