|
17 | 17 |
|
18 | 18 | package org.apache.spark.ui.viz |
19 | 19 |
|
| 20 | +import scala.collection.mutable |
| 21 | +import scala.collection.mutable.ArrayBuffer |
| 22 | + |
| 23 | +import org.apache.spark.rdd.RDD |
20 | 24 | import org.apache.spark.scheduler._ |
| 25 | +import org.apache.spark.storage.RDDInfo |
21 | 26 |
|
22 | 27 | /** |
23 | 28 | * A SparkListener that... |
24 | 29 | */ |
25 | 30 | private[spark] class VisualizationListener extends SparkListener { |
| 31 | + private val graphsByStageId = new mutable.HashMap[Int, VizGraph] // stage ID -> viz graph |
| 32 | + |
| 33 | + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { |
| 34 | + val stageId = stageSubmitted.stageInfo.stageId |
| 35 | + val rddInfos = stageSubmitted.stageInfo.rddInfos |
| 36 | + |
| 37 | + println(s"******** STAGE $stageId **********") |
| 38 | + rddInfos.foreach(println) |
| 39 | + |
| 40 | + val vizGraph = makeVizGraph(rddInfos) |
| 41 | + graphsByStageId(stageId) = vizGraph |
| 42 | + |
| 43 | + println( |
| 44 | + s""" |
| 45 | + |===================================================== |
| 46 | + |${VisualizationListener.makeDotFile(vizGraph)} |
| 47 | + |===================================================== |
| 48 | + """.stripMargin) |
| 49 | + } |
| 50 | + |
| 51 | + /** |
| 52 | + * |
| 53 | + */ |
| 54 | + private def makeVizGraph(rddInfos: Seq[RDDInfo]): VizGraph = { |
| 55 | + val edges = new mutable.HashSet[VizEdge] |
| 56 | + val nodes = new mutable.HashMap[Int, VizNode] |
| 57 | + val scopes = new mutable.HashMap[String, VizScope] // scope ID -> viz scope |
| 58 | + |
| 59 | + // Entities that are not part of any scopes |
| 60 | + val rootNodes = new ArrayBuffer[VizNode] |
| 61 | + val rootScopes = new mutable.HashSet[VizScope] |
| 62 | + |
| 63 | + // Populate nodes, edges, and scopes |
| 64 | + rddInfos.foreach { rdd => |
| 65 | + val node = nodes.getOrElseUpdate(rdd.id, VizNode(rdd.id, rdd.name)) |
| 66 | + edges ++= rdd.parentIds.map { parentId => VizEdge(parentId, rdd.id) } |
| 67 | + |
| 68 | + if (rdd.scope == null) { |
| 69 | + // There is no encompassing scope, so this is a root node |
| 70 | + rootNodes += node |
| 71 | + } else { |
| 72 | + // Attach children scopes and nodes to each scope |
| 73 | + var previousScope: VizScope = null |
| 74 | + val scopeIt = rdd.scope.split(RDD.SCOPE_NESTING_DELIMITER).iterator |
| 75 | + while (scopeIt.hasNext) { |
| 76 | + val scopeId = scopeIt.next() |
| 77 | + val scope = scopes.getOrElseUpdate(scopeId, new VizScope(scopeId)) |
| 78 | + scope.attachChildNode(node) |
| 79 | + // RDD scopes are hierarchical, with the outermost scopes ordered first |
| 80 | + // If there is not a previous scope, then this must be a root scope |
| 81 | + if (previousScope == null) { |
| 82 | + rootScopes += scope |
| 83 | + } else { |
| 84 | + // Otherwise, attach this scope to its parent |
| 85 | + previousScope.attachChildScope(scope) |
| 86 | + } |
| 87 | + previousScope = scope |
| 88 | + } |
| 89 | + } |
| 90 | + } |
| 91 | + |
| 92 | + // Remove any edges with nodes belonging to other stages so we do not have orphaned nodes |
| 93 | + edges.retain { case VizEdge(f, t) => nodes.contains(f) && nodes.contains(t) } |
| 94 | + |
| 95 | + new VizGraph(edges.toSeq, rootNodes, rootScopes.toSeq) |
| 96 | + } |
| 97 | +} |
26 | 98 |
|
27 | | - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { |
28 | | - synchronized { |
29 | | - stageSubmitted.stageInfo.rddInfos |
| 99 | +private object VisualizationListener { |
| 100 | + |
| 101 | + /** |
| 102 | + * |
| 103 | + */ |
| 104 | + def makeDotFile(graph: VizGraph): String = { |
| 105 | + val dotFile = new StringBuilder |
| 106 | + dotFile.append("digraph G {\n") |
| 107 | + // |
| 108 | + graph.rootScopes.foreach { scope => |
| 109 | + dotFile.append(makeDotSubgraph(scope, " ")) |
| 110 | + } |
| 111 | + // |
| 112 | + graph.rootNodes.foreach { node => |
| 113 | + dotFile.append(" " + makeDotNode(node) + "\n") |
30 | 114 | } |
| 115 | + // |
| 116 | + graph.edges.foreach { edge => |
| 117 | + dotFile.append(" " + edge.fromId + "->" + edge.toId + "\n") |
| 118 | + } |
| 119 | + dotFile.append("}") |
| 120 | + dotFile.toString() |
| 121 | + } |
| 122 | + |
| 123 | + /** |
| 124 | + * |
| 125 | + */ |
| 126 | + private def makeDotNode(node: VizNode): String = { |
| 127 | + node.id + " [label = \"" + node.name + "\"]" |
| 128 | + } |
| 129 | + |
| 130 | + /** |
| 131 | + * |
| 132 | + */ |
| 133 | + private def makeDotSubgraph(scope: VizScope, indent: String): String = { |
| 134 | + val subgraph = new StringBuilder |
| 135 | + subgraph.append(indent + "subgraph cluster" + scope.id + "{\n") |
| 136 | + subgraph.append(indent + " label = \"" + scope.name + "\"\n") |
| 137 | + scope.childrenNodes.foreach { node => |
| 138 | + subgraph.append(indent + " " + makeDotNode(node) + "\n") |
| 139 | + } |
| 140 | + scope.childrenScopes.foreach { cscope => |
| 141 | + subgraph.append(makeDotSubgraph(cscope, indent + " ")) |
| 142 | + } |
| 143 | + subgraph.append(indent + "}\n") |
| 144 | + subgraph.toString() |
31 | 145 | } |
32 | 146 | } |
| 147 | + |
| 148 | +private case class VizNode(id: Int, name: String) |
| 149 | +private case class VizEdge(fromId: Int, toId: Int) |
| 150 | + |
| 151 | +private class VizScope(val id: String) { |
| 152 | + private val _childrenNodes = new ArrayBuffer[VizNode] |
| 153 | + private val _childrenScopes = new ArrayBuffer[VizScope] |
| 154 | + val name: String = id.split(RDD.SCOPE_NAME_DELIMITER).head |
| 155 | + |
| 156 | + def childrenNodes: Seq[VizNode] = _childrenNodes.iterator.toSeq |
| 157 | + def childrenScopes: Seq[VizScope] = _childrenScopes.iterator.toSeq |
| 158 | + |
| 159 | + def attachChildNode(childNode: VizNode): Unit = { _childrenNodes += childNode } |
| 160 | + def attachChildScope(childScope: VizScope): Unit = { _childrenScopes += childScope } |
| 161 | +} |
| 162 | + |
| 163 | +private case class VizGraph( |
| 164 | + edges: Seq[VizEdge], |
| 165 | + rootNodes: Seq[VizNode], |
| 166 | + rootScopes: Seq[VizScope]) |
0 commit comments