From a459f4956879c3fbf2e1c52350ef9f667f70d135 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 28 Apr 2015 16:54:37 -0700 Subject: [PATCH] Add a dash line to processing time graphs --- .../apache/spark/ui/static/streaming-page.css | 8 +++ .../apache/spark/ui/static/streaming-page.js | 60 ++++++++++++++++++- .../scala/org/apache/spark/ui/UIUtils.scala | 6 +- .../spark/streaming/ui/StreamingPage.scala | 27 ++++++--- 4 files changed, 87 insertions(+), 14 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css index 6b401c6dec76e..357a0b61a3b9e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css +++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css @@ -30,6 +30,10 @@ fill: gray; } +.tooltip-inner { + max-width: 500px !important; +} + .line { fill: none; stroke: #0088cc; @@ -47,6 +51,10 @@ fill: rgb(0, 194, 255); } +.stable-text text:hover { + fill: #0088cc; +} + .timeline { width: 500px; } diff --git a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js index 308ae0416a529..d473d0dd16f59 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js +++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js @@ -63,7 +63,7 @@ function hideBootstrapTooltip(node) { * @param maxY the max value of Y axis * @param unitY the unit of Y axis */ -function drawTimeline(id, data, minX, maxX, minY, maxY, unitY) { +function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchTime) { d3.select(d3.select(id).node().parentNode).style("padding", "8px 0 8px 8px").style("border-right", "0px solid white"); var margin = {top: 20, right: 27, bottom: 30, left: timelineMarginLeft}; var width = 500 - margin.left - margin.right; @@ -101,6 +101,26 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY) { .attr("transform", "translate(0," + (-3) + ")") .text(unitY); + + if (batchTime) { + var batchTimeline = d3.svg.line() + .x(function(d) { return x(d.x); }) + .y(function(d) { return y(d.y); }); + + console.log(batchTime); + var batchTimeData = [ + {x: minX, y: batchTime}, {x: maxX, y: batchTime} + ]; + console.log(batchTimeData); + + svg.append("path") + .datum(batchTimeData) + .style("stroke-dasharray", ("6, 6")) + .style("stroke", "lightblue") + .attr("class", "line") + .attr("d", batchTimeline); + } + svg.append("path") .datum(data) .attr("class", "line") @@ -136,6 +156,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY) { .attr("fill", "white") .attr("opacity", "0"); }); + } /** @@ -145,7 +166,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY) { * @param maxY the max value of Y axis * @param unitY the unit of Y axis */ -function drawDistribution(id, values, minY, maxY, unitY) { +function drawDistribution(id, values, minY, maxY, unitY, batchTime) { d3.select(d3.select(id).node().parentNode).style("padding", "8px 8px 8px 0").style("border-left", "0px solid white"); var margin = {top: 20, right: 30, bottom: 30, left: 10}; var width = 300 - margin.left - margin.right; @@ -169,6 +190,25 @@ function drawDistribution(id, values, minY, maxY, unitY) { .append("g") .attr("transform", "translate(" + margin.left + "," + margin.top + ")"); + if (batchTime) { + var batchTimeline = d3.svg.line() + .x(function(d) { return x(d.x); }) + .y(function(d) { return y(d.y); }); + + console.log(batchTime); + var batchTimeData = [ + {x: distributionMinX, y: batchTime}, {x: distributionMaxX, y: batchTime} + ]; + console.log(batchTimeData); + + svg.append("path") + .datum(batchTimeData) + .style("stroke-dasharray", ("6, 6")) + .style("stroke", "lightblue") + .attr("class", "line") + .attr("d", batchTimeline); + } + svg.append("g") .attr("class", "x axis") .attr("transform", "translate(0," + 0 + ")") @@ -208,6 +248,22 @@ function drawDistribution(id, values, minY, maxY, unitY) { hideBootstrapTooltip(d3.select(this).node()); //hideGraphTooltip(); }); + + if (batchTime && batchTime <= maxY) { + svg.append("text") + .style("fill", "lightblue") + .attr("class", "stable-text") + .attr("text-anchor", "middle") + .attr("transform", "translate(" + (x(distributionMaxX)-20) +"," + (y(batchTime) + 15) + ")") + .text("stable") + .on('mouseover', function(d) { + var tip = "Processing Time <= Batch Interval (" + formatBinValue(batchTime) +" " + unitY +")"; + showBootstrapTooltip(d3.select(this).node(), tip); + }) + .on('mouseout', function() { + hideBootstrapTooltip(d3.select(this).node()); + }); + } } function prepareTimeline(minY, maxY) { diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 827b945ec2787..db0836533388b 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -42,10 +42,10 @@ private[spark] object UIUtils extends Logging { case TimeUnit.NANOSECONDS => "ns" case TimeUnit.MICROSECONDS => "us" case TimeUnit.MILLISECONDS => "ms" - case TimeUnit.SECONDS => "s" + case TimeUnit.SECONDS => "sec" case TimeUnit.MINUTES => "min" - case TimeUnit.HOURS => "h" - case TimeUnit.DAYS => "d" + case TimeUnit.HOURS => "hrs" + case TimeUnit.DAYS => "days" } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index d61662500b83e..4ff77e6c09141 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -40,16 +40,20 @@ import org.apache.spark.util.Distribution * @param unitY the unit of Y axis */ private[ui] class TimelineUIData(divId: String, data: Seq[(Long, _)], minX: Long, maxX: Long, - minY: Double, maxY: Double, unitY: String) { + minY: Double, maxY: Double, unitY: String, batchTime: Option[Double] = None) { def toHtml(jsCollector: JsCollector): Seq[Node] = { val jsForData = data.map { case (x, y) => s"""{"x": $x, "y": $y}""" }.mkString("[", ",", "]") jsCollector.addPreparedStatement(s"prepareTimeline($minY, $maxY);") - jsCollector.addStatement( - s"drawTimeline('#$divId', $jsForData, $minX, $maxX, $minY, $maxY, '$unitY');") - + if (batchTime.isDefined) { + jsCollector.addStatement( + s"drawTimeline('#$divId', $jsForData, $minX, $maxX, $minY, $maxY, '$unitY', ${batchTime.get});") + } else { + jsCollector.addStatement( + s"drawTimeline('#$divId', $jsForData, $minX, $maxX, $minY, $maxY, '$unitY');") + }
} } @@ -62,13 +66,16 @@ private[ui] class TimelineUIData(divId: String, data: Seq[(Long, _)], minX: Long * @param unitY the unit of Y axis */ private[ui] class DistributionUIData( - divId: String, data: Seq[_], minY: Double, maxY: Double, unitY: String) { + divId: String, data: Seq[_], minY: Double, maxY: Double, unitY: String, batchTime: Option[Double] = None) { def toHtml(jsCollector: JsCollector): Seq[Node] = { val jsForData = data.mkString("[", ",", "]") jsCollector.addPreparedStatement(s"prepareDistribution($jsForData, $minY, $maxY);") - jsCollector.addStatement(s"drawDistribution('#$divId', $jsForData, $minY, $maxY, '$unitY');") - + if (batchTime.isDefined) { + jsCollector.addStatement(s"drawDistribution('#$divId', $jsForData, $minY, $maxY, '$unitY', ${batchTime.get});") + } else { + jsCollector.addStatement(s"drawDistribution('#$divId', $jsForData, $minY, $maxY, '$unitY');") + }
} } @@ -204,6 +211,8 @@ private[ui] class StreamingPage(parent: StreamingTab) |else {$$(this).html('$BLACK_RIGHT_TRIANGLE_HTML');status = false;} |window.history.pushState('', document.title, window.location.pathname + '?show-receivers-detail=' + status);""".stripMargin.replaceAll("\\n", "") + var batchTime = StreamingPage.convertToTimeUnit(listener.batchDuration, unit) + val timelineDataForEventRateOfAllReceivers = new TimelineUIData( "all-receiver-events-timeline", @@ -248,7 +257,7 @@ private[ui] class StreamingPage(parent: StreamingTab) maxBatchTime, minTime, maxTime, - formattedUnit).toHtml(jsCollector) + formattedUnit, Some(batchTime)).toHtml(jsCollector) val distributionDataForProcessingTime = new DistributionUIData( @@ -256,7 +265,7 @@ private[ui] class StreamingPage(parent: StreamingTab) processingTime.distributionData(unit), minTime, maxTime, - formattedUnit).toHtml(jsCollector) + formattedUnit, Some(batchTime)).toHtml(jsCollector) val timelineDataForTotalDelay = new TimelineUIData(