Skip to content

Commit

Permalink
Add a dash line to processing time graphs
Browse files Browse the repository at this point in the history
  • Loading branch information
zsxwing committed Apr 29, 2015
1 parent 8e4363c commit a459f49
Show file tree
Hide file tree
Showing 4 changed files with 87 additions and 14 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,10 @@
fill: gray;
}

.tooltip-inner {
max-width: 500px !important;
}

.line {
fill: none;
stroke: #0088cc;
Expand All @@ -47,6 +51,10 @@
fill: rgb(0, 194, 255);
}

.stable-text text:hover {
fill: #0088cc;
}

.timeline {
width: 500px;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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")
Expand Down Expand Up @@ -136,6 +156,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY) {
.attr("fill", "white")
.attr("opacity", "0");
});

}

/**
Expand All @@ -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;
Expand All @@ -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 + ")")
Expand Down Expand Up @@ -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) {
Expand Down
6 changes: 3 additions & 3 deletions core/src/main/scala/org/apache/spark/ui/UIUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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"
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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');")
}
<div id={divId}></div>
}
}
Expand All @@ -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');")
}
<div id={divId}></div>
}
}
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -248,15 +257,15 @@ private[ui] class StreamingPage(parent: StreamingTab)
maxBatchTime,
minTime,
maxTime,
formattedUnit).toHtml(jsCollector)
formattedUnit, Some(batchTime)).toHtml(jsCollector)

val distributionDataForProcessingTime =
new DistributionUIData(
"processing-time-distribution",
processingTime.distributionData(unit),
minTime,
maxTime,
formattedUnit).toHtml(jsCollector)
formattedUnit, Some(batchTime)).toHtml(jsCollector)

val timelineDataForTotalDelay =
new TimelineUIData(
Expand Down

0 comments on commit a459f49

Please sign in to comment.