+ * div.stage-metadata >
+ * div.[dot-file | incoming-edge | outgoing-edge]
+ *
+ * Output DOM hierarchy:
+ * div#dag-viz-graph >
+ * svg >
+ * g#cluster_stage_[stageId]
+ *
+ * Note that the input metadata is populated by o.a.s.ui.UIUtils.showDagViz.
+ * Any changes in the input format here must be reflected there.
+ */
+function renderDagViz(forJob) {
+
+ // If there is not a dot file to render, fail fast and report error
+ if (metadataContainer().empty()) {
+ graphContainer().append("div").text(
+ "No visualization information available for this " + (forJob ? "job" : "stage"));
+ return;
+ }
+
+ var svg = graphContainer().append("svg");
+ if (forJob) {
+ renderDagVizForJob(svg);
+ } else {
+ renderDagVizForStage(svg);
+ }
+
+ // Find cached RDDs
+ metadataContainer().selectAll(".cached-rdd").each(function(v) {
+ var nodeId = VizConstants.nodePrefix + d3.select(this).text();
+ graphContainer().selectAll("#" + nodeId).classed("cached", true);
+ });
+
+ // Set the appropriate SVG dimensions to ensure that all elements are displayed
+ var boundingBox = svg.node().getBBox();
+ svg.style("width", (boundingBox.width + VizConstants.svgMarginX) + "px");
+ svg.style("height", (boundingBox.height + VizConstants.svgMarginY) + "px");
+
+ // Add labels to clusters because dagre-d3 doesn't do this for us
+ svg.selectAll("g.cluster rect").each(function() {
+ var rect = d3.select(this);
+ var cluster = d3.select(this.parentNode);
+ // Shift the boxes up a little to make room for the labels
+ rect.attr("y", toFloat(rect.attr("y")) - 10);
+ rect.attr("height", toFloat(rect.attr("height")) + 10);
+ var labelX = toFloat(rect.attr("x")) + toFloat(rect.attr("width")) - 5;
+ var labelY = toFloat(rect.attr("y")) + 15;
+ var labelText = cluster.attr("name").replace(VizConstants.clusterPrefix, "");
+ cluster.append("text")
+ .attr("x", labelX)
+ .attr("y", labelY)
+ .attr("text-anchor", "end")
+ .text(labelText);
+ });
+
+ // We have shifted a few elements upwards, so we should fix the SVG views
+ var startX = -VizConstants.svgMarginX;
+ var startY = -VizConstants.svgMarginY;
+ var endX = toFloat(svg.style("width")) + VizConstants.svgMarginX;
+ var endY = toFloat(svg.style("height")) + VizConstants.svgMarginY;
+ var newViewBox = startX + " " + startY + " " + endX + " " + endY;
+ svg.attr("viewBox", newViewBox);
+
+ // Lastly, apply some custom style to the DAG
+ styleDagViz(forJob);
+}
+
+/* Render the RDD DAG visualization for a stage. */
+function renderDagVizForStage(svgContainer) {
+ var metadata = metadataContainer().select(".stage-metadata");
+ var dot = metadata.select(".dot-file").text();
+ var containerId = VizConstants.graphPrefix + metadata.attr("stageId");
+ var container = svgContainer.append("g").attr("id", containerId);
+ renderDot(dot, container);
+}
+
+/*
+ * Render the RDD DAG visualization for a job.
+ *
+ * Due to limitations in dagre-d3, each stage is rendered independently so that
+ * we have more control on how to position them. Unfortunately, this means we
+ * cannot rely on dagre-d3 to render edges that cross stages and must render
+ * these manually on our own.
+ */
+function renderDagVizForJob(svgContainer) {
+ var crossStageEdges = [];
+
+ metadataContainer().selectAll(".stage-metadata").each(function(d, i) {
+ var metadata = d3.select(this);
+ var dot = metadata.select(".dot-file").text();
+ var stageId = metadata.attr("stageId");
+ var containerId = VizConstants.graphPrefix + stageId;
+ // TODO: handle stage attempts
+ var stageLink =
+ "/stages/stage/?id=" + stageId.replace(VizConstants.stagePrefix, "") + "&attempt=0";
+ var container = svgContainer
+ .append("a").attr("xlink:href", stageLink)
+ .append("g").attr("id", containerId);
+ // Now we need to shift the container for this stage so it doesn't overlap
+ // with existing ones. We do not need to do this for the first stage.
+ if (i > 0) {
+ // Take into account the position and width of the last stage's container
+ var existingStages = stageClusters();
+ if (!existingStages.empty()) {
+ var lastStage = existingStages[0].pop();
+ var lastStageId = d3.select(lastStage).attr("id");
+ var lastStageWidth = toFloat(d3.select("#" + lastStageId + " rect").attr("width"));
+ var lastStagePosition = getAbsolutePosition(lastStageId);
+ var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep;
+ container.attr("transform", "translate(" + offset + ", 0)");
+ }
+ }
+ renderDot(dot, container);
+ // If there are any incoming edges into this graph, keep track of them to render
+ // them separately later. Note that we cannot draw them now because we need to
+ // put these edges in a separate container that is on top of all stage graphs.
+ metadata.selectAll(".incoming-edge").each(function(v) {
+ var edge = d3.select(this).text().split(","); // e.g. 3,4 => [3, 4]
+ crossStageEdges.push(edge);
+ });
+ });
+
+ // Draw edges that cross stages
+ if (crossStageEdges.length > 0) {
+ var container = svgContainer.append("g").attr("id", "cross-stage-edges");
+ for (var i = 0; i < crossStageEdges.length; i++) {
+ var fromRDDId = crossStageEdges[i][0];
+ var toRDDId = crossStageEdges[i][1];
+ connectRDDs(fromRDDId, toRDDId, container);
+ }
+ }
+}
+
+/* Render the dot file as an SVG in the given container. */
+function renderDot(dot, container) {
+ var escaped_dot = dot
+ .replace(/</g, "<")
+ .replace(/>/g, ">")
+ .replace(/"/g, "\"");
+ var g = graphlibDot.read(escaped_dot);
+ var renderer = new dagreD3.render();
+ renderer(container, g);
+}
+
+/* Style the visualization we just rendered. */
+function styleDagViz(forJob) {
+ graphContainer().selectAll("svg g.cluster rect")
+ .style("fill", "white")
+ .style("stroke", VizConstants.rddOperationColor)
+ .style("stroke-width", "4px")
+ .style("stroke-opacity", "0.5");
+ graphContainer().selectAll("svg g.cluster text")
+ .attr("fill", VizConstants.clusterLabelColor)
+ .attr("font-size", "11px");
+ graphContainer().selectAll("svg path")
+ .style("stroke", VizConstants.edgeColor)
+ .style("stroke-width", VizConstants.edgeWidth);
+ stageClusters()
+ .select("rect")
+ .style("stroke", VizConstants.stageColor)
+ .style("strokeWidth", "6px");
+
+ // Put an arrow at the end of every edge
+ // We need to do this because we manually render some edges ourselves
+ // For these edges, we borrow the arrow marker generated by dagre-d3
+ var dagreD3Marker = graphContainer().select("svg g.edgePaths marker").node();
+ graphContainer().select("svg")
+ .append(function() { return dagreD3Marker.cloneNode(true); })
+ .attr("id", "marker-arrow")
+ .select("path")
+ .attr("fill", VizConstants.edgeColor)
+ .attr("strokeWidth", "0px");
+ graphContainer().selectAll("svg g > path").attr("marker-end", "url(#marker-arrow)");
+ graphContainer().selectAll("svg g.edgePaths def").remove(); // We no longer need these
+
+ // Apply any job or stage specific styles
+ if (forJob) {
+ styleDagVizForJob();
+ } else {
+ styleDagVizForStage();
+ }
+}
+
+/* Apply job-page-specific style to the visualization. */
+function styleDagVizForJob() {
+ graphContainer().selectAll("svg g.node circle")
+ .style("fill", VizConstants.rddColor);
+ // TODO: add a legend to explain what a highlighted dot means
+ graphContainer().selectAll("svg g.cached circle")
+ .style("fill", VizConstants.rddCachedColor);
+ graphContainer().selectAll("svg g#cross-stage-edges path")
+ .style("fill", "none");
+}
+
+/* Apply stage-page-specific style to the visualization. */
+function styleDagVizForStage() {
+ graphContainer().selectAll("svg g.node rect")
+ .style("fill", "none")
+ .style("stroke", VizConstants.rddColor)
+ .style("stroke-width", "2px")
+ .attr("rx", "5") // round corners
+ .attr("ry", "5");
+ // TODO: add a legend to explain what a highlighted RDD means
+ graphContainer().selectAll("svg g.cached rect")
+ .style("stroke", VizConstants.rddCachedColor);
+ graphContainer().selectAll("svg g.node g.label text tspan")
+ .style("fill", VizConstants.rddColor);
+}
+
+/*
+ * (Job page only) Helper method to compute the absolute
+ * position of the group element identified by the given ID.
+ */
+function getAbsolutePosition(groupId) {
+ var obj = d3.select("#" + groupId).filter("g");
+ var _x = 0, _y = 0;
+ while (!obj.empty()) {
+ var transformText = obj.attr("transform");
+ var translate = d3.transform(transformText).translate
+ _x += translate[0];
+ _y += translate[1];
+ obj = d3.select(obj.node().parentNode).filter("g")
+ }
+ return { x: _x, y: _y };
+}
+
+/* (Job page only) Connect two RDD nodes with a curved edge. */
+function connectRDDs(fromRDDId, toRDDId, container) {
+ var fromNodeId = VizConstants.nodePrefix + fromRDDId;
+ var toNodeId = VizConstants.nodePrefix + toRDDId
+ var fromPos = getAbsolutePosition(fromNodeId);
+ var toPos = getAbsolutePosition(toNodeId);
+
+ // On the job page, RDDs are rendered as dots (circles). When rendering the path,
+ // we need to account for the radii of these circles. Otherwise the arrow heads
+ // will bleed into the circle itself.
+ var delta = toFloat(graphContainer()
+ .select("g.node#" + toNodeId)
+ .select("circle")
+ .attr("r"));
+ if (fromPos.x < toPos.x) {
+ fromPos.x += delta;
+ toPos.x -= delta;
+ } else if (fromPos.x > toPos.x) {
+ fromPos.x -= delta;
+ toPos.x += delta;
+ }
+
+ if (fromPos.y == toPos.y) {
+ // If they are on the same rank, curve the middle part of the edge
+ // upward a little to avoid interference with things in between
+ // e.g. _______
+ // _____/ \_____
+ var points = [
+ [fromPos.x, fromPos.y],
+ [fromPos.x + (toPos.x - fromPos.x) * 0.2, fromPos.y],
+ [fromPos.x + (toPos.x - fromPos.x) * 0.3, fromPos.y - 20],
+ [fromPos.x + (toPos.x - fromPos.x) * 0.7, fromPos.y - 20],
+ [fromPos.x + (toPos.x - fromPos.x) * 0.8, toPos.y],
+ [toPos.x, toPos.y]
+ ];
+ } else {
+ // Otherwise, draw a curved edge that flattens out on both ends
+ // e.g. _____
+ // /
+ // |
+ // _____/
+ var points = [
+ [fromPos.x, fromPos.y],
+ [fromPos.x + (toPos.x - fromPos.x) * 0.4, fromPos.y],
+ [fromPos.x + (toPos.x - fromPos.x) * 0.6, toPos.y],
+ [toPos.x, toPos.y]
+ ];
+ }
+
+ var line = d3.svg.line().interpolate("basis");
+ container.append("path").datum(points).attr("d", line);
+}
+
+/* Helper d3 accessor to clusters that represent stages. */
+function stageClusters() {
+ return graphContainer().selectAll("g.cluster").filter(function() {
+ return d3.select(this).attr("id").indexOf(VizConstants.stageClusterPrefix) > -1;
+ });
+}
+
+/* Helper method to convert attributes to numeric values. */
+function toFloat(f) {
+ return parseFloat(f.replace(/px$/, ""));
+}
+
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
new file mode 100644
index 0000000000000..5da9d631ad124
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+.graph {
+ font: 10px sans-serif;
+}
+
+.axis path, .axis line {
+ fill: none;
+ stroke: gray;
+ shape-rendering: crispEdges;
+}
+
+.axis text {
+ fill: gray;
+}
+
+.tooltip-inner {
+ max-width: 500px !important; // Make sure we only have one line tooltip
+}
+
+.line {
+ fill: none;
+ stroke: #0088cc;
+ stroke-width: 1.5px;
+}
+
+.bar rect {
+ fill: #0088cc;
+ shape-rendering: crispEdges;
+}
+
+.bar rect:hover {
+ fill: #00c2ff;
+}
+
+.timeline {
+ width: 500px;
+}
+
+.histogram {
+ width: auto;
+}
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
new file mode 100644
index 0000000000000..a4e03b156f13e
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+// timeFormat: StreamingPage.scala will generate a global "timeFormat" dictionary to store the time
+// and its formatted string. Because we cannot specify a timezone in JavaScript, to make sure the
+// server and client use the same timezone, we use the "timeFormat" dictionary to format all time
+// values used in the graphs.
+
+// A global margin left for all timeline graphs. It will be set in "registerTimeline". This will be
+// used to align all timeline graphs.
+var maxMarginLeftForTimeline = 0;
+
+// The max X values for all histograms. It will be set in "registerHistogram".
+var maxXForHistogram = 0;
+
+var histogramBinCount = 10;
+var yValueFormat = d3.format(",.2f");
+
+// Show a tooltip "text" for "node"
+function showBootstrapTooltip(node, text) {
+ $(node).tooltip({title: text, trigger: "manual", container: "body"});
+ $(node).tooltip("show");
+}
+
+// Hide the tooltip for "node"
+function hideBootstrapTooltip(node) {
+ $(node).tooltip("destroy");
+}
+
+// Register a timeline graph. All timeline graphs should be register before calling any
+// "drawTimeline" so that we can determine the max margin left for all timeline graphs.
+function registerTimeline(minY, maxY) {
+ var numOfChars = yValueFormat(maxY).length;
+ // A least width for "maxY" in the graph
+ var pxForMaxY = numOfChars * 8 + 10;
+ // Make sure we have enough space to show the ticks in the y axis of timeline
+ maxMarginLeftForTimeline = pxForMaxY > maxMarginLeftForTimeline? pxForMaxY : maxMarginLeftForTimeline;
+}
+
+// Register a histogram graph. All histogram graphs should be register before calling any
+// "drawHistogram" so that we can determine the max X value for histograms.
+function registerHistogram(values, minY, maxY) {
+ var data = d3.layout.histogram().range([minY, maxY]).bins(histogramBinCount)(values);
+ // d.x is the y values while d.y is the x values
+ var maxX = d3.max(data, function(d) { return d.y; });
+ maxXForHistogram = maxX > maxXForHistogram ? maxX : maxXForHistogram;
+}
+
+// Draw a line between (x1, y1) and (x2, y2)
+function drawLine(svg, xFunc, yFunc, x1, y1, x2, y2) {
+ var line = d3.svg.line()
+ .x(function(d) { return xFunc(d.x); })
+ .y(function(d) { return yFunc(d.y); });
+ var data = [{x: x1, y: y1}, {x: x2, y: y2}];
+ svg.append("path")
+ .datum(data)
+ .style("stroke-dasharray", ("6, 6"))
+ .style("stroke", "lightblue")
+ .attr("class", "line")
+ .attr("d", line);
+}
+
+/**
+ * @param id the `id` used in the html `div` tag
+ * @param data the data for the timeline graph
+ * @param minX the min value of X axis
+ * @param maxX the max value of X axis
+ * @param minY the min value of Y axis
+ * @param maxY the max value of Y axis
+ * @param unitY the unit of Y axis
+ * @param batchInterval if "batchInterval" is specified, we will draw a line for "batchInterval" in the graph
+ */
+function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) {
+ // Hide the right border of "". We cannot use "css" directly, or "sorttable.js" will override them.
+ 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: maxMarginLeftForTimeline};
+ var width = 500 - margin.left - margin.right;
+ var height = 150 - margin.top - margin.bottom;
+
+ var x = d3.scale.linear().domain([minX, maxX]).range([0, width]);
+ var y = d3.scale.linear().domain([minY, maxY]).range([height, 0]);
+
+ var xAxis = d3.svg.axis().scale(x).orient("bottom").tickFormat(function(d) { return timeFormat[d]; });
+ var formatYValue = d3.format(",.2f");
+ var yAxis = d3.svg.axis().scale(y).orient("left").ticks(5).tickFormat(formatYValue);
+
+ var line = d3.svg.line()
+ .x(function(d) { return x(d.x); })
+ .y(function(d) { return y(d.y); });
+
+ var svg = d3.select(id).append("svg")
+ .attr("width", width + margin.left + margin.right)
+ .attr("height", height + margin.top + margin.bottom)
+ .append("g")
+ .attr("transform", "translate(" + margin.left + "," + margin.top + ")");
+
+ // Only show the first and last time in the graph
+ xAxis.tickValues(x.domain());
+
+ svg.append("g")
+ .attr("class", "x axis")
+ .attr("transform", "translate(0," + height + ")")
+ .call(xAxis)
+
+ svg.append("g")
+ .attr("class", "y axis")
+ .call(yAxis)
+ .append("text")
+ .attr("transform", "translate(0," + (-3) + ")")
+ .text(unitY);
+
+
+ if (batchInterval && batchInterval <= maxY) {
+ drawLine(svg, x, y, minX, batchInterval, maxX, batchInterval);
+ }
+
+ svg.append("path")
+ .datum(data)
+ .attr("class", "line")
+ .attr("d", line);
+
+ // Add points to the line. However, we make it invisible at first. But when the user moves mouse
+ // over a point, it will be displayed with its detail.
+ svg.selectAll(".point")
+ .data(data)
+ .enter().append("circle")
+ .attr("stroke", "white") // white and opacity = 0 make it invisible
+ .attr("fill", "white")
+ .attr("opacity", "0")
+ .attr("cx", function(d) { return x(d.x); })
+ .attr("cy", function(d) { return y(d.y); })
+ .attr("r", function(d) { return 3; })
+ .on('mouseover', function(d) {
+ var tip = formatYValue(d.y) + " " + unitY + " at " + timeFormat[d.x];
+ showBootstrapTooltip(d3.select(this).node(), tip);
+ // show the point
+ d3.select(this)
+ .attr("stroke", "steelblue")
+ .attr("fill", "steelblue")
+ .attr("opacity", "1");
+ })
+ .on('mouseout', function() {
+ hideBootstrapTooltip(d3.select(this).node());
+ // hide the point
+ d3.select(this)
+ .attr("stroke", "white")
+ .attr("fill", "white")
+ .attr("opacity", "0");
+ })
+ .on("click", function(d) {
+ window.location.href = "batch/?id=" + d.x;
+ });
+}
+
+/**
+ * @param id the `id` used in the html `div` tag
+ * @param values the data for the histogram graph
+ * @param minY the min value of Y axis
+ * @param maxY the max value of Y axis
+ * @param unitY the unit of Y axis
+ * @param batchInterval if "batchInterval" is specified, we will draw a line for "batchInterval" in the graph
+ */
+function drawHistogram(id, values, minY, maxY, unitY, batchInterval) {
+ // Hide the left border of " ". We cannot use "css" directly, or "sorttable.js" will override them.
+ 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;
+ var height = 150 - margin.top - margin.bottom;
+
+ var x = d3.scale.linear().domain([0, maxXForHistogram]).range([0, width]);
+ var y = d3.scale.linear().domain([minY, maxY]).range([height, 0]);
+
+ var xAxis = d3.svg.axis().scale(x).orient("top").ticks(5);
+ var yAxis = d3.svg.axis().scale(y).orient("left").ticks(0).tickFormat(function(d) { return ""; });
+
+ var data = d3.layout.histogram().range([minY, maxY]).bins(histogramBinCount)(values);
+
+ var svg = d3.select(id).append("svg")
+ .attr("width", width + margin.left + margin.right)
+ .attr("height", height + margin.top + margin.bottom)
+ .append("g")
+ .attr("transform", "translate(" + margin.left + "," + margin.top + ")");
+
+ if (batchInterval && batchInterval <= maxY) {
+ drawLine(svg, x, y, 0, batchInterval, maxXForHistogram, batchInterval);
+ }
+
+ svg.append("g")
+ .attr("class", "x axis")
+ .call(xAxis)
+
+ svg.append("g")
+ .attr("class", "y axis")
+ .call(yAxis)
+
+ var bar = svg.selectAll(".bar")
+ .data(data)
+ .enter()
+ .append("g")
+ .attr("transform", function(d) { return "translate(0," + (y(d.x) - height + y(d.dx)) + ")";})
+ .attr("class", "bar").append("rect")
+ .attr("width", function(d) { return x(d.y); })
+ .attr("height", function(d) { return height - y(d.dx); })
+ .on('mouseover', function(d) {
+ var percent = yValueFormat(d.y * 100.0 / values.length) + "%";
+ var tip = d.y + " batches (" + percent + ") between " + yValueFormat(d.x) + " and " + yValueFormat(d.x + d.dx) + " " + unitY;
+ showBootstrapTooltip(d3.select(this).node(), tip);
+ })
+ .on('mouseout', function() {
+ hideBootstrapTooltip(d3.select(this).node());
+ });
+
+ if (batchInterval && batchInterval <= maxY) {
+ // Add the "stable" text to the graph below the batch interval line.
+ var stableXOffset = x(maxXForHistogram) - 20;
+ var stableYOffset = y(batchInterval) + 15;
+ svg.append("text")
+ .style("fill", "lightblue")
+ .attr("class", "stable-text")
+ .attr("text-anchor", "middle")
+ .attr("transform", "translate(" + stableXOffset + "," + stableYOffset + ")")
+ .text("stable")
+ .on('mouseover', function(d) {
+ var tip = "Processing Time <= Batch Interval (" + yValueFormat(batchInterval) +" " + unitY +")";
+ showBootstrapTooltip(d3.select(this).node(), tip);
+ })
+ .on('mouseout', function() {
+ hideBootstrapTooltip(d3.select(this).node());
+ });
+ }
+}
+
+$(function() {
+ function getParameterFromURL(param)
+ {
+ var parameters = window.location.search.substring(1); // Remove "?"
+ var keyValues = parameters.split('&');
+ for (var i = 0; i < keyValues.length; i++)
+ {
+ var paramKeyValue = keyValues[i].split('=');
+ if (paramKeyValue[0] == param)
+ {
+ return paramKeyValue[1];
+ }
+ }
+ }
+
+ if (getParameterFromURL("show-streams-detail") == "true") {
+ // Show the details for all InputDStream
+ $('#inputs-table').toggle('collapsed');
+ $('#triangle').html('▼');
+ }
+});
diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css
index 4910744d1d790..669ad48937c05 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/webui.css
+++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css
@@ -145,7 +145,7 @@ pre {
border: none;
}
-span.expand-additional-metrics {
+span.expand-additional-metrics, span.expand-dag-viz {
cursor: pointer;
}
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index 228d9149df2a2..66bda68088502 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -21,7 +21,10 @@ import java.util.concurrent.TimeUnit
import scala.collection.mutable
+import com.codahale.metrics.{Gauge, MetricRegistry}
+
import org.apache.spark.scheduler._
+import org.apache.spark.metrics.source.Source
import org.apache.spark.util.{ThreadUtils, Clock, SystemClock, Utils}
/**
@@ -144,6 +147,9 @@ private[spark] class ExecutorAllocationManager(
private val executor =
ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-dynamic-executor-allocation")
+ // Metric source for ExecutorAllocationManager to expose internal status to MetricsSystem.
+ val executorAllocationManagerSource = new ExecutorAllocationManagerSource
+
/**
* Verify that the settings specified through the config are valid.
* If not, throw an appropriate exception.
@@ -579,6 +585,29 @@ private[spark] class ExecutorAllocationManager(
}
}
+ /**
+ * Metric source for ExecutorAllocationManager to expose its internal executor allocation
+ * status to MetricsSystem.
+ * Note: These metrics heavily rely on the internal implementation of
+ * ExecutorAllocationManager, metrics or value of metrics will be changed when internal
+ * implementation is changed, so these metrics are not stable across Spark version.
+ */
+ private[spark] class ExecutorAllocationManagerSource extends Source {
+ val sourceName = "ExecutorAllocationManager"
+ val metricRegistry = new MetricRegistry()
+
+ private def registerGauge[T](name: String, value: => T, defaultValue: T): Unit = {
+ metricRegistry.register(MetricRegistry.name("executors", name), new Gauge[T] {
+ override def getValue: T = synchronized { Option(value).getOrElse(defaultValue) }
+ })
+ }
+
+ registerGauge("numberExecutorsToAdd", numExecutorsToAdd, 0)
+ registerGauge("numberExecutorsPendingToRemove", executorsPendingToRemove.size, 0)
+ registerGauge("numberAllExecutors", executorIds.size, 0)
+ registerGauge("numberTargetExecutors", numExecutorsTarget, 0)
+ registerGauge("numberMaxNeededExecutors", maxNumExecutorsNeeded(), 0)
+ }
}
private object ExecutorAllocationManager {
diff --git a/core/src/main/scala/org/apache/spark/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/SizeEstimator.scala
new file mode 100644
index 0000000000000..54fc3a856adfa
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/SizeEstimator.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import org.apache.spark.annotation.DeveloperApi
+
+/**
+ * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in
+ * memory-aware caches.
+ *
+ * Based on the following JavaWorld article:
+ * http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html
+ */
+@DeveloperApi
+object SizeEstimator {
+ /**
+ * :: DeveloperApi ::
+ * Estimate the number of bytes that the given object takes up on the JVM heap. The estimate
+ * includes space taken up by objects referenced by the given object, their references, and so on
+ * and so forth.
+ *
+ * This is useful for determining the amount of heap space a broadcast variable will occupy on
+ * each executor or the amount of space each object will take when caching objects in
+ * deserialized form. This is not the same as the serialized size of the object, which will
+ * typically be much smaller.
+ */
+ @DeveloperApi
+ def estimate(obj: AnyRef): Long = org.apache.spark.util.SizeEstimator.estimate(obj)
+}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 4ef90546a2452..2ca6882c8d890 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -428,7 +428,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
_ui =
if (conf.getBoolean("spark.ui.enabled", true)) {
Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener,
- _env.securityManager,appName))
+ _env.securityManager,appName, startTime = startTime))
} else {
// For tests, do not enable the UI
None
@@ -537,6 +537,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
_taskScheduler.postStartHook()
_env.metricsSystem.registerSource(new DAGSchedulerSource(dagScheduler))
_env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager))
+ _executorAllocationManager.foreach { e =>
+ _env.metricsSystem.registerSource(e.executorAllocationManagerSource)
+ }
// Make sure the context is stopped if the user forgets about it. This avoids leaving
// unfinished event logs around after the JVM exits cleanly. It doesn't help if the JVM
@@ -659,6 +662,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null)
}
+ /**
+ * Execute a block of code in a scope such that all new RDDs created in this body will
+ * be part of the same scope. For more detail, see {{org.apache.spark.rdd.RDDOperationScope}}.
+ *
+ * Note: Return statements are NOT allowed in the given body.
+ */
+ private def withScope[U](body: => U): U = RDDOperationScope.withScope[U](this)(body)
+
// Methods for creating RDDs
/** Distribute a local Scala collection to form an RDD.
@@ -669,7 +680,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* @note avoid using `parallelize(Seq())` to create an empty `RDD`. Consider `emptyRDD` for an
* RDD with no partitions, or `parallelize(Seq[T]())` for an RDD of `T` with empty partitions.
*/
- def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
+ def parallelize[T: ClassTag](
+ seq: Seq[T],
+ numSlices: Int = defaultParallelism): RDD[T] = withScope {
assertNotStopped()
new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())
}
@@ -678,14 +691,16 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
*
* This method is identical to `parallelize`.
*/
- def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
+ def makeRDD[T: ClassTag](
+ seq: Seq[T],
+ numSlices: Int = defaultParallelism): RDD[T] = withScope {
parallelize(seq, numSlices)
}
/** Distribute a local Scala collection to form an RDD, with one or more
* location preferences (hostnames of Spark nodes) for each object.
* Create a new partition for each collection item. */
- def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = {
+ def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope {
assertNotStopped()
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
@@ -695,10 +710,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* Read a text file from HDFS, a local file system (available on all nodes), or any
* Hadoop-supported file system URI, and return it as an RDD of Strings.
*/
- def textFile(path: String, minPartitions: Int = defaultMinPartitions): RDD[String] = {
+ def textFile(
+ path: String,
+ minPartitions: Int = defaultMinPartitions): RDD[String] = withScope {
assertNotStopped()
hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text],
- minPartitions).map(pair => pair._2.toString).setName(path)
+ minPartitions).map(pair => pair._2.toString)
}
/**
@@ -728,8 +745,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
*
* @param minPartitions A suggestion value of the minimal splitting number for input data.
*/
- def wholeTextFiles(path: String, minPartitions: Int = defaultMinPartitions):
- RDD[(String, String)] = {
+ def wholeTextFiles(
+ path: String,
+ minPartitions: Int = defaultMinPartitions): RDD[(String, String)] = withScope {
assertNotStopped()
val job = new NewHadoopJob(hadoopConfiguration)
// Use setInputPaths so that wholeTextFiles aligns with hadoopFile/textFile in taking
@@ -776,8 +794,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* @note Small files are preferred; very large files may cause bad performance.
*/
@Experimental
- def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions):
- RDD[(String, PortableDataStream)] = {
+ def binaryFiles(
+ path: String,
+ minPartitions: Int = defaultMinPartitions): RDD[(String, PortableDataStream)] = withScope {
assertNotStopped()
val job = new NewHadoopJob(hadoopConfiguration)
// Use setInputPaths so that binaryFiles aligns with hadoopFile/textFile in taking
@@ -806,8 +825,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* @return An RDD of data with values, represented as byte arrays
*/
@Experimental
- def binaryRecords(path: String, recordLength: Int, conf: Configuration = hadoopConfiguration)
- : RDD[Array[Byte]] = {
+ def binaryRecords(
+ path: String,
+ recordLength: Int,
+ conf: Configuration = hadoopConfiguration): RDD[Array[Byte]] = withScope {
assertNotStopped()
conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength)
val br = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path,
@@ -848,8 +869,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
- minPartitions: Int = defaultMinPartitions
- ): RDD[(K, V)] = {
+ minPartitions: Int = defaultMinPartitions): RDD[(K, V)] = withScope {
assertNotStopped()
// Add necessary security credentials to the JobConf before broadcasting it.
SparkHadoopUtil.get.addCredentials(conf)
@@ -869,8 +889,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
- minPartitions: Int = defaultMinPartitions
- ): RDD[(K, V)] = {
+ minPartitions: Int = defaultMinPartitions): RDD[(K, V)] = withScope {
assertNotStopped()
// A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it.
val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration))
@@ -901,7 +920,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
*/
def hadoopFile[K, V, F <: InputFormat[K, V]]
(path: String, minPartitions: Int)
- (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = {
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = withScope {
hadoopFile(path,
fm.runtimeClass.asInstanceOf[Class[F]],
km.runtimeClass.asInstanceOf[Class[K]],
@@ -924,13 +943,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* copy them using a `map` function.
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String)
- (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] =
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = withScope {
hadoopFile[K, V, F](path, defaultMinPartitions)
+ }
/** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */
def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]
(path: String)
- (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = {
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = withScope {
newAPIHadoopFile(
path,
fm.runtimeClass.asInstanceOf[Class[F]],
@@ -953,7 +973,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
fClass: Class[F],
kClass: Class[K],
vClass: Class[V],
- conf: Configuration = hadoopConfiguration): RDD[(K, V)] = {
+ conf: Configuration = hadoopConfiguration): RDD[(K, V)] = withScope {
assertNotStopped()
// The call to new NewHadoopJob automatically adds security credentials to conf,
// so we don't need to explicitly add them ourselves
@@ -987,7 +1007,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
conf: Configuration = hadoopConfiguration,
fClass: Class[F],
kClass: Class[K],
- vClass: Class[V]): RDD[(K, V)] = {
+ vClass: Class[V]): RDD[(K, V)] = withScope {
assertNotStopped()
// Add necessary security credentials to the JobConf. Required to access secure HDFS.
val jconf = new JobConf(conf)
@@ -1007,7 +1027,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
keyClass: Class[K],
valueClass: Class[V],
minPartitions: Int
- ): RDD[(K, V)] = {
+ ): RDD[(K, V)] = withScope {
assertNotStopped()
val inputFormatClass = classOf[SequenceFileInputFormat[K, V]]
hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions)
@@ -1021,7 +1041,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
* If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first
* copy them using a `map` function.
* */
- def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] = {
+ def sequenceFile[K, V](
+ path: String,
+ keyClass: Class[K],
+ valueClass: Class[V]): RDD[(K, V)] = withScope {
assertNotStopped()
sequenceFile(path, keyClass, valueClass, defaultMinPartitions)
}
@@ -1051,16 +1074,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
def sequenceFile[K, V]
(path: String, minPartitions: Int = defaultMinPartitions)
(implicit km: ClassTag[K], vm: ClassTag[V],
- kcf: () => WritableConverter[K], vcf: () => WritableConverter[V])
- : RDD[(K, V)] = {
- assertNotStopped()
- val kc = kcf()
- val vc = vcf()
- val format = classOf[SequenceFileInputFormat[Writable, Writable]]
- val writables = hadoopFile(path, format,
+ kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]): RDD[(K, V)] = {
+ withScope {
+ assertNotStopped()
+ val kc = kcf()
+ val vc = vcf()
+ val format = classOf[SequenceFileInputFormat[Writable, Writable]]
+ val writables = hadoopFile(path, format,
kc.writableClass(km).asInstanceOf[Class[Writable]],
vc.writableClass(vm).asInstanceOf[Class[Writable]], minPartitions)
- writables.map { case (k, v) => (kc.convert(k), vc.convert(v)) }
+ writables.map { case (k, v) => (kc.convert(k), vc.convert(v)) }
+ }
}
/**
@@ -1073,21 +1097,18 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
*/
def objectFile[T: ClassTag](
path: String,
- minPartitions: Int = defaultMinPartitions
- ): RDD[T] = {
+ minPartitions: Int = defaultMinPartitions): RDD[T] = withScope {
assertNotStopped()
sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minPartitions)
.flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes, Utils.getContextOrSparkClassLoader))
}
- protected[spark] def checkpointFile[T: ClassTag](
- path: String
- ): RDD[T] = {
+ protected[spark] def checkpointFile[T: ClassTag](path: String): RDD[T] = withScope {
new CheckpointRDD[T](this, path)
}
/** Build the union of a list of RDDs. */
- def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = {
+ def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = withScope {
val partitioners = rdds.flatMap(_.partitioner).toSet
if (rdds.forall(_.partitioner.isDefined) && partitioners.size == 1) {
new PartitionerAwareUnionRDD(this, rdds)
@@ -1097,8 +1118,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
}
/** Build the union of a list of RDDs passed as variable-length arguments. */
- def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] =
+ def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = withScope {
union(Seq(first) ++ rest)
+ }
/** Get an RDD that has no partitions or elements. */
def emptyRDD[T: ClassTag]: EmptyRDD[T] = new EmptyRDD[T](this)
@@ -1657,7 +1679,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
partitions: Seq[Int],
allowLocal: Boolean
): Array[U] = {
- runJob(rdd, (context: TaskContext, iter: Iterator[T]) => func(iter), partitions, allowLocal)
+ val cleanedFunc = clean(func)
+ runJob(rdd, (ctx: TaskContext, it: Iterator[T]) => cleanedFunc(it), partitions, allowLocal)
}
/**
@@ -1711,7 +1734,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
val callSite = getCallSite
logInfo("Starting job: " + callSite.shortForm)
val start = System.nanoTime
- val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout,
+ val cleanedFunc = clean(func)
+ val result = dagScheduler.runApproximateJob(rdd, cleanedFunc, evaluator, callSite, timeout,
localProperties.get)
logInfo(
"Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s")
@@ -2060,10 +2084,10 @@ object SparkContext extends Logging {
}
private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description"
-
private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id"
-
private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel"
+ private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope"
+ private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride"
/**
* Executor id for the driver. In earlier versions of Spark, this was ``, but this was
diff --git a/core/src/main/scala/org/apache/spark/annotation/Private.java b/core/src/main/scala/org/apache/spark/annotation/Private.java
new file mode 100644
index 0000000000000..9082fcf0c84bc
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/annotation/Private.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.annotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * A class that is considered private to the internals of Spark -- there is a high-likelihood
+ * they will be changed in future versions of Spark.
+ *
+ * This should be used only when the standard Scala / Java means of protecting classes are
+ * insufficient. In particular, Java has no equivalent of private[spark], so we use this annotation
+ * in its place.
+ *
+ * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first
+ * line of the comment must be ":: Private ::" with no trailing blank line. This is because
+ * of the known issue that Scaladoc displays only either the annotation or the comment, whichever
+ * comes first.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER,
+ ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE})
+public @interface Private {}
diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
index 257491e90dd66..1f1debcf84ad4 100644
--- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
@@ -56,16 +56,13 @@ private[spark] object SerDeUtil extends Logging {
// {'\0', 0, 0, 0} /* Sentinel */
// };
// TODO: support Py_UNICODE with 2 bytes
- // FIXME: unpickle array of float is wrong in Pyrolite, so we reverse the
- // machine code for float/double here to workaround it.
- // we should fix this after Pyrolite fix them
val machineCodes: Map[Char, Int] = if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) {
Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 3, 'h' -> 5, 'I' -> 7, 'i' -> 9,
- 'L' -> 11, 'l' -> 13, 'f' -> 14, 'd' -> 16, 'u' -> 21
+ 'L' -> 11, 'l' -> 13, 'f' -> 15, 'd' -> 17, 'u' -> 21
)
} else {
Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 2, 'h' -> 4, 'I' -> 6, 'i' -> 8,
- 'L' -> 10, 'l' -> 12, 'f' -> 15, 'd' -> 17, 'u' -> 20
+ 'L' -> 10, 'l' -> 12, 'f' -> 14, 'd' -> 16, 'u' -> 20
)
}
override def construct(args: Array[Object]): Object = {
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 42b5d41b7b526..8a0327984e195 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -701,7 +701,7 @@ object SparkSubmit {
}
/** Provides utility functions to be used inside SparkSubmit. */
-private[deploy] object SparkSubmitUtils {
+private[spark] object SparkSubmitUtils {
// Exposed for testing
var printStream = SparkSubmit.printStream
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
index 6a5011af17458..298a8201960d1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
@@ -19,7 +19,7 @@ package org.apache.spark.deploy.history
import org.apache.spark.ui.SparkUI
-private[history] case class ApplicationAttemptInfo(
+private[spark] case class ApplicationAttemptInfo(
attemptId: Option[String],
startTime: Long,
endTime: Long,
@@ -27,7 +27,7 @@ private[history] case class ApplicationAttemptInfo(
sparkUser: String,
completed: Boolean = false)
-private[history] case class ApplicationHistoryInfo(
+private[spark] case class ApplicationHistoryInfo(
id: String,
name: String,
attempts: List[ApplicationAttemptInfo])
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 993763f3aa092..45c2be34c8680 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -17,23 +17,21 @@
package org.apache.spark.deploy.history
-import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream}
+import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream}
import java.util.concurrent.{ExecutorService, Executors, TimeUnit}
import scala.collection.mutable
-import scala.concurrent.duration.Duration
-import com.google.common.util.concurrent.ThreadFactoryBuilder
-
-import com.google.common.util.concurrent.MoreExecutors
-import org.apache.hadoop.fs.permission.AccessControlException
+import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.fs.permission.AccessControlException
+
+import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.io.CompressionCodec
import org.apache.spark.scheduler._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
-import org.apache.spark.{Logging, SecurityManager, SparkConf}
/**
* A class that provides application history from event logs stored in the file system.
@@ -151,7 +149,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
val conf = this.conf.clone()
val appSecManager = new SecurityManager(conf)
SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId,
- HistoryServer.getAttemptURI(appId, attempt.attemptId))
+ HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime)
// Do not call ui.bind() to avoid creating a new server for each application
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 754c8e9b6668b..50522e69dc519 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -25,6 +25,7 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot}
import org.apache.spark.ui.{SparkUI, UIUtils, WebUI}
import org.apache.spark.ui.JettyUtils._
import org.apache.spark.util.{SignalLogger, Utils}
@@ -45,7 +46,7 @@ class HistoryServer(
provider: ApplicationHistoryProvider,
securityManager: SecurityManager,
port: Int)
- extends WebUI(securityManager, port, conf) with Logging {
+ extends WebUI(securityManager, port, conf) with Logging with UIRoot {
// How many applications to retain
private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
@@ -56,7 +57,7 @@ class HistoryServer(
require(parts.length == 1 || parts.length == 2, s"Invalid app key $key")
val ui = provider
.getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None)
- .getOrElse(throw new NoSuchElementException())
+ .getOrElse(throw new NoSuchElementException(s"no app with key $key"))
attachSparkUI(ui)
ui
}
@@ -113,6 +114,10 @@ class HistoryServer(
}
}
+ def getSparkUI(appKey: String): Option[SparkUI] = {
+ Option(appCache.get(appKey))
+ }
+
initialize()
/**
@@ -123,6 +128,9 @@ class HistoryServer(
*/
def initialize() {
attachPage(new HistoryPage(this))
+
+ attachHandler(JsonRootResource.getJsonServlet(this))
+
attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
val contextHandler = new ServletContextHandler
@@ -160,7 +168,13 @@ class HistoryServer(
*
* @return List of all known applications.
*/
- def getApplicationList(): Iterable[ApplicationHistoryInfo] = provider.getListing()
+ def getApplicationList(): Iterable[ApplicationHistoryInfo] = {
+ provider.getListing()
+ }
+
+ def getApplicationInfoList: Iterator[ApplicationInfo] = {
+ getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo)
+ }
/**
* Returns the provider configuration to show in the listing page.
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index 85e1e83c6f5b7..aa54ed9360f36 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -27,7 +27,7 @@ import org.apache.spark.deploy.ApplicationDescription
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.util.Utils
-private[deploy] class ApplicationInfo(
+private[spark] class ApplicationInfo(
val startTime: Long,
val id: String,
val desc: ApplicationDescription,
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index d68bc08fd4c08..7938b77a69ed5 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -769,9 +769,9 @@ private[master] class Master(
/**
* Rebuild a new SparkUI from the given application's event logs.
- * Return whether this is successful.
+ * Return the UI if successful, else None
*/
- private def rebuildSparkUI(app: ApplicationInfo): Boolean = {
+ private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = {
val appName = app.desc.name
val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found"
try {
@@ -779,7 +779,7 @@ private[master] class Master(
.getOrElse {
// Event logging is not enabled for this application
app.desc.appUiUrl = notFoundBasePath
- return false
+ return None
}
val eventLogFilePrefix = EventLoggingListener.getLogPath(
@@ -802,7 +802,7 @@ private[master] class Master(
val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs)
val replayBus = new ReplayListenerBus()
val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf),
- appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
+ appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime)
val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS)
try {
replayBus.replay(logInput, eventLogFile, maybeTruncated)
@@ -813,7 +813,7 @@ private[master] class Master(
webUi.attachSparkUI(ui)
// Application UI is successfully rebuilt, so link the Master UI to it
app.desc.appUiUrl = ui.basePath
- true
+ Some(ui)
} catch {
case fnf: FileNotFoundException =>
// Event logging is enabled for this application, but no event logs are found
@@ -823,7 +823,7 @@ private[master] class Master(
msg += " Did you specify the correct logging directory?"
msg = URLEncoder.encode(msg, "UTF-8")
app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
- false
+ None
case e: Exception =>
// Relay exception message to application UI page
val title = s"Application history load error (${app.id})"
@@ -832,7 +832,7 @@ private[master] class Master(
logError(msg, e)
msg = URLEncoder.encode(msg, "UTF-8")
app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title"
- false
+ None
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 101bc66c35d0c..e28e7e379ac91 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -21,10 +21,7 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import org.json4s.JValue
-import org.json4s.JsonAST.JNothing
-
-import org.apache.spark.deploy.{ExecutorState, JsonProtocol}
+import org.apache.spark.deploy.ExecutorState
import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import org.apache.spark.deploy.master.ExecutorDesc
import org.apache.spark.ui.{UIUtils, WebUIPage}
@@ -34,20 +31,6 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
private val master = parent.masterEndpointRef
- /** Executor details for a particular application */
- override def renderJson(request: HttpServletRequest): JValue = {
- val appId = request.getParameter("appId")
- val state = master.askWithRetry[MasterStateResponse](RequestMasterState)
- val app = state.activeApps.find(_.id == appId).getOrElse({
- state.completedApps.find(_.id == appId).getOrElse(null)
- })
- if (app == null) {
- JNothing
- } else {
- JsonProtocol.writeApplicationInfo(app)
- }
- }
-
/** Executor details for a particular application */
def render(request: HttpServletRequest): Seq[Node] = {
val appId = request.getParameter("appId")
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index 47ac996f5ef7c..1da69b764de01 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -32,9 +32,12 @@ import org.apache.spark.util.Utils
private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
private val master = parent.masterEndpointRef
+ def getMasterState: MasterStateResponse = {
+ master.askWithRetry[MasterStateResponse](RequestMasterState)
+ }
+
override def renderJson(request: HttpServletRequest): JValue = {
- val state = master.askWithRetry[MasterStateResponse](RequestMasterState)
- JsonProtocol.writeMasterState(state)
+ JsonProtocol.writeMasterState(getMasterState)
}
def handleAppKillRequest(request: HttpServletRequest): Unit = {
@@ -66,7 +69,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
/** Index view listing applications and executors */
def render(request: HttpServletRequest): Seq[Node] = {
- val state = master.askWithRetry[MasterStateResponse](RequestMasterState)
+ val state = getMasterState
val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory")
val workers = state.workers.sortBy(_.id)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index d78eb1763ede4..bdc05fbceb4df 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -19,6 +19,7 @@ package org.apache.spark.deploy.master.ui
import org.apache.spark.Logging
import org.apache.spark.deploy.master.Master
+import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot}
import org.apache.spark.ui.{SparkUI, WebUI}
import org.apache.spark.ui.JettyUtils._
@@ -27,11 +28,14 @@ import org.apache.spark.ui.JettyUtils._
*/
private[master]
class MasterWebUI(val master: Master, requestedPort: Int)
- extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging {
+ extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging
+ with UIRoot {
val masterEndpointRef = master.self
val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true)
+ val masterPage = new MasterPage(this)
+
initialize()
/** Initialize all components of the server. */
@@ -41,6 +45,7 @@ class MasterWebUI(val master: Master, requestedPort: Int)
attachPage(new HistoryNotFoundPage(this))
attachPage(masterPage)
attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
+ attachHandler(JsonRootResource.getJsonServlet(this))
attachHandler(createRedirectHandler(
"/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST")))
attachHandler(createRedirectHandler(
@@ -58,6 +63,23 @@ class MasterWebUI(val master: Master, requestedPort: Int)
assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs")
ui.getHandlers.foreach(detachHandler)
}
+
+ def getApplicationInfoList: Iterator[ApplicationInfo] = {
+ val state = masterPage.getMasterState
+ val activeApps = state.activeApps.sortBy(_.startTime).reverse
+ val completedApps = state.completedApps.sortBy(_.endTime).reverse
+ activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++
+ completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) }
+ }
+
+ def getSparkUI(appId: String): Option[SparkUI] = {
+ val state = masterPage.getMasterState
+ val activeApps = state.activeApps.sortBy(_.startTime).reverse
+ val completedApps = state.completedApps.sortBy(_.endTime).reverse
+ (activeApps ++ completedApps).find { _.id == appId }.flatMap {
+ master.rebuildSparkUI
+ }
+ }
}
private[master] object MasterWebUI {
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index 3406a7e97e368..ec185340c3a2d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -33,7 +33,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
/**
* Returns a future for counting the number of elements in the RDD.
*/
- def countAsync(): FutureAction[Long] = {
+ def countAsync(): FutureAction[Long] = self.withScope {
val totalCount = new AtomicLong
self.context.submitJob(
self,
@@ -53,7 +53,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
/**
* Returns a future for retrieving all elements of this RDD.
*/
- def collectAsync(): FutureAction[Seq[T]] = {
+ def collectAsync(): FutureAction[Seq[T]] = self.withScope {
val results = new Array[Array[T]](self.partitions.length)
self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.length),
(index, data) => results(index) = data, results.flatten.toSeq)
@@ -62,7 +62,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
/**
* Returns a future for retrieving the first num elements of the RDD.
*/
- def takeAsync(num: Int): FutureAction[Seq[T]] = {
+ def takeAsync(num: Int): FutureAction[Seq[T]] = self.withScope {
val f = new ComplexFutureAction[Seq[T]]
f.run {
@@ -109,7 +109,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
/**
* Applies a function f to all elements of this RDD.
*/
- def foreachAsync(f: T => Unit): FutureAction[Unit] = {
+ def foreachAsync(f: T => Unit): FutureAction[Unit] = self.withScope {
val cleanF = self.context.clean(f)
self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.length),
(index, data) => Unit, Unit)
@@ -118,7 +118,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
/**
* Applies a function f to each partition of this RDD.
*/
- def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = {
+ def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = self.withScope {
self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.length),
(index, data) => Unit, Unit)
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
index 71578d1210fde..922030263756b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -31,7 +31,7 @@ private[spark]
class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds: Array[BlockId])
extends RDD[T](sc, Nil) {
- @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
+ @transient lazy val _locations = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
@volatile private var _isValid = true
override def getPartitions: Array[Partition] = {
@@ -54,7 +54,7 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds
override def getPreferredLocations(split: Partition): Seq[String] = {
assertValid()
- locations_(split.asInstanceOf[BlockRDDPartition].blockId)
+ _locations(split.asInstanceOf[BlockRDDPartition].blockId)
}
/**
@@ -79,14 +79,14 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds
/** Check if this BlockRDD is valid. If not valid, exception is thrown. */
private[spark] def assertValid() {
- if (!_isValid) {
+ if (!isValid) {
throw new SparkException(
"Attempted to use %s after its blocks have been removed!".format(toString))
}
}
protected def getBlockIdLocations(): Map[BlockId, Seq[String]] = {
- locations_
+ _locations
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index 843a893235e56..926bce6f15a2a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -30,7 +30,7 @@ import org.apache.spark.util.StatCounter
*/
class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
/** Add up the elements in this RDD. */
- def sum(): Double = {
+ def sum(): Double = self.withScope {
self.fold(0.0)(_ + _)
}
@@ -38,37 +38,49 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
* Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and
* count of the RDD's elements in one operation.
*/
- def stats(): StatCounter = {
+ def stats(): StatCounter = self.withScope {
self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b))
}
/** Compute the mean of this RDD's elements. */
- def mean(): Double = stats().mean
+ def mean(): Double = self.withScope {
+ stats().mean
+ }
/** Compute the variance of this RDD's elements. */
- def variance(): Double = stats().variance
+ def variance(): Double = self.withScope {
+ stats().variance
+ }
/** Compute the standard deviation of this RDD's elements. */
- def stdev(): Double = stats().stdev
+ def stdev(): Double = self.withScope {
+ stats().stdev
+ }
/**
* Compute the sample standard deviation of this RDD's elements (which corrects for bias in
* estimating the standard deviation by dividing by N-1 instead of N).
*/
- def sampleStdev(): Double = stats().sampleStdev
+ def sampleStdev(): Double = self.withScope {
+ stats().sampleStdev
+ }
/**
* Compute the sample variance of this RDD's elements (which corrects for bias in
* estimating the variance by dividing by N-1 instead of N).
*/
- def sampleVariance(): Double = stats().sampleVariance
+ def sampleVariance(): Double = self.withScope {
+ stats().sampleVariance
+ }
/**
* :: Experimental ::
* Approximate operation to return the mean within a timeout.
*/
@Experimental
- def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
+ def meanApprox(
+ timeout: Long,
+ confidence: Double = 0.95): PartialResult[BoundedDouble] = self.withScope {
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
val evaluator = new MeanEvaluator(self.partitions.length, confidence)
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
@@ -79,7 +91,9 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
* Approximate operation to return the sum within a timeout.
*/
@Experimental
- def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
+ def sumApprox(
+ timeout: Long,
+ confidence: Double = 0.95): PartialResult[BoundedDouble] = self.withScope {
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
val evaluator = new SumEvaluator(self.partitions.length, confidence)
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
@@ -93,7 +107,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
* If the RDD contains infinity, NaN throws an exception
* If the elements in RDD do not vary (max == min) always returns a single bucket.
*/
- def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = {
+ def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = self.withScope {
// Scala's built-in range has issues. See #SI-8782
def customRange(min: Double, max: Double, steps: Int): IndexedSeq[Double] = {
val span = max - min
@@ -140,7 +154,9 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
* the maximum value of the last position and all NaN entries will be counted
* in that bucket.
*/
- def histogram(buckets: Array[Double], evenBuckets: Boolean = false): Array[Long] = {
+ def histogram(
+ buckets: Array[Double],
+ evenBuckets: Boolean = false): Array[Long] = self.withScope {
if (buckets.length < 2) {
throw new IllegalArgumentException("buckets array must have at least two elements")
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index f77abac42b623..2cefe63d44b20 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -99,7 +99,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp
*/
@DeveloperApi
class HadoopRDD[K, V](
- sc: SparkContext,
+ @transient sc: SparkContext,
broadcastedConf: Broadcast[SerializableWritable[Configuration]],
initLocalJobConfFuncOpt: Option[JobConf => Unit],
inputFormatClass: Class[_ <: InputFormat[K, V]],
@@ -108,6 +108,10 @@ class HadoopRDD[K, V](
minPartitions: Int)
extends RDD[(K, V)](sc, Nil) with Logging {
+ if (initLocalJobConfFuncOpt.isDefined) {
+ sc.clean(initLocalJobConfFuncOpt.get)
+ }
+
def this(
sc: SparkContext,
conf: JobConf,
diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index 6afe50161dacd..d71bb63000904 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -57,7 +57,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
*/
// TODO: this currently doesn't work on P other than Tuple2!
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.length)
- : RDD[(K, V)] =
+ : RDD[(K, V)] = self.withScope
{
val part = new RangePartitioner(numPartitions, self, ascending)
new ShuffledRDD[K, V, V](self, part)
@@ -71,7 +71,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
* This is more efficient than calling `repartition` and then sorting within each partition
* because it can push the sorting down into the shuffle machinery.
*/
- def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = {
+ def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = self.withScope {
new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering)
}
@@ -81,7 +81,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
* performed efficiently by only scanning the partitions that might contain matching elements.
* Otherwise, a standard `filter` is applied to all partitions.
*/
- def filterByRange(lower: K, upper: K): RDD[P] = {
+ def filterByRange(lower: K, upper: K): RDD[P] = self.withScope {
def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper)
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 05351ba4ff76b..a6d5d2c94e17f 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -29,7 +29,7 @@ import scala.util.DynamicVariable
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus
import org.apache.hadoop.conf.{Configurable, Configuration}
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.io.SequenceFile.CompressionType
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat}
@@ -75,7 +75,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
mergeCombiners: (C, C) => C,
partitioner: Partitioner,
mapSideCombine: Boolean = true,
- serializer: Serializer = null): RDD[(K, C)] = {
+ serializer: Serializer = null): RDD[(K, C)] = self.withScope {
require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0
if (keyClass.isArray) {
if (mapSideCombine) {
@@ -108,7 +108,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
def combineByKey[C](createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C,
- numPartitions: Int): RDD[(K, C)] = {
+ numPartitions: Int): RDD[(K, C)] = self.withScope {
combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions))
}
@@ -122,7 +122,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* instead of creating a new U.
*/
def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U,
- combOp: (U, U) => U): RDD[(K, U)] = {
+ combOp: (U, U) => U): RDD[(K, U)] = self.withScope {
// Serialize the zero value to a byte array so that we can get a new clone of it on each key
val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue)
val zeroArray = new Array[Byte](zeroBuffer.limit)
@@ -131,7 +131,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
lazy val cachedSerializer = SparkEnv.get.serializer.newInstance()
val createZero = () => cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray))
- combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner)
+ // We will clean the combiner closure later in `combineByKey`
+ val cleanedSeqOp = self.context.clean(seqOp)
+ combineByKey[U]((v: V) => cleanedSeqOp(createZero(), v), cleanedSeqOp, combOp, partitioner)
}
/**
@@ -144,7 +146,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* instead of creating a new U.
*/
def aggregateByKey[U: ClassTag](zeroValue: U, numPartitions: Int)(seqOp: (U, V) => U,
- combOp: (U, U) => U): RDD[(K, U)] = {
+ combOp: (U, U) => U): RDD[(K, U)] = self.withScope {
aggregateByKey(zeroValue, new HashPartitioner(numPartitions))(seqOp, combOp)
}
@@ -158,7 +160,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* instead of creating a new U.
*/
def aggregateByKey[U: ClassTag](zeroValue: U)(seqOp: (U, V) => U,
- combOp: (U, U) => U): RDD[(K, U)] = {
+ combOp: (U, U) => U): RDD[(K, U)] = self.withScope {
aggregateByKey(zeroValue, defaultPartitioner(self))(seqOp, combOp)
}
@@ -167,7 +169,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* may be added to the result an arbitrary number of times, and must not change the result
* (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).
*/
- def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = {
+ def foldByKey(
+ zeroValue: V,
+ partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = self.withScope {
// Serialize the zero value to a byte array so that we can get a new clone of it on each key
val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue)
val zeroArray = new Array[Byte](zeroBuffer.limit)
@@ -177,7 +181,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
lazy val cachedSerializer = SparkEnv.get.serializer.newInstance()
val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray))
- combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner)
+ val cleanedFunc = self.context.clean(func)
+ combineByKey[V]((v: V) => cleanedFunc(createZero(), v), cleanedFunc, cleanedFunc, partitioner)
}
/**
@@ -185,7 +190,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* may be added to the result an arbitrary number of times, and must not change the result
* (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).
*/
- def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = {
+ def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = self.withScope {
foldByKey(zeroValue, new HashPartitioner(numPartitions))(func)
}
@@ -194,7 +199,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* may be added to the result an arbitrary number of times, and must not change the result
* (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.).
*/
- def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = {
+ def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = self.withScope {
foldByKey(zeroValue, defaultPartitioner(self))(func)
}
@@ -213,7 +218,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
*/
def sampleByKey(withReplacement: Boolean,
fractions: Map[K, Double],
- seed: Long = Utils.random.nextLong): RDD[(K, V)] = {
+ seed: Long = Utils.random.nextLong): RDD[(K, V)] = self.withScope {
require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.")
@@ -242,9 +247,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* @return RDD containing the sampled subset
*/
@Experimental
- def sampleByKeyExact(withReplacement: Boolean,
+ def sampleByKeyExact(
+ withReplacement: Boolean,
fractions: Map[K, Double],
- seed: Long = Utils.random.nextLong): RDD[(K, V)] = {
+ seed: Long = Utils.random.nextLong): RDD[(K, V)] = self.withScope {
require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.")
@@ -261,7 +267,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce.
*/
- def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = {
+ def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = self.withScope {
combineByKey[V]((v: V) => v, func, func, partitioner)
}
@@ -270,7 +276,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
*/
- def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = {
+ def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = self.withScope {
reduceByKey(new HashPartitioner(numPartitions), func)
}
@@ -280,7 +286,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/
* parallelism level.
*/
- def reduceByKey(func: (V, V) => V): RDD[(K, V)] = {
+ def reduceByKey(func: (V, V) => V): RDD[(K, V)] = self.withScope {
reduceByKey(defaultPartitioner(self), func)
}
@@ -289,7 +295,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* immediately to the master as a Map. This will also perform the merging locally on each mapper
* before sending results to a reducer, similarly to a "combiner" in MapReduce.
*/
- def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = {
+ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = self.withScope {
if (keyClass.isArray) {
throw new SparkException("reduceByKeyLocally() does not support array keys")
@@ -317,7 +323,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
/** Alias for reduceByKeyLocally */
@deprecated("Use reduceByKeyLocally", "1.0.0")
- def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func)
+ def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = self.withScope {
+ reduceByKeyLocally(func)
+ }
/**
* Count the number of elements for each key, collecting the results to a local Map.
@@ -327,7 +335,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* To handle very large results, consider using rdd.mapValues(_ => 1L).reduceByKey(_ + _), which
* returns an RDD[T, Long] instead of a map.
*/
- def countByKey(): Map[K, Long] = self.mapValues(_ => 1L).reduceByKey(_ + _).collect().toMap
+ def countByKey(): Map[K, Long] = self.withScope {
+ self.mapValues(_ => 1L).reduceByKey(_ + _).collect().toMap
+ }
/**
* :: Experimental ::
@@ -336,7 +346,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
*/
@Experimental
def countByKeyApprox(timeout: Long, confidence: Double = 0.95)
- : PartialResult[Map[K, BoundedDouble]] = {
+ : PartialResult[Map[K, BoundedDouble]] = self.withScope {
self.map(_._1).countByValueApprox(timeout, confidence)
}
@@ -360,7 +370,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* @param partitioner Partitioner to use for the resulting RDD.
*/
@Experimental
- def countApproxDistinctByKey(p: Int, sp: Int, partitioner: Partitioner): RDD[(K, Long)] = {
+ def countApproxDistinctByKey(
+ p: Int,
+ sp: Int,
+ partitioner: Partitioner): RDD[(K, Long)] = self.withScope {
require(p >= 4, s"p ($p) must be >= 4")
require(sp <= 32, s"sp ($sp) must be <= 32")
require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)")
@@ -392,7 +405,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* It must be greater than 0.000017.
* @param partitioner partitioner of the resulting RDD
*/
- def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = {
+ def countApproxDistinctByKey(
+ relativeSD: Double,
+ partitioner: Partitioner): RDD[(K, Long)] = self.withScope {
require(relativeSD > 0.000017, s"accuracy ($relativeSD) must be greater than 0.000017")
val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt
assert(p <= 32)
@@ -410,7 +425,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* It must be greater than 0.000017.
* @param numPartitions number of partitions of the resulting RDD
*/
- def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = {
+ def countApproxDistinctByKey(
+ relativeSD: Double,
+ numPartitions: Int): RDD[(K, Long)] = self.withScope {
countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions))
}
@@ -424,7 +441,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* @param relativeSD Relative accuracy. Smaller values create counters that require more space.
* It must be greater than 0.000017.
*/
- def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = {
+ def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = self.withScope {
countApproxDistinctByKey(relativeSD, defaultPartitioner(self))
}
@@ -441,7 +458,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any
* key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
*/
- def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = {
+ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = self.withScope {
// groupByKey shouldn't use map side combine because map side combine does not
// reduce the amount of data shuffled and requires all map side data be inserted
// into a hash table, leading to more objects in the old gen.
@@ -465,14 +482,14 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any
* key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
*/
- def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = {
+ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = self.withScope {
groupByKey(new HashPartitioner(numPartitions))
}
/**
* Return a copy of the RDD partitioned using the specified partitioner.
*/
- def partitionBy(partitioner: Partitioner): RDD[(K, V)] = {
+ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = self.withScope {
if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
@@ -488,7 +505,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
* (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
*/
- def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
+ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = self.withScope {
this.cogroup(other, partitioner).flatMapValues( pair =>
for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, w)
)
@@ -500,7 +517,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to
* partition the output RDD.
*/
- def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = {
+ def leftOuterJoin[W](
+ other: RDD[(K, W)],
+ partitioner: Partitioner): RDD[(K, (V, Option[W]))] = self.withScope {
this.cogroup(other, partitioner).flatMapValues { pair =>
if (pair._2.isEmpty) {
pair._1.iterator.map(v => (v, None))
@@ -517,7 +536,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* partition the output RDD.
*/
def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner)
- : RDD[(K, (Option[V], W))] = {
+ : RDD[(K, (Option[V], W))] = self.withScope {
this.cogroup(other, partitioner).flatMapValues { pair =>
if (pair._1.isEmpty) {
pair._2.iterator.map(w => (None, w))
@@ -536,7 +555,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* in `this` have key k. Uses the given Partitioner to partition the output RDD.
*/
def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner)
- : RDD[(K, (Option[V], Option[W]))] = {
+ : RDD[(K, (Option[V], Option[W]))] = self.withScope {
this.cogroup(other, partitioner).flatMapValues {
case (vs, Seq()) => vs.iterator.map(v => (Some(v), None))
case (Seq(), ws) => ws.iterator.map(w => (None, Some(w)))
@@ -549,7 +568,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* existing partitioner/parallelism level.
*/
def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C)
- : RDD[(K, C)] = {
+ : RDD[(K, C)] = self.withScope {
combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self))
}
@@ -563,7 +582,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
*/
- def groupByKey(): RDD[(K, Iterable[V])] = {
+ def groupByKey(): RDD[(K, Iterable[V])] = self.withScope {
groupByKey(defaultPartitioner(self))
}
@@ -572,7 +591,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
* (k, v2) is in `other`. Performs a hash join across the cluster.
*/
- def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = {
+ def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = self.withScope {
join(other, defaultPartitioner(self, other))
}
@@ -581,7 +600,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
* (k, v2) is in `other`. Performs a hash join across the cluster.
*/
- def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = {
+ def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = self.withScope {
join(other, new HashPartitioner(numPartitions))
}
@@ -591,7 +610,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
* using the existing partitioner/parallelism level.
*/
- def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = {
+ def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = self.withScope {
leftOuterJoin(other, defaultPartitioner(self, other))
}
@@ -601,7 +620,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
* into `numPartitions` partitions.
*/
- def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = {
+ def leftOuterJoin[W](
+ other: RDD[(K, W)],
+ numPartitions: Int): RDD[(K, (V, Option[W]))] = self.withScope {
leftOuterJoin(other, new HashPartitioner(numPartitions))
}
@@ -611,7 +632,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
* RDD using the existing partitioner/parallelism level.
*/
- def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = {
+ def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = self.withScope {
rightOuterJoin(other, defaultPartitioner(self, other))
}
@@ -621,7 +642,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
* RDD into the given number of partitions.
*/
- def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = {
+ def rightOuterJoin[W](
+ other: RDD[(K, W)],
+ numPartitions: Int): RDD[(K, (Option[V], W))] = self.withScope {
rightOuterJoin(other, new HashPartitioner(numPartitions))
}
@@ -634,7 +657,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/
* parallelism level.
*/
- def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))] = {
+ def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))] = self.withScope {
fullOuterJoin(other, defaultPartitioner(self, other))
}
@@ -646,7 +669,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements
* in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions.
*/
- def fullOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], Option[W]))] = {
+ def fullOuterJoin[W](
+ other: RDD[(K, W)],
+ numPartitions: Int): RDD[(K, (Option[V], Option[W]))] = self.withScope {
fullOuterJoin(other, new HashPartitioner(numPartitions))
}
@@ -656,7 +681,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Warning: this doesn't return a multimap (so if you have multiple values to the same key, only
* one value per key is preserved in the map returned)
*/
- def collectAsMap(): Map[K, V] = {
+ def collectAsMap(): Map[K, V] = self.withScope {
val data = self.collect()
val map = new mutable.HashMap[K, V]
map.sizeHint(data.length)
@@ -668,7 +693,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Pass each value in the key-value pair RDD through a map function without changing the keys;
* this also retains the original RDD's partitioning.
*/
- def mapValues[U](f: V => U): RDD[(K, U)] = {
+ def mapValues[U](f: V => U): RDD[(K, U)] = self.withScope {
val cleanF = self.context.clean(f)
new MapPartitionsRDD[(K, U), (K, V)](self,
(context, pid, iter) => iter.map { case (k, v) => (k, cleanF(v)) },
@@ -679,7 +704,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Pass each value in the key-value pair RDD through a flatMap function without changing the
* keys; this also retains the original RDD's partitioning.
*/
- def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = {
+ def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = self.withScope {
val cleanF = self.context.clean(f)
new MapPartitionsRDD[(K, U), (K, V)](self,
(context, pid, iter) => iter.flatMap { case (k, v) =>
@@ -697,7 +722,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
other2: RDD[(K, W2)],
other3: RDD[(K, W3)],
partitioner: Partitioner)
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope {
if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
@@ -715,7 +740,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* list of values for that key in `this` as well as `other`.
*/
def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner)
- : RDD[(K, (Iterable[V], Iterable[W]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope {
if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
@@ -730,7 +755,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* tuple with the list of values for that key in `this`, `other1` and `other2`.
*/
def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner)
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope {
if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) {
throw new SparkException("Default partitioner cannot partition array keys.")
}
@@ -748,7 +773,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* for that key in `this`, `other1`, `other2` and `other3`.
*/
def cogroup[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)])
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope {
cogroup(other1, other2, other3, defaultPartitioner(self, other1, other2, other3))
}
@@ -756,7 +781,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
* list of values for that key in `this` as well as `other`.
*/
- def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = {
+ def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope {
cogroup(other, defaultPartitioner(self, other))
}
@@ -765,7 +790,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* tuple with the list of values for that key in `this`, `other1` and `other2`.
*/
def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)])
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope {
cogroup(other1, other2, defaultPartitioner(self, other1, other2))
}
@@ -773,7 +798,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
* list of values for that key in `this` as well as `other`.
*/
- def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = {
+ def cogroup[W](
+ other: RDD[(K, W)],
+ numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope {
cogroup(other, new HashPartitioner(numPartitions))
}
@@ -782,7 +809,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* tuple with the list of values for that key in `this`, `other1` and `other2`.
*/
def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int)
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope {
cogroup(other1, other2, new HashPartitioner(numPartitions))
}
@@ -795,24 +822,24 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
other2: RDD[(K, W2)],
other3: RDD[(K, W3)],
numPartitions: Int)
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope {
cogroup(other1, other2, other3, new HashPartitioner(numPartitions))
}
/** Alias for cogroup. */
- def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = {
+ def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope {
cogroup(other, defaultPartitioner(self, other))
}
/** Alias for cogroup. */
def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)])
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope {
cogroup(other1, other2, defaultPartitioner(self, other1, other2))
}
/** Alias for cogroup. */
def groupWith[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)])
- : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = {
+ : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope {
cogroup(other1, other2, other3, defaultPartitioner(self, other1, other2, other3))
}
@@ -822,22 +849,27 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
- def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = self.withScope {
subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length)))
+ }
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
- def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](
+ other: RDD[(K, W)],
+ numPartitions: Int): RDD[(K, V)] = self.withScope {
subtractByKey(other, new HashPartitioner(numPartitions))
+ }
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
- def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = self.withScope {
new SubtractedRDD[K, V, W](self, other, p)
+ }
/**
* Return the list of values in the RDD for key `key`. This operation is done efficiently if the
* RDD has a known partitioner by only searching the partition that the key maps to.
*/
- def lookup(key: K): Seq[V] = {
+ def lookup(key: K): Seq[V] = self.withScope {
self.partitioner match {
case Some(p) =>
val index = p.getPartition(key)
@@ -859,7 +891,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class
* supporting the key and value types K and V in this RDD.
*/
- def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) {
+ def saveAsHadoopFile[F <: OutputFormat[K, V]](
+ path: String)(implicit fm: ClassTag[F]): Unit = self.withScope {
saveAsHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
@@ -869,7 +902,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* supplied codec.
*/
def saveAsHadoopFile[F <: OutputFormat[K, V]](
- path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) {
+ path: String,
+ codec: Class[_ <: CompressionCodec])(implicit fm: ClassTag[F]): Unit = self.withScope {
val runtimeClass = fm.runtimeClass
saveAsHadoopFile(path, keyClass, valueClass, runtimeClass.asInstanceOf[Class[F]], codec)
}
@@ -878,7 +912,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
* (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
*/
- def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) {
+ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](
+ path: String)(implicit fm: ClassTag[F]): Unit = self.withScope {
saveAsNewAPIHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
@@ -891,8 +926,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
keyClass: Class[_],
valueClass: Class[_],
outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
- conf: Configuration = self.context.hadoopConfiguration)
- {
+ conf: Configuration = self.context.hadoopConfiguration): Unit = self.withScope {
// Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038).
val hadoopConf = conf
val job = new NewAPIHadoopJob(hadoopConf)
@@ -912,7 +946,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
keyClass: Class[_],
valueClass: Class[_],
outputFormatClass: Class[_ <: OutputFormat[_, _]],
- codec: Class[_ <: CompressionCodec]) {
+ codec: Class[_ <: CompressionCodec]): Unit = self.withScope {
saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass,
new JobConf(self.context.hadoopConfiguration), Some(codec))
}
@@ -927,7 +961,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
valueClass: Class[_],
outputFormatClass: Class[_ <: OutputFormat[_, _]],
conf: JobConf = new JobConf(self.context.hadoopConfiguration),
- codec: Option[Class[_ <: CompressionCodec]] = None) {
+ codec: Option[Class[_ <: CompressionCodec]] = None): Unit = self.withScope {
// Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038).
val hadoopConf = conf
hadoopConf.setOutputKeyClass(keyClass)
@@ -960,7 +994,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* output paths required (e.g. a table name to write to) in the same way as it would be
* configured for a Hadoop MapReduce job.
*/
- def saveAsNewAPIHadoopDataset(conf: Configuration) {
+ def saveAsNewAPIHadoopDataset(conf: Configuration): Unit = self.withScope {
// Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038).
val hadoopConf = conf
val job = new NewAPIHadoopJob(hadoopConf)
@@ -1027,7 +1061,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* (e.g. a table name to write to) in the same way as it would be configured for a Hadoop
* MapReduce job.
*/
- def saveAsHadoopDataset(conf: JobConf) {
+ def saveAsHadoopDataset(conf: JobConf): Unit = self.withScope {
// Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038).
val hadoopConf = conf
val wrappedConf = new SerializableWritable(hadoopConf)
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 31c07c73fe07b..b3b60578c92e8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -25,7 +25,7 @@ import scala.language.implicitConversions
import scala.reflect.{classTag, ClassTag}
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus
-import org.apache.hadoop.io.{Writable, BytesWritable, NullWritable, Text}
+import org.apache.hadoop.io.{BytesWritable, NullWritable, Text}
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.mapred.TextOutputFormat
@@ -277,12 +277,20 @@ abstract class RDD[T: ClassTag](
if (isCheckpointed) firstParent[T].iterator(split, context) else compute(split, context)
}
+ /**
+ * Execute a block of code in a scope such that all new RDDs created in this body will
+ * be part of the same scope. For more detail, see {{org.apache.spark.rdd.RDDOperationScope}}.
+ *
+ * Note: Return statements are NOT allowed in the given body.
+ */
+ private[spark] def withScope[U](body: => U): U = RDDOperationScope.withScope[U](sc)(body)
+
// Transformations (return a new RDD)
/**
* Return a new RDD by applying a function to all elements of this RDD.
*/
- def map[U: ClassTag](f: T => U): RDD[U] = {
+ def map[U: ClassTag](f: T => U): RDD[U] = withScope {
val cleanF = sc.clean(f)
new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.map(cleanF))
}
@@ -291,7 +299,7 @@ abstract class RDD[T: ClassTag](
* Return a new RDD by first applying a function to all elements of this
* RDD, and then flattening the results.
*/
- def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = {
+ def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = withScope {
val cleanF = sc.clean(f)
new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.flatMap(cleanF))
}
@@ -299,7 +307,7 @@ abstract class RDD[T: ClassTag](
/**
* Return a new RDD containing only the elements that satisfy a predicate.
*/
- def filter(f: T => Boolean): RDD[T] = {
+ def filter(f: T => Boolean): RDD[T] = withScope {
val cleanF = sc.clean(f)
new MapPartitionsRDD[T, T](
this,
@@ -310,13 +318,16 @@ abstract class RDD[T: ClassTag](
/**
* Return a new RDD containing the distinct elements in this RDD.
*/
- def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] =
+ def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope {
map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1)
+ }
/**
* Return a new RDD containing the distinct elements in this RDD.
*/
- def distinct(): RDD[T] = distinct(partitions.length)
+ def distinct(): RDD[T] = withScope {
+ distinct(partitions.length)
+ }
/**
* Return a new RDD that has exactly numPartitions partitions.
@@ -327,7 +338,7 @@ abstract class RDD[T: ClassTag](
* If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
* which can avoid performing a shuffle.
*/
- def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = {
+ def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope {
coalesce(numPartitions, shuffle = true)
}
@@ -352,7 +363,7 @@ abstract class RDD[T: ClassTag](
* data distributed using a hash partitioner.
*/
def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null)
- : RDD[T] = {
+ : RDD[T] = withScope {
if (shuffle) {
/** Distributes elements evenly across output partitions, starting from a random partition. */
val distributePartition = (index: Int, items: Iterator[T]) => {
@@ -377,16 +388,17 @@ abstract class RDD[T: ClassTag](
/**
* Return a sampled subset of this RDD.
- *
+ *
* @param withReplacement can elements be sampled multiple times (replaced when sampled out)
* @param fraction expected size of the sample as a fraction of this RDD's size
* without replacement: probability that each element is chosen; fraction must be [0, 1]
* with replacement: expected number of times each element is chosen; fraction must be >= 0
* @param seed seed for the random number generator
*/
- def sample(withReplacement: Boolean,
+ def sample(
+ withReplacement: Boolean,
fraction: Double,
- seed: Long = Utils.random.nextLong): RDD[T] = {
+ seed: Long = Utils.random.nextLong): RDD[T] = withScope {
require(fraction >= 0.0, "Negative fraction value: " + fraction)
if (withReplacement) {
new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed)
@@ -403,7 +415,9 @@ abstract class RDD[T: ClassTag](
*
* @return split RDDs in an array
*/
- def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]] = {
+ def randomSplit(
+ weights: Array[Double],
+ seed: Long = Utils.random.nextLong): Array[RDD[T]] = withScope {
val sum = weights.sum
val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _)
normalizedCumWeights.sliding(2).map { x =>
@@ -435,7 +449,9 @@ abstract class RDD[T: ClassTag](
* @param seed seed for the random number generator
* @return sample of specified size in an array
*/
- def takeSample(withReplacement: Boolean,
+ // TODO: rewrite this without return statements so we can wrap it in a scope
+ def takeSample(
+ withReplacement: Boolean,
num: Int,
seed: Long = Utils.random.nextLong): Array[T] = {
val numStDev = 10.0
@@ -483,7 +499,7 @@ abstract class RDD[T: ClassTag](
* Return the union of this RDD and another one. Any identical elements will appear multiple
* times (use `.distinct()` to eliminate them).
*/
- def union(other: RDD[T]): RDD[T] = {
+ def union(other: RDD[T]): RDD[T] = withScope {
if (partitioner.isDefined && other.partitioner == partitioner) {
new PartitionerAwareUnionRDD(sc, Array(this, other))
} else {
@@ -495,7 +511,9 @@ abstract class RDD[T: ClassTag](
* Return the union of this RDD and another one. Any identical elements will appear multiple
* times (use `.distinct()` to eliminate them).
*/
- def ++(other: RDD[T]): RDD[T] = this.union(other)
+ def ++(other: RDD[T]): RDD[T] = withScope {
+ this.union(other)
+ }
/**
* Return this RDD sorted by the given key function.
@@ -504,10 +522,11 @@ abstract class RDD[T: ClassTag](
f: (T) => K,
ascending: Boolean = true,
numPartitions: Int = this.partitions.length)
- (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] =
+ (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] = withScope {
this.keyBy[K](f)
.sortByKey(ascending, numPartitions)
.values
+ }
/**
* Return the intersection of this RDD and another one. The output will not contain any duplicate
@@ -515,7 +534,7 @@ abstract class RDD[T: ClassTag](
*
* Note that this method performs a shuffle internally.
*/
- def intersection(other: RDD[T]): RDD[T] = {
+ def intersection(other: RDD[T]): RDD[T] = withScope {
this.map(v => (v, null)).cogroup(other.map(v => (v, null)))
.filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty }
.keys
@@ -529,8 +548,9 @@ abstract class RDD[T: ClassTag](
*
* @param partitioner Partitioner to use for the resulting RDD
*/
- def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null)
- : RDD[T] = {
+ def intersection(
+ other: RDD[T],
+ partitioner: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = withScope {
this.map(v => (v, null)).cogroup(other.map(v => (v, null)), partitioner)
.filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty }
.keys
@@ -544,16 +564,14 @@ abstract class RDD[T: ClassTag](
*
* @param numPartitions How many partitions to use in the resulting RDD
*/
- def intersection(other: RDD[T], numPartitions: Int): RDD[T] = {
- this.map(v => (v, null)).cogroup(other.map(v => (v, null)), new HashPartitioner(numPartitions))
- .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty }
- .keys
+ def intersection(other: RDD[T], numPartitions: Int): RDD[T] = withScope {
+ intersection(other, new HashPartitioner(numPartitions))
}
/**
* Return an RDD created by coalescing all elements within each partition into an array.
*/
- def glom(): RDD[Array[T]] = {
+ def glom(): RDD[Array[T]] = withScope {
new MapPartitionsRDD[Array[T], T](this, (context, pid, iter) => Iterator(iter.toArray))
}
@@ -561,7 +579,9 @@ abstract class RDD[T: ClassTag](
* Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of
* elements (a, b) where a is in `this` and b is in `other`.
*/
- def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other)
+ def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope {
+ new CartesianRDD(sc, this, other)
+ }
/**
* Return an RDD of grouped items. Each group consists of a key and a sequence of elements
@@ -572,8 +592,9 @@ abstract class RDD[T: ClassTag](
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
*/
- def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] =
+ def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = withScope {
groupBy[K](f, defaultPartitioner(this))
+ }
/**
* Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
@@ -584,8 +605,11 @@ abstract class RDD[T: ClassTag](
* aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
*/
- def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] =
+ def groupBy[K](
+ f: T => K,
+ numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = withScope {
groupBy(f, new HashPartitioner(numPartitions))
+ }
/**
* Return an RDD of grouped items. Each group consists of a key and a sequence of elements
@@ -597,7 +621,7 @@ abstract class RDD[T: ClassTag](
* or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
*/
def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null)
- : RDD[(K, Iterable[T])] = {
+ : RDD[(K, Iterable[T])] = withScope {
val cleanF = sc.clean(f)
this.map(t => (cleanF(t), t)).groupByKey(p)
}
@@ -605,13 +629,16 @@ abstract class RDD[T: ClassTag](
/**
* Return an RDD created by piping elements to a forked external process.
*/
- def pipe(command: String): RDD[String] = new PipedRDD(this, command)
+ def pipe(command: String): RDD[String] = withScope {
+ new PipedRDD(this, command)
+ }
/**
* Return an RDD created by piping elements to a forked external process.
*/
- def pipe(command: String, env: Map[String, String]): RDD[String] =
+ def pipe(command: String, env: Map[String, String]): RDD[String] = withScope {
new PipedRDD(this, command, env)
+ }
/**
* Return an RDD created by piping elements to a forked external process.
@@ -619,7 +646,7 @@ abstract class RDD[T: ClassTag](
*
* @param command command to run in forked process.
* @param env environment variables to set.
- * @param printPipeContext Before piping elements, this function is called as an oppotunity
+ * @param printPipeContext Before piping elements, this function is called as an opportunity
* to pipe context data. Print line function (like out.println) will be
* passed as printPipeContext's parameter.
* @param printRDDElement Use this function to customize how to pipe elements. This function
@@ -637,7 +664,7 @@ abstract class RDD[T: ClassTag](
env: Map[String, String] = Map(),
printPipeContext: (String => Unit) => Unit = null,
printRDDElement: (T, String => Unit) => Unit = null,
- separateWorkingDir: Boolean = false): RDD[String] = {
+ separateWorkingDir: Boolean = false): RDD[String] = withScope {
new PipedRDD(this, command, env,
if (printPipeContext ne null) sc.clean(printPipeContext) else null,
if (printRDDElement ne null) sc.clean(printRDDElement) else null,
@@ -651,9 +678,13 @@ abstract class RDD[T: ClassTag](
* should be `false` unless this is a pair RDD and the input function doesn't modify the keys.
*/
def mapPartitions[U: ClassTag](
- f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
- val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter)
- new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
+ f: Iterator[T] => Iterator[U],
+ preservesPartitioning: Boolean = false): RDD[U] = withScope {
+ val cleanedF = sc.clean(f)
+ new MapPartitionsRDD(
+ this,
+ (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(iter),
+ preservesPartitioning)
}
/**
@@ -664,9 +695,13 @@ abstract class RDD[T: ClassTag](
* should be `false` unless this is a pair RDD and the input function doesn't modify the keys.
*/
def mapPartitionsWithIndex[U: ClassTag](
- f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
- val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter)
- new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
+ f: (Int, Iterator[T]) => Iterator[U],
+ preservesPartitioning: Boolean = false): RDD[U] = withScope {
+ val cleanedF = sc.clean(f)
+ new MapPartitionsRDD(
+ this,
+ (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(index, iter),
+ preservesPartitioning)
}
/**
@@ -681,7 +716,7 @@ abstract class RDD[T: ClassTag](
@deprecated("use TaskContext.get", "1.2.0")
def mapPartitionsWithContext[U: ClassTag](
f: (TaskContext, Iterator[T]) => Iterator[U],
- preservesPartitioning: Boolean = false): RDD[U] = {
+ preservesPartitioning: Boolean = false): RDD[U] = withScope {
val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter)
new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
}
@@ -692,7 +727,8 @@ abstract class RDD[T: ClassTag](
*/
@deprecated("use mapPartitionsWithIndex", "0.7.0")
def mapPartitionsWithSplit[U: ClassTag](
- f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
+ f: (Int, Iterator[T]) => Iterator[U],
+ preservesPartitioning: Boolean = false): RDD[U] = withScope {
mapPartitionsWithIndex(f, preservesPartitioning)
}
@@ -704,7 +740,7 @@ abstract class RDD[T: ClassTag](
@deprecated("use mapPartitionsWithIndex", "1.0.0")
def mapWith[A, U: ClassTag]
(constructA: Int => A, preservesPartitioning: Boolean = false)
- (f: (T, A) => U): RDD[U] = {
+ (f: (T, A) => U): RDD[U] = withScope {
mapPartitionsWithIndex((index, iter) => {
val a = constructA(index)
iter.map(t => f(t, a))
@@ -719,7 +755,7 @@ abstract class RDD[T: ClassTag](
@deprecated("use mapPartitionsWithIndex and flatMap", "1.0.0")
def flatMapWith[A, U: ClassTag]
(constructA: Int => A, preservesPartitioning: Boolean = false)
- (f: (T, A) => Seq[U]): RDD[U] = {
+ (f: (T, A) => Seq[U]): RDD[U] = withScope {
mapPartitionsWithIndex((index, iter) => {
val a = constructA(index)
iter.flatMap(t => f(t, a))
@@ -732,11 +768,11 @@ abstract class RDD[T: ClassTag](
* partition with the index of that partition.
*/
@deprecated("use mapPartitionsWithIndex and foreach", "1.0.0")
- def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit) {
+ def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit): Unit = withScope {
mapPartitionsWithIndex { (index, iter) =>
val a = constructA(index)
iter.map(t => {f(t, a); t})
- }.foreach(_ => {})
+ }
}
/**
@@ -745,7 +781,7 @@ abstract class RDD[T: ClassTag](
* partition with the index of that partition.
*/
@deprecated("use mapPartitionsWithIndex and filter", "1.0.0")
- def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
+ def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = withScope {
mapPartitionsWithIndex((index, iter) => {
val a = constructA(index)
iter.filter(t => p(t, a))
@@ -758,7 +794,7 @@ abstract class RDD[T: ClassTag](
* partitions* and the *same number of elements in each partition* (e.g. one was made through
* a map on the other).
*/
- def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = {
+ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope {
zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) =>
new Iterator[(T, U)] {
def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match {
@@ -780,33 +816,39 @@ abstract class RDD[T: ClassTag](
*/
def zipPartitions[B: ClassTag, V: ClassTag]
(rdd2: RDD[B], preservesPartitioning: Boolean)
- (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
+ (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope {
new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning)
+ }
def zipPartitions[B: ClassTag, V: ClassTag]
(rdd2: RDD[B])
- (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
- new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false)
+ (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope {
+ zipPartitions(rdd2, preservesPartitioning = false)(f)
+ }
def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean)
- (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
+ (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = withScope {
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning)
+ }
def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C])
- (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
- new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false)
+ (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = withScope {
+ zipPartitions(rdd2, rdd3, preservesPartitioning = false)(f)
+ }
def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean)
- (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
+ (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = withScope {
new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning)
+ }
def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D])
- (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
- new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, false)
+ (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = withScope {
+ zipPartitions(rdd2, rdd3, rdd4, preservesPartitioning = false)(f)
+ }
// Actions (launch a job to return a value to the user program)
@@ -814,7 +856,7 @@ abstract class RDD[T: ClassTag](
/**
* Applies a function f to all elements of this RDD.
*/
- def foreach(f: T => Unit) {
+ def foreach(f: T => Unit): Unit = withScope {
val cleanF = sc.clean(f)
sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF))
}
@@ -822,7 +864,7 @@ abstract class RDD[T: ClassTag](
/**
* Applies a function f to each partition of this RDD.
*/
- def foreachPartition(f: Iterator[T] => Unit) {
+ def foreachPartition(f: Iterator[T] => Unit): Unit = withScope {
val cleanF = sc.clean(f)
sc.runJob(this, (iter: Iterator[T]) => cleanF(iter))
}
@@ -830,7 +872,7 @@ abstract class RDD[T: ClassTag](
/**
* Return an array that contains all of the elements in this RDD.
*/
- def collect(): Array[T] = {
+ def collect(): Array[T] = withScope {
val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
Array.concat(results: _*)
}
@@ -840,7 +882,7 @@ abstract class RDD[T: ClassTag](
*
* The iterator will consume as much memory as the largest partition in this RDD.
*/
- def toLocalIterator: Iterator[T] = {
+ def toLocalIterator: Iterator[T] = withScope {
def collectPartition(p: Int): Array[T] = {
sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head
}
@@ -851,12 +893,14 @@ abstract class RDD[T: ClassTag](
* Return an array that contains all of the elements in this RDD.
*/
@deprecated("use collect", "1.0.0")
- def toArray(): Array[T] = collect()
+ def toArray(): Array[T] = withScope {
+ collect()
+ }
/**
* Return an RDD that contains all matching values by applying `f`.
*/
- def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = {
+ def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = withScope {
filter(f.isDefinedAt).map(f)
}
@@ -866,19 +910,23 @@ abstract class RDD[T: ClassTag](
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
- def subtract(other: RDD[T]): RDD[T] =
+ def subtract(other: RDD[T]): RDD[T] = withScope {
subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.length)))
+ }
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
- def subtract(other: RDD[T], numPartitions: Int): RDD[T] =
+ def subtract(other: RDD[T], numPartitions: Int): RDD[T] = withScope {
subtract(other, new HashPartitioner(numPartitions))
+ }
/**
* Return an RDD with the elements from `this` that are not in `other`.
*/
- def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = {
+ def subtract(
+ other: RDD[T],
+ p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = withScope {
if (partitioner == Some(p)) {
// Our partitioner knows how to handle T (which, since we have a partitioner, is
// really (K, V)) so make a new Partitioner that will de-tuple our fake tuples
@@ -900,7 +948,7 @@ abstract class RDD[T: ClassTag](
* Reduces the elements of this RDD using the specified commutative and
* associative binary operator.
*/
- def reduce(f: (T, T) => T): T = {
+ def reduce(f: (T, T) => T): T = withScope {
val cleanF = sc.clean(f)
val reducePartition: Iterator[T] => Option[T] = iter => {
if (iter.hasNext) {
@@ -929,7 +977,7 @@ abstract class RDD[T: ClassTag](
* @param depth suggested depth of the tree (default: 2)
* @see [[org.apache.spark.rdd.RDD#reduce]]
*/
- def treeReduce(f: (T, T) => T, depth: Int = 2): T = {
+ def treeReduce(f: (T, T) => T, depth: Int = 2): T = withScope {
require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.")
val cleanF = context.clean(f)
val reducePartition: Iterator[T] => Option[T] = iter => {
@@ -961,7 +1009,7 @@ abstract class RDD[T: ClassTag](
* modify t1 and return it as its result value to avoid object allocation; however, it should not
* modify t2.
*/
- def fold(zeroValue: T)(op: (T, T) => T): T = {
+ def fold(zeroValue: T)(op: (T, T) => T): T = withScope {
// Clone the zero value since we will also be serializing it as part of tasks
var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance())
val cleanOp = sc.clean(op)
@@ -979,7 +1027,7 @@ abstract class RDD[T: ClassTag](
* allowed to modify and return their first argument instead of creating a new U to avoid memory
* allocation.
*/
- def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = {
+ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = withScope {
// Clone the zero value since we will also be serializing it as part of tasks
var jobResult = Utils.clone(zeroValue, sc.env.serializer.newInstance())
val cleanSeqOp = sc.clean(seqOp)
@@ -999,26 +1047,29 @@ abstract class RDD[T: ClassTag](
def treeAggregate[U: ClassTag](zeroValue: U)(
seqOp: (U, T) => U,
combOp: (U, U) => U,
- depth: Int = 2): U = {
+ depth: Int = 2): U = withScope {
require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.")
if (partitions.length == 0) {
- return Utils.clone(zeroValue, context.env.closureSerializer.newInstance())
- }
- val cleanSeqOp = context.clean(seqOp)
- val cleanCombOp = context.clean(combOp)
- val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp)
- var partiallyAggregated = mapPartitions(it => Iterator(aggregatePartition(it)))
- var numPartitions = partiallyAggregated.partitions.length
- val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2)
- // If creating an extra level doesn't help reduce the wall-clock time, we stop tree aggregation.
- while (numPartitions > scale + numPartitions / scale) {
- numPartitions /= scale
- val curNumPartitions = numPartitions
- partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { (i, iter) =>
- iter.map((i % curNumPartitions, _))
- }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values
+ Utils.clone(zeroValue, context.env.closureSerializer.newInstance())
+ } else {
+ val cleanSeqOp = context.clean(seqOp)
+ val cleanCombOp = context.clean(combOp)
+ val aggregatePartition =
+ (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp)
+ var partiallyAggregated = mapPartitions(it => Iterator(aggregatePartition(it)))
+ var numPartitions = partiallyAggregated.partitions.length
+ val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2)
+ // If creating an extra level doesn't help reduce
+ // the wall-clock time, we stop tree aggregation.
+ while (numPartitions > scale + numPartitions / scale) {
+ numPartitions /= scale
+ val curNumPartitions = numPartitions
+ partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex {
+ (i, iter) => iter.map((i % curNumPartitions, _))
+ }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values
+ }
+ partiallyAggregated.reduce(cleanCombOp)
}
- partiallyAggregated.reduce(cleanCombOp)
}
/**
@@ -1032,7 +1083,9 @@ abstract class RDD[T: ClassTag](
* within a timeout, even if not all tasks have finished.
*/
@Experimental
- def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
+ def countApprox(
+ timeout: Long,
+ confidence: Double = 0.95): PartialResult[BoundedDouble] = withScope {
val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) =>
var result = 0L
while (iter.hasNext) {
@@ -1053,7 +1106,7 @@ abstract class RDD[T: ClassTag](
* To handle very large results, consider using rdd.map(x => (x, 1L)).reduceByKey(_ + _), which
* returns an RDD[T, Long] instead of a map.
*/
- def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = {
+ def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = withScope {
map(value => (value, null)).countByKey()
}
@@ -1064,8 +1117,7 @@ abstract class RDD[T: ClassTag](
@Experimental
def countByValueApprox(timeout: Long, confidence: Double = 0.95)
(implicit ord: Ordering[T] = null)
- : PartialResult[Map[T, BoundedDouble]] =
- {
+ : PartialResult[Map[T, BoundedDouble]] = withScope {
if (elementClassTag.runtimeClass.isArray) {
throw new SparkException("countByValueApprox() does not support arrays")
}
@@ -1098,7 +1150,7 @@ abstract class RDD[T: ClassTag](
* If `sp` equals 0, the sparse representation is skipped.
*/
@Experimental
- def countApproxDistinct(p: Int, sp: Int): Long = {
+ def countApproxDistinct(p: Int, sp: Int): Long = withScope {
require(p >= 4, s"p ($p) must be at least 4")
require(sp <= 32, s"sp ($sp) cannot be greater than 32")
require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)")
@@ -1124,7 +1176,7 @@ abstract class RDD[T: ClassTag](
* @param relativeSD Relative accuracy. Smaller values create counters that require more space.
* It must be greater than 0.000017.
*/
- def countApproxDistinct(relativeSD: Double = 0.05): Long = {
+ def countApproxDistinct(relativeSD: Double = 0.05): Long = withScope {
val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt
countApproxDistinct(p, 0)
}
@@ -1142,7 +1194,9 @@ abstract class RDD[T: ClassTag](
* and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee
* the same index assignments, you should sort the RDD with sortByKey() or save it to a file.
*/
- def zipWithIndex(): RDD[(T, Long)] = new ZippedWithIndexRDD(this)
+ def zipWithIndex(): RDD[(T, Long)] = withScope {
+ new ZippedWithIndexRDD(this)
+ }
/**
* Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k,
@@ -1154,7 +1208,7 @@ abstract class RDD[T: ClassTag](
* and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee
* the same index assignments, you should sort the RDD with sortByKey() or save it to a file.
*/
- def zipWithUniqueId(): RDD[(T, Long)] = {
+ def zipWithUniqueId(): RDD[(T, Long)] = withScope {
val n = this.partitions.length.toLong
this.mapPartitionsWithIndex { case (k, iter) =>
iter.zipWithIndex.map { case (item, i) =>
@@ -1171,48 +1225,50 @@ abstract class RDD[T: ClassTag](
* @note due to complications in the internal implementation, this method will raise
* an exception if called on an RDD of `Nothing` or `Null`.
*/
- def take(num: Int): Array[T] = {
+ def take(num: Int): Array[T] = withScope {
if (num == 0) {
- return new Array[T](0)
- }
-
- val buf = new ArrayBuffer[T]
- val totalParts = this.partitions.length
- var partsScanned = 0
- while (buf.size < num && partsScanned < totalParts) {
- // The number of partitions to try in this iteration. It is ok for this number to be
- // greater than totalParts because we actually cap it at totalParts in runJob.
- var numPartsToTry = 1
- if (partsScanned > 0) {
- // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise,
- // interpolate the number of partitions we need to try, but overestimate it by 50%.
- // We also cap the estimation in the end.
- if (buf.size == 0) {
- numPartsToTry = partsScanned * 4
- } else {
- // the left side of max is >=1 whenever partsScanned >= 2
- numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1)
- numPartsToTry = Math.min(numPartsToTry, partsScanned * 4)
+ new Array[T](0)
+ } else {
+ val buf = new ArrayBuffer[T]
+ val totalParts = this.partitions.length
+ var partsScanned = 0
+ while (buf.size < num && partsScanned < totalParts) {
+ // The number of partitions to try in this iteration. It is ok for this number to be
+ // greater than totalParts because we actually cap it at totalParts in runJob.
+ var numPartsToTry = 1
+ if (partsScanned > 0) {
+ // If we didn't find any rows after the previous iteration, quadruple and retry.
+ // Otherwise, interpolate the number of partitions we need to try, but overestimate
+ // it by 50%. We also cap the estimation in the end.
+ if (buf.size == 0) {
+ numPartsToTry = partsScanned * 4
+ } else {
+ // the left side of max is >=1 whenever partsScanned >= 2
+ numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1)
+ numPartsToTry = Math.min(numPartsToTry, partsScanned * 4)
+ }
}
- }
- val left = num - buf.size
- val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
- val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true)
+ val left = num - buf.size
+ val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
+ val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true)
- res.foreach(buf ++= _.take(num - buf.size))
- partsScanned += numPartsToTry
- }
+ res.foreach(buf ++= _.take(num - buf.size))
+ partsScanned += numPartsToTry
+ }
- buf.toArray
+ buf.toArray
+ }
}
/**
* Return the first element in this RDD.
*/
- def first(): T = take(1) match {
- case Array(t) => t
- case _ => throw new UnsupportedOperationException("empty collection")
+ def first(): T = withScope {
+ take(1) match {
+ case Array(t) => t
+ case _ => throw new UnsupportedOperationException("empty collection")
+ }
}
/**
@@ -1230,7 +1286,9 @@ abstract class RDD[T: ClassTag](
* @param ord the implicit ordering for T
* @return an array of top elements
*/
- def top(num: Int)(implicit ord: Ordering[T]): Array[T] = takeOrdered(num)(ord.reverse)
+ def top(num: Int)(implicit ord: Ordering[T]): Array[T] = withScope {
+ takeOrdered(num)(ord.reverse)
+ }
/**
* Returns the first k (smallest) elements from this RDD as defined by the specified
@@ -1248,7 +1306,7 @@ abstract class RDD[T: ClassTag](
* @param ord the implicit ordering for T
* @return an array of top elements
*/
- def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = {
+ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = withScope {
if (num == 0) {
Array.empty
} else {
@@ -1273,13 +1331,17 @@ abstract class RDD[T: ClassTag](
* Returns the max of this RDD as defined by the implicit Ordering[T].
* @return the maximum element of the RDD
* */
- def max()(implicit ord: Ordering[T]): T = this.reduce(ord.max)
+ def max()(implicit ord: Ordering[T]): T = withScope {
+ this.reduce(ord.max)
+ }
/**
* Returns the min of this RDD as defined by the implicit Ordering[T].
* @return the minimum element of the RDD
* */
- def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min)
+ def min()(implicit ord: Ordering[T]): T = withScope {
+ this.reduce(ord.min)
+ }
/**
* @note due to complications in the internal implementation, this method will raise an
@@ -1289,12 +1351,14 @@ abstract class RDD[T: ClassTag](
* @return true if and only if the RDD contains no elements at all. Note that an RDD
* may be empty even when it has at least 1 partition.
*/
- def isEmpty(): Boolean = partitions.length == 0 || take(1).length == 0
+ def isEmpty(): Boolean = withScope {
+ partitions.length == 0 || take(1).length == 0
+ }
/**
* Save this RDD as a text file, using string representations of elements.
*/
- def saveAsTextFile(path: String) {
+ def saveAsTextFile(path: String): Unit = withScope {
// https://issues.apache.org/jira/browse/SPARK-2075
//
// NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit
@@ -1321,7 +1385,7 @@ abstract class RDD[T: ClassTag](
/**
* Save this RDD as a compressed text file, using string representations of elements.
*/
- def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) {
+ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = withScope {
// https://issues.apache.org/jira/browse/SPARK-2075
val nullWritableClassTag = implicitly[ClassTag[NullWritable]]
val textClassTag = implicitly[ClassTag[Text]]
@@ -1339,7 +1403,7 @@ abstract class RDD[T: ClassTag](
/**
* Save this RDD as a SequenceFile of serialized objects.
*/
- def saveAsObjectFile(path: String) {
+ def saveAsObjectFile(path: String): Unit = withScope {
this.mapPartitions(iter => iter.grouped(10).map(_.toArray))
.map(x => (NullWritable.get(), new BytesWritable(Utils.serialize(x))))
.saveAsSequenceFile(path)
@@ -1348,12 +1412,13 @@ abstract class RDD[T: ClassTag](
/**
* Creates tuples of the elements in this RDD by applying `f`.
*/
- def keyBy[K](f: T => K): RDD[(K, T)] = {
- map(x => (f(x), x))
+ def keyBy[K](f: T => K): RDD[(K, T)] = withScope {
+ val cleanedF = sc.clean(f)
+ map(x => (cleanedF(x), x))
}
/** A private method for tests, to look at the contents of each partition */
- private[spark] def collectPartitions(): Array[Array[T]] = {
+ private[spark] def collectPartitions(): Array[Array[T]] = withScope {
sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
}
@@ -1392,6 +1457,17 @@ abstract class RDD[T: ClassTag](
/** User code that created this RDD (e.g. `textFile`, `parallelize`). */
@transient private[spark] val creationSite = sc.getCallSite()
+ /**
+ * The scope associated with the operation that created this RDD.
+ *
+ * This is more flexible than the call site and can be defined hierarchically. For more
+ * detail, see the documentation of {{RDDOperationScope}}. This scope is not defined if the
+ * user instantiates this RDD himself without using any Spark operations.
+ */
+ @transient private[spark] val scope: Option[RDDOperationScope] = {
+ Option(sc.getLocalProperty(SparkContext.RDD_SCOPE_KEY)).map(RDDOperationScope.fromJson)
+ }
+
private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("")
private[spark] def elementClassTag: ClassTag[T] = classTag[T]
@@ -1470,7 +1546,7 @@ abstract class RDD[T: ClassTag](
/** A description of this RDD and its recursive dependencies for debugging. */
def toDebugString: String = {
// Get a debug description of an rdd without its children
- def debugSelf (rdd: RDD[_]): Seq[String] = {
+ def debugSelf(rdd: RDD[_]): Seq[String] = {
import Utils.bytesToString
val persistence = if (storageLevel != StorageLevel.NONE) storageLevel.description else ""
@@ -1527,10 +1603,11 @@ abstract class RDD[T: ClassTag](
case (desc: String, _) => s"$nextPrefix$desc"
} ++ debugChildren(rdd, nextPrefix)
}
- def debugString(rdd: RDD[_],
- prefix: String = "",
- isShuffle: Boolean = true,
- isLastChild: Boolean = false): Seq[String] = {
+ def debugString(
+ rdd: RDD[_],
+ prefix: String = "",
+ isShuffle: Boolean = true,
+ isLastChild: Boolean = false): Seq[String] = {
if (isShuffle) {
shuffleDebugString(rdd, prefix, isLastChild)
} else {
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala
new file mode 100644
index 0000000000000..537b56b49f866
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.rdd
+
+import java.util.concurrent.atomic.AtomicInteger
+
+import com.fasterxml.jackson.annotation.{JsonIgnore, JsonInclude, JsonPropertyOrder}
+import com.fasterxml.jackson.annotation.JsonInclude.Include
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+
+import org.apache.spark.SparkContext
+
+/**
+ * A general, named code block representing an operation that instantiates RDDs.
+ *
+ * All RDDs instantiated in the corresponding code block will store a pointer to this object.
+ * Examples include, but will not be limited to, existing RDD operations, such as textFile,
+ * reduceByKey, and treeAggregate.
+ *
+ * An operation scope may be nested in other scopes. For instance, a SQL query may enclose
+ * scopes associated with the public RDD APIs it uses under the hood.
+ *
+ * There is no particular relationship between an operation scope and a stage or a job.
+ * A scope may live inside one stage (e.g. map) or span across multiple jobs (e.g. take).
+ */
+@JsonInclude(Include.NON_NULL)
+@JsonPropertyOrder(Array("id", "name", "parent"))
+private[spark] class RDDOperationScope(
+ val name: String,
+ val parent: Option[RDDOperationScope] = None) {
+
+ val id: Int = RDDOperationScope.nextScopeId()
+
+ def toJson: String = {
+ RDDOperationScope.jsonMapper.writeValueAsString(this)
+ }
+
+ /**
+ * Return a list of scopes that this scope is a part of, including this scope itself.
+ * The result is ordered from the outermost scope (eldest ancestor) to this scope.
+ */
+ @JsonIgnore
+ def getAllScopes: Seq[RDDOperationScope] = {
+ parent.map(_.getAllScopes).getOrElse(Seq.empty) ++ Seq(this)
+ }
+
+ override def equals(other: Any): Boolean = {
+ other match {
+ case s: RDDOperationScope =>
+ id == s.id && name == s.name && parent == s.parent
+ case _ => false
+ }
+ }
+
+ override def toString: String = toJson
+}
+
+/**
+ * A collection of utility methods to construct a hierarchical representation of RDD scopes.
+ * An RDD scope tracks the series of operations that created a given RDD.
+ */
+private[spark] object RDDOperationScope {
+ private val jsonMapper = new ObjectMapper().registerModule(DefaultScalaModule)
+ private val scopeCounter = new AtomicInteger(0)
+
+ def fromJson(s: String): RDDOperationScope = {
+ jsonMapper.readValue(s, classOf[RDDOperationScope])
+ }
+
+ /** Return a globally unique operation scope ID. */
+ def nextScopeId(): Int = scopeCounter.getAndIncrement
+
+ /**
+ * Execute the given body such that all RDDs created in this body will have the same scope.
+ * The name of the scope will be the name of the method that immediately encloses this one.
+ *
+ * Note: Return statements are NOT allowed in body.
+ */
+ private[spark] def withScope[T](
+ sc: SparkContext,
+ allowNesting: Boolean = false)(body: => T): T = {
+ val callerMethodName = Thread.currentThread.getStackTrace()(3).getMethodName
+ withScope[T](sc, callerMethodName, allowNesting)(body)
+ }
+
+ /**
+ * Execute the given body such that all RDDs created in this body will have the same scope.
+ *
+ * If nesting is allowed, this concatenates the previous scope with the new one in a way that
+ * signifies the hierarchy. Otherwise, if nesting is not allowed, then any children calls to
+ * this method executed in the body will have no effect.
+ *
+ * Note: Return statements are NOT allowed in body.
+ */
+ private[spark] def withScope[T](
+ sc: SparkContext,
+ name: String,
+ allowNesting: Boolean = false)(body: => T): T = {
+ // Save the old scope to restore it later
+ val scopeKey = SparkContext.RDD_SCOPE_KEY
+ val noOverrideKey = SparkContext.RDD_SCOPE_NO_OVERRIDE_KEY
+ val oldScopeJson = sc.getLocalProperty(scopeKey)
+ val oldScope = Option(oldScopeJson).map(RDDOperationScope.fromJson)
+ val oldNoOverride = sc.getLocalProperty(noOverrideKey)
+ try {
+ // Set the scope only if the higher level caller allows us to do so
+ if (sc.getLocalProperty(noOverrideKey) == null) {
+ sc.setLocalProperty(scopeKey, new RDDOperationScope(name, oldScope).toJson)
+ }
+ // Optionally disallow the child body to override our scope
+ if (!allowNesting) {
+ sc.setLocalProperty(noOverrideKey, "true")
+ }
+ body
+ } finally {
+ // Remember to restore any state that was modified before exiting
+ sc.setLocalProperty(scopeKey, oldScopeJson)
+ sc.setLocalProperty(noOverrideKey, oldNoOverride)
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
index 059f8963691f0..3dfcf67f0eb66 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
@@ -85,7 +85,9 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag
* byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported
* file system.
*/
- def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) {
+ def saveAsSequenceFile(
+ path: String,
+ codec: Option[Class[_ <: CompressionCodec]] = None): Unit = self.withScope {
def anyToWritable[U <% Writable](u: U): Writable = u
// TODO We cannot force the return type of `anyToWritable` be same as keyWritableClass and
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index cf3db0b94a0b3..e439d2a7e1229 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -33,6 +33,7 @@ class StageInfo(
val name: String,
val numTasks: Int,
val rddInfos: Seq[RDDInfo],
+ val parentIds: Seq[Int],
val details: String) {
/** When this stage was submitted from the DAGScheduler to a TaskScheduler. */
var submissionTime: Option[Long] = None
@@ -78,6 +79,7 @@ private[spark] object StageInfo {
stage.name,
numTasks.getOrElse(stage.numTasks),
rddInfos,
+ stage.parents.map(_.id),
stage.details)
}
}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala
new file mode 100644
index 0000000000000..5783df5d8220c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.{Arrays, Date, List => JList}
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(ui: SparkUI) {
+
+ @GET
+ def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = {
+ val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
+ AllJobsResource.getStatusToJobs(ui)
+ val adjStatuses: JList[JobExecutionStatus] = {
+ if (statuses.isEmpty) {
+ Arrays.asList(JobExecutionStatus.values(): _*)
+ } else {
+ statuses
+ }
+ }
+ val jobInfos = for {
+ (status, jobs) <- statusToJobs
+ job <- jobs if adjStatuses.contains(status)
+ } yield {
+ AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
+ }
+ jobInfos.sortBy{- _.jobId}
+ }
+
+}
+
+private[v1] object AllJobsResource {
+
+ def getStatusToJobs(ui: SparkUI): Seq[(JobExecutionStatus, Seq[JobUIData])] = {
+ val statusToJobs = ui.jobProgressListener.synchronized {
+ Seq(
+ JobExecutionStatus.RUNNING -> ui.jobProgressListener.activeJobs.values.toSeq,
+ JobExecutionStatus.SUCCEEDED -> ui.jobProgressListener.completedJobs.toSeq,
+ JobExecutionStatus.FAILED -> ui.jobProgressListener.failedJobs.reverse.toSeq
+ )
+ }
+ statusToJobs
+ }
+
+ def convertJobData(
+ job: JobUIData,
+ listener: JobProgressListener,
+ includeStageDetails: Boolean): JobData = {
+ listener.synchronized {
+ val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+ val lastStageData = lastStageInfo.flatMap { s =>
+ listener.stageIdToData.get((s.stageId, s.attemptId))
+ }
+ val lastStageName = lastStageInfo.map { _.name }.getOrElse("(Unknown Stage Name)")
+ val lastStageDescription = lastStageData.flatMap { _.description }
+ new JobData(
+ jobId = job.jobId,
+ name = lastStageName,
+ description = lastStageDescription,
+ submissionTime = job.submissionTime.map{new Date(_)},
+ completionTime = job.completionTime.map{new Date(_)},
+ stageIds = job.stageIds,
+ jobGroup = job.jobGroup,
+ status = job.status,
+ numTasks = job.numTasks,
+ numActiveTasks = job.numActiveTasks,
+ numCompletedTasks = job.numCompletedTasks,
+ numSkippedTasks = job.numCompletedTasks,
+ numFailedTasks = job.numFailedTasks,
+ numActiveStages = job.numActiveStages,
+ numCompletedStages = job.completedStageIndices.size,
+ numSkippedStages = job.numSkippedStages,
+ numFailedStages = job.numFailedStages
+ )
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
new file mode 100644
index 0000000000000..645ede26a0879
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{GET, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.storage.StorageListener
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllRDDResource(ui: SparkUI) {
+
+ @GET
+ def rddList(): Seq[RDDStorageInfo] = {
+ val storageStatusList = ui.storageListener.storageStatusList
+ val rddInfos = ui.storageListener.rddInfoList
+ rddInfos.map{rddInfo =>
+ AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList,
+ includeDetails = false)
+ }
+ }
+
+}
+
+private[spark] object AllRDDResource {
+
+ def getRDDStorageInfo(
+ rddId: Int,
+ listener: StorageListener,
+ includeDetails: Boolean): Option[RDDStorageInfo] = {
+ val storageStatusList = listener.storageStatusList
+ listener.rddInfoList.find { _.id == rddId }.map { rddInfo =>
+ getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails)
+ }
+ }
+
+ def getRDDStorageInfo(
+ rddId: Int,
+ rddInfo: RDDInfo,
+ storageStatusList: Seq[StorageStatus],
+ includeDetails: Boolean): RDDStorageInfo = {
+ val workers = storageStatusList.map { (rddId, _) }
+ val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList)
+ val blocks = storageStatusList
+ .flatMap { _.rddBlocksById(rddId) }
+ .sortWith { _._1.name < _._1.name }
+ .map { case (blockId, status) =>
+ (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown")))
+ }
+
+ val dataDistribution = if (includeDetails) {
+ Some(storageStatusList.map { status =>
+ new RDDDataDistribution(
+ address = status.blockManagerId.hostPort,
+ memoryUsed = status.memUsedByRdd(rddId),
+ memoryRemaining = status.memRemaining,
+ diskUsed = status.diskUsedByRdd(rddId)
+ ) } )
+ } else {
+ None
+ }
+ val partitions = if (includeDetails) {
+ Some(blocks.map { case (id, block, locations) =>
+ new RDDPartitionInfo(
+ blockName = id.name,
+ storageLevel = block.storageLevel.description,
+ memoryUsed = block.memSize,
+ diskUsed = block.diskSize,
+ executors = locations
+ )
+ } )
+ } else {
+ None
+ }
+
+ new RDDStorageInfo(
+ id = rddId,
+ name = rddInfo.name,
+ numPartitions = rddInfo.numPartitions,
+ numCachedPartitions = rddInfo.numCachedPartitions,
+ storageLevel = rddInfo.storageLevel.description,
+ memoryUsed = rddInfo.memSize,
+ diskUsed = rddInfo.diskSize,
+ dataDistribution = dataDistribution,
+ partitions = partitions
+ )
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
new file mode 100644
index 0000000000000..50608588f09ae
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.{Arrays, Date, List => JList}
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(ui: SparkUI) {
+
+ @GET
+ def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = {
+ val listener = ui.jobProgressListener
+ val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+ val adjStatuses = {
+ if (statuses.isEmpty()) {
+ Arrays.asList(StageStatus.values(): _*)
+ } else {
+ statuses
+ }
+ }
+ for {
+ (status, stageList) <- stageAndStatus
+ stageInfo: StageInfo <- stageList if adjStatuses.contains(status)
+ stageUiData: StageUIData <- listener.synchronized {
+ listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId))
+ }
+ } yield {
+ AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false)
+ }
+ }
+}
+
+private[v1] object AllStagesResource {
+ def stageUiToStageData(
+ status: StageStatus,
+ stageInfo: StageInfo,
+ stageUiData: StageUIData,
+ includeDetails: Boolean): StageData = {
+
+ val taskData = if (includeDetails) {
+ Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } )
+ } else {
+ None
+ }
+ val executorSummary = if (includeDetails) {
+ Some(stageUiData.executorSummary.map { case (k, summary) =>
+ k -> new ExecutorStageSummary(
+ taskTime = summary.taskTime,
+ failedTasks = summary.failedTasks,
+ succeededTasks = summary.succeededTasks,
+ inputBytes = summary.inputBytes,
+ outputBytes = summary.outputBytes,
+ shuffleRead = summary.shuffleRead,
+ shuffleWrite = summary.shuffleWrite,
+ memoryBytesSpilled = summary.memoryBytesSpilled,
+ diskBytesSpilled = summary.diskBytesSpilled
+ )
+ })
+ } else {
+ None
+ }
+
+ val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq
+
+ new StageData(
+ status = status,
+ stageId = stageInfo.stageId,
+ attemptId = stageInfo.attemptId,
+ numActiveTasks = stageUiData.numActiveTasks,
+ numCompleteTasks = stageUiData.numCompleteTasks,
+ numFailedTasks = stageUiData.numFailedTasks,
+ executorRunTime = stageUiData.executorRunTime,
+ inputBytes = stageUiData.inputBytes,
+ inputRecords = stageUiData.inputRecords,
+ outputBytes = stageUiData.outputBytes,
+ outputRecords = stageUiData.outputRecords,
+ shuffleReadBytes = stageUiData.shuffleReadTotalBytes,
+ shuffleReadRecords = stageUiData.shuffleReadRecords,
+ shuffleWriteBytes = stageUiData.shuffleWriteBytes,
+ shuffleWriteRecords = stageUiData.shuffleWriteRecords,
+ memoryBytesSpilled = stageUiData.memoryBytesSpilled,
+ diskBytesSpilled = stageUiData.diskBytesSpilled,
+ schedulingPool = stageUiData.schedulingPool,
+ name = stageInfo.name,
+ details = stageInfo.details,
+ accumulatorUpdates = accumulableInfo,
+ tasks = taskData,
+ executorSummary = executorSummary
+ )
+ }
+
+ def stagesAndStatus(ui: SparkUI): Seq[(StageStatus, Seq[StageInfo])] = {
+ val listener = ui.jobProgressListener
+ listener.synchronized {
+ Seq(
+ StageStatus.ACTIVE -> listener.activeStages.values.toSeq,
+ StageStatus.COMPLETE -> listener.completedStages.reverse.toSeq,
+ StageStatus.FAILED -> listener.failedStages.reverse.toSeq,
+ StageStatus.PENDING -> listener.pendingStages.values.toSeq
+ )
+ }
+ }
+
+ def convertTaskData(uiData: TaskUIData): TaskData = {
+ new TaskData(
+ taskId = uiData.taskInfo.taskId,
+ index = uiData.taskInfo.index,
+ attempt = uiData.taskInfo.attempt,
+ launchTime = new Date(uiData.taskInfo.launchTime),
+ executorId = uiData.taskInfo.executorId,
+ host = uiData.taskInfo.host,
+ taskLocality = uiData.taskInfo.taskLocality.toString(),
+ speculative = uiData.taskInfo.speculative,
+ accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo },
+ errorMessage = uiData.errorMessage,
+ taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics }
+ )
+ }
+
+ def taskMetricDistributions(
+ allTaskData: Iterable[TaskUIData],
+ quantiles: Array[Double]): TaskMetricDistributions = {
+
+ val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq
+
+ def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] =
+ Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles)
+
+ // We need to do a lot of similar munging to nested metrics here. For each one,
+ // we want (a) extract the values for nested metrics (b) make a distribution for each metric
+ // (c) shove the distribution into the right field in our return type and (d) only return
+ // a result if the option is defined for any of the tasks. MetricHelper is a little util
+ // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just
+ // implement one "build" method, which just builds the quantiles for each field.
+
+ val inputMetrics: Option[InputMetricDistributions] =
+ new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) {
+ def getSubmetrics(raw: InternalTaskMetrics): Option[InternalInputMetrics] = {
+ raw.inputMetrics
+ }
+
+ def build: InputMetricDistributions = new InputMetricDistributions(
+ bytesRead = submetricQuantiles(_.bytesRead),
+ recordsRead = submetricQuantiles(_.recordsRead)
+ )
+ }.metricOption
+
+ val outputMetrics: Option[OutputMetricDistributions] =
+ new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) {
+ def getSubmetrics(raw:InternalTaskMetrics): Option[InternalOutputMetrics] = {
+ raw.outputMetrics
+ }
+ def build: OutputMetricDistributions = new OutputMetricDistributions(
+ bytesWritten = submetricQuantiles(_.bytesWritten),
+ recordsWritten = submetricQuantiles(_.recordsWritten)
+ )
+ }.metricOption
+
+ val shuffleReadMetrics: Option[ShuffleReadMetricDistributions] =
+ new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics,
+ quantiles) {
+ def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleReadMetrics] = {
+ raw.shuffleReadMetrics
+ }
+ def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions(
+ readBytes = submetricQuantiles(_.totalBytesRead),
+ readRecords = submetricQuantiles(_.recordsRead),
+ remoteBytesRead = submetricQuantiles(_.remoteBytesRead),
+ remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched),
+ localBlocksFetched = submetricQuantiles(_.localBlocksFetched),
+ totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched),
+ fetchWaitTime = submetricQuantiles(_.fetchWaitTime)
+ )
+ }.metricOption
+
+ val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions] =
+ new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics,
+ quantiles) {
+ def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleWriteMetrics] = {
+ raw.shuffleWriteMetrics
+ }
+ def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions(
+ writeBytes = submetricQuantiles(_.shuffleBytesWritten),
+ writeRecords = submetricQuantiles(_.shuffleRecordsWritten),
+ writeTime = submetricQuantiles(_.shuffleWriteTime)
+ )
+ }.metricOption
+
+ new TaskMetricDistributions(
+ quantiles = quantiles,
+ executorDeserializeTime = metricQuantiles(_.executorDeserializeTime),
+ executorRunTime = metricQuantiles(_.executorRunTime),
+ resultSize = metricQuantiles(_.resultSize),
+ jvmGcTime = metricQuantiles(_.jvmGCTime),
+ resultSerializationTime = metricQuantiles(_.resultSerializationTime),
+ memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled),
+ diskBytesSpilled = metricQuantiles(_.diskBytesSpilled),
+ inputMetrics = inputMetrics,
+ outputMetrics = outputMetrics,
+ shuffleReadMetrics = shuffleReadMetrics,
+ shuffleWriteMetrics = shuffleWriteMetrics
+ )
+ }
+
+ def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = {
+ new AccumulableInfo(acc.id, acc.name, acc.update, acc.value)
+ }
+
+ def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = {
+ new TaskMetrics(
+ executorDeserializeTime = internal.executorDeserializeTime,
+ executorRunTime = internal.executorRunTime,
+ resultSize = internal.resultSize,
+ jvmGcTime = internal.jvmGCTime,
+ resultSerializationTime = internal.resultSerializationTime,
+ memoryBytesSpilled = internal.memoryBytesSpilled,
+ diskBytesSpilled = internal.diskBytesSpilled,
+ inputMetrics = internal.inputMetrics.map { convertInputMetrics },
+ outputMetrics = Option(internal.outputMetrics).flatten.map { convertOutputMetrics },
+ shuffleReadMetrics = internal.shuffleReadMetrics.map { convertShuffleReadMetrics },
+ shuffleWriteMetrics = internal.shuffleWriteMetrics.map { convertShuffleWriteMetrics }
+ )
+ }
+
+ def convertInputMetrics(internal: InternalInputMetrics): InputMetrics = {
+ new InputMetrics(
+ bytesRead = internal.bytesRead,
+ recordsRead = internal.recordsRead
+ )
+ }
+
+ def convertOutputMetrics(internal: InternalOutputMetrics): OutputMetrics = {
+ new OutputMetrics(
+ bytesWritten = internal.bytesWritten,
+ recordsWritten = internal.recordsWritten
+ )
+ }
+
+ def convertShuffleReadMetrics(internal: InternalShuffleReadMetrics): ShuffleReadMetrics = {
+ new ShuffleReadMetrics(
+ remoteBlocksFetched = internal.remoteBlocksFetched,
+ localBlocksFetched = internal.localBlocksFetched,
+ fetchWaitTime = internal.fetchWaitTime,
+ remoteBytesRead = internal.remoteBytesRead,
+ totalBlocksFetched = internal.totalBlocksFetched,
+ recordsRead = internal.recordsRead
+ )
+ }
+
+ def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = {
+ new ShuffleWriteMetrics(
+ bytesWritten = internal.shuffleBytesWritten,
+ writeTime = internal.shuffleWriteTime,
+ recordsWritten = internal.shuffleRecordsWritten
+ )
+ }
+}
+
+/**
+ * Helper for getting distributions from nested metric types. Many of the metrics we want are
+ * contained in options inside TaskMetrics (eg., ShuffleWriteMetrics). This makes it easy to handle
+ * the options (returning None if the metrics are all empty), and extract the quantiles for each
+ * metric. After creating an instance, call metricOption to get the result type.
+ */
+private[v1] abstract class MetricHelper[I,O](
+ rawMetrics: Seq[InternalTaskMetrics],
+ quantiles: Array[Double]) {
+
+ def getSubmetrics(raw: InternalTaskMetrics): Option[I]
+
+ def build: O
+
+ val data: Seq[I] = rawMetrics.flatMap(getSubmetrics)
+
+ /** applies the given function to all input metrics, and returns the quantiles */
+ def submetricQuantiles(f: I => Double): IndexedSeq[Double] = {
+ Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles)
+ }
+
+ def metricOption: Option[O] = {
+ if (data.isEmpty) {
+ None
+ } else {
+ Some(build)
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala
new file mode 100644
index 0000000000000..17b521f3e1d41
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.{Arrays, Date, List => JList}
+import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.deploy.history.ApplicationHistoryInfo
+import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo}
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class ApplicationListResource(uiRoot: UIRoot) {
+
+ @GET
+ def appList(
+ @QueryParam("status") status: JList[ApplicationStatus],
+ @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam,
+ @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam)
+ : Iterator[ApplicationInfo] = {
+ val allApps = uiRoot.getApplicationInfoList
+ val adjStatus = {
+ if (status.isEmpty) {
+ Arrays.asList(ApplicationStatus.values(): _*)
+ } else {
+ status
+ }
+ }
+ val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED)
+ val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING)
+ allApps.filter { app =>
+ val anyRunning = app.attempts.exists(!_.completed)
+ // if any attempt is still running, we consider the app to also still be running
+ val statusOk = (!anyRunning && includeCompleted) ||
+ (anyRunning && includeRunning)
+ // keep the app if *any* attempts fall in the right time window
+ val dateOk = app.attempts.exists { attempt =>
+ attempt.startTime.getTime >= minDate.timestamp &&
+ attempt.startTime.getTime <= maxDate.timestamp
+ }
+ statusOk && dateOk
+ }
+ }
+}
+
+private[spark] object ApplicationsListResource {
+ def appHistoryInfoToPublicAppInfo(app: ApplicationHistoryInfo): ApplicationInfo = {
+ new ApplicationInfo(
+ id = app.id,
+ name = app.name,
+ attempts = app.attempts.map { internalAttemptInfo =>
+ new ApplicationAttemptInfo(
+ attemptId = internalAttemptInfo.attemptId,
+ startTime = new Date(internalAttemptInfo.startTime),
+ endTime = new Date(internalAttemptInfo.endTime),
+ sparkUser = internalAttemptInfo.sparkUser,
+ completed = internalAttemptInfo.completed
+ )
+ }
+ )
+ }
+
+ def convertApplicationInfo(
+ internal: InternalApplicationInfo,
+ completed: Boolean): ApplicationInfo = {
+ // standalone application info always has just one attempt
+ new ApplicationInfo(
+ id = internal.id,
+ name = internal.desc.name,
+ attempts = Seq(new ApplicationAttemptInfo(
+ attemptId = None,
+ startTime = new Date(internal.startTime),
+ endTime = new Date(internal.endTime),
+ sparkUser = internal.desc.user,
+ completed = completed
+ ))
+ )
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
new file mode 100644
index 0000000000000..8ad4656b4dada
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
@@ -0,0 +1,36 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements. See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{GET, PathParam, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.exec.ExecutorsPage
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class ExecutorListResource(ui: SparkUI) {
+
+ @GET
+ def executorList(): Seq[ExecutorSummary] = {
+ val listener = ui.executorsListener
+ val storageStatusList = listener.storageStatusList
+ (0 until storageStatusList.size).map { statusId =>
+ ExecutorsPage.getExecInfo(listener, statusId)
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
new file mode 100644
index 0000000000000..202a5191ad57d
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.io.OutputStream
+import java.lang.annotation.Annotation
+import java.lang.reflect.Type
+import java.text.SimpleDateFormat
+import java.util.{Calendar, SimpleTimeZone}
+import javax.ws.rs.Produces
+import javax.ws.rs.core.{MediaType, MultivaluedMap}
+import javax.ws.rs.ext.{MessageBodyWriter, Provider}
+
+import com.fasterxml.jackson.annotation.JsonInclude
+import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature}
+
+/**
+ * This class converts the POJO metric responses into json, using jackson.
+ *
+ * This doesn't follow the standard jersey-jackson plugin options, because we want to stick
+ * with an old version of jersey (since we have it from yarn anyway) and don't want to pull in lots
+ * of dependencies from a new plugin.
+ *
+ * Note that jersey automatically discovers this class based on its package and its annotations.
+ */
+@Provider
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{
+
+ val mapper = new ObjectMapper() {
+ override def writeValueAsString(t: Any): String = {
+ super.writeValueAsString(t)
+ }
+ }
+ mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule)
+ mapper.enable(SerializationFeature.INDENT_OUTPUT)
+ mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL)
+ mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat)
+
+ override def isWriteable(
+ aClass: Class[_],
+ `type`: Type,
+ annotations: Array[Annotation],
+ mediaType: MediaType): Boolean = {
+ true
+ }
+
+ override def writeTo(
+ t: Object,
+ aClass: Class[_],
+ `type`: Type,
+ annotations: Array[Annotation],
+ mediaType: MediaType,
+ multivaluedMap: MultivaluedMap[String, AnyRef],
+ outputStream: OutputStream): Unit = {
+ t match {
+ case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8"))
+ case _ => mapper.writeValue(outputStream, t)
+ }
+ }
+
+ override def getSize(
+ t: Object,
+ aClass: Class[_],
+ `type`: Type,
+ annotations: Array[Annotation],
+ mediaType: MediaType): Long = {
+ -1L
+ }
+}
+
+private[spark] object JacksonMessageWriter {
+ def makeISODateFormat: SimpleDateFormat = {
+ val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'")
+ val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT"))
+ iso8601.setCalendar(cal)
+ iso8601
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
new file mode 100644
index 0000000000000..c3ec45f54681b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.servlet.ServletContext
+import javax.ws.rs._
+import javax.ws.rs.core.{Context, Response}
+
+import com.sun.jersey.api.core.ResourceConfig
+import com.sun.jersey.spi.container.servlet.ServletContainer
+import org.eclipse.jetty.server.handler.ContextHandler
+import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.ui.SparkUI
+
+/**
+ * Main entry point for serving spark application metrics as json, using JAX-RS.
+ *
+ * Each resource should have endpoints that return **public** classes defined in api.scala. Mima
+ * binary compatibility checks ensure that we don't inadvertently make changes that break the api.
+ * The returned objects are automatically converted to json by jackson with JacksonMessageWriter.
+ * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden
+ * files". Any changes and additions should be reflected there as well -- see the notes in
+ * HistoryServerSuite.
+ */
+@Path("/v1")
+private[v1] class JsonRootResource extends UIRootFromServletContext {
+
+ @Path("applications")
+ def getApplicationList(): ApplicationListResource = {
+ new ApplicationListResource(uiRoot)
+ }
+
+ @Path("applications/{appId}")
+ def getApplication(): OneApplicationResource = {
+ new OneApplicationResource(uiRoot)
+ }
+
+ @Path("applications/{appId}/{attemptId}/jobs")
+ def getJobs(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): AllJobsResource = {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new AllJobsResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/jobs")
+ def getJobs(@PathParam("appId") appId: String): AllJobsResource = {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new AllJobsResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/jobs/{jobId: \\d+}")
+ def getJob(@PathParam("appId") appId: String): OneJobResource = {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new OneJobResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}")
+ def getJob(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): OneJobResource = {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new OneJobResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/executors")
+ def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new ExecutorListResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/{attemptId}/executors")
+ def getExecutors(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): ExecutorListResource = {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new ExecutorListResource(ui)
+ }
+ }
+
+
+ @Path("applications/{appId}/stages")
+ def getStages(@PathParam("appId") appId: String): AllStagesResource= {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new AllStagesResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/{attemptId}/stages")
+ def getStages(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): AllStagesResource= {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new AllStagesResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/stages/{stageId: \\d+}")
+ def getStage(@PathParam("appId") appId: String): OneStageResource= {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new OneStageResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}")
+ def getStage(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): OneStageResource = {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new OneStageResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/storage/rdd")
+ def getRdds(@PathParam("appId") appId: String): AllRDDResource = {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new AllRDDResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/{attemptId}/storage/rdd")
+ def getRdds(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): AllRDDResource = {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new AllRDDResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/storage/rdd/{rddId: \\d+}")
+ def getRdd(@PathParam("appId") appId: String): OneRDDResource = {
+ uiRoot.withSparkUI(appId, None) { ui =>
+ new OneRDDResource(ui)
+ }
+ }
+
+ @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}")
+ def getRdd(
+ @PathParam("appId") appId: String,
+ @PathParam("attemptId") attemptId: String): OneRDDResource = {
+ uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+ new OneRDDResource(ui)
+ }
+ }
+
+}
+
+private[spark] object JsonRootResource {
+
+ def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = {
+ val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS)
+ jerseyContext.setContextPath("/json")
+ val holder:ServletHolder = new ServletHolder(classOf[ServletContainer])
+ holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+ "com.sun.jersey.api.core.PackagesResourceConfig")
+ holder.setInitParameter("com.sun.jersey.config.property.packages",
+ "org.apache.spark.status.api.v1")
+ holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS,
+ classOf[SecurityFilter].getCanonicalName)
+ UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot)
+ jerseyContext.addServlet(holder, "/*")
+ jerseyContext
+ }
+}
+
+/**
+ * This trait is shared by the all the root containers for application UI information --
+ * the HistoryServer, the Master UI, and the application UI. This provides the common
+ * interface needed for them all to expose application info as json.
+ */
+private[spark] trait UIRoot {
+ def getSparkUI(appKey: String): Option[SparkUI]
+ def getApplicationInfoList: Iterator[ApplicationInfo]
+
+ /**
+ * Get the spark UI with the given appID, and apply a function
+ * to it. If there is no such app, throw an appropriate exception
+ */
+ def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = {
+ val appKey = attemptId.map(appId + "/" + _).getOrElse(appId)
+ getSparkUI(appKey) match {
+ case Some(ui) =>
+ f(ui)
+ case None => throw new NotFoundException("no such app: " + appId)
+ }
+ }
+ def securityManager: SecurityManager
+}
+
+private[v1] object UIRootFromServletContext {
+
+ private val attribute = getClass.getCanonicalName
+
+ def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = {
+ contextHandler.setAttribute(attribute, uiRoot)
+ }
+
+ def getUiRoot(context: ServletContext): UIRoot = {
+ context.getAttribute(attribute).asInstanceOf[UIRoot]
+ }
+}
+
+private[v1] trait UIRootFromServletContext {
+ @Context
+ var servletContext: ServletContext = _
+
+ def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext)
+}
+
+private[v1] class NotFoundException(msg: String) extends WebApplicationException(
+ new NoSuchElementException(msg),
+ Response
+ .status(Response.Status.NOT_FOUND)
+ .entity(ErrorWrapper(msg))
+ .build()
+)
+
+private[v1] class BadParameterException(msg: String) extends WebApplicationException(
+ new IllegalArgumentException(msg),
+ Response
+ .status(Response.Status.BAD_REQUEST)
+ .entity(ErrorWrapper(msg))
+ .build()
+) {
+ def this(param: String, exp: String, actual: String) = {
+ this(raw"""Bad value for parameter "$param". Expected a $exp, got "$actual"""")
+ }
+}
+
+/**
+ * Signal to JacksonMessageWriter to not convert the message into json (which would result in an
+ * extra set of quotes).
+ */
+private[v1] case class ErrorWrapper(s: String)
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
new file mode 100644
index 0000000000000..b5ef72649e295
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.core.MediaType
+import javax.ws.rs.{Produces, PathParam, GET}
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneApplicationResource(uiRoot: UIRoot) {
+
+ @GET
+ def getApp(@PathParam("appId") appId: String): ApplicationInfo = {
+ val apps = uiRoot.getApplicationInfoList.find { _.id == appId }
+ apps.getOrElse(throw new NotFoundException("unknown app: " + appId))
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
new file mode 100644
index 0000000000000..6d8a60d480aed
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{PathParam, GET, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneJobResource(ui: SparkUI) {
+
+ @GET
+ def oneJob(@PathParam("jobId") jobId: Int): JobData = {
+ val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
+ AllJobsResource.getStatusToJobs(ui)
+ val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId}
+ jobOpt.map { job =>
+ AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
+ }.getOrElse {
+ throw new NotFoundException("unknown job: " + jobId)
+ }
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala
new file mode 100644
index 0000000000000..07b224fac4786
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{PathParam, GET, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.SparkUI
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneRDDResource(ui: SparkUI) {
+
+ @GET
+ def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo = {
+ AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, true).getOrElse(
+ throw new NotFoundException(s"no rdd found w/ id $rddId")
+ )
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala
new file mode 100644
index 0000000000000..fd24aea63a8a1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.status.api.v1.StageStatus._
+import org.apache.spark.status.api.v1.TaskSorting._
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(ui: SparkUI) {
+
+ @GET
+ @Path("")
+ def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = {
+ withStage(stageId){ stageAttempts =>
+ stageAttempts.map { stage =>
+ AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui,
+ includeDetails = true)
+ }
+ }
+ }
+
+ @GET
+ @Path("/{stageAttemptId: \\d+}")
+ def oneAttemptData(
+ @PathParam("stageId") stageId: Int,
+ @PathParam("stageAttemptId") stageAttemptId: Int): StageData = {
+ withStageAttempt(stageId, stageAttemptId) { stage =>
+ AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui,
+ includeDetails = true)
+ }
+ }
+
+ @GET
+ @Path("/{stageAttemptId: \\d+}/taskSummary")
+ def taskSummary(
+ @PathParam("stageId") stageId: Int,
+ @PathParam("stageAttemptId") stageAttemptId: Int,
+ @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String)
+ : TaskMetricDistributions = {
+ withStageAttempt(stageId, stageAttemptId) { stage =>
+ val quantiles = quantileString.split(",").map { s =>
+ try {
+ s.toDouble
+ } catch {
+ case nfe: NumberFormatException =>
+ throw new BadParameterException("quantiles", "double", s)
+ }
+ }
+ AllStagesResource.taskMetricDistributions(stage.ui.taskData.values, quantiles)
+ }
+ }
+
+ @GET
+ @Path("/{stageAttemptId: \\d+}/taskList")
+ def taskList(
+ @PathParam("stageId") stageId: Int,
+ @PathParam("stageAttemptId") stageAttemptId: Int,
+ @DefaultValue("0") @QueryParam("offset") offset: Int,
+ @DefaultValue("20") @QueryParam("length") length: Int,
+ @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = {
+ withStageAttempt(stageId, stageAttemptId) { stage =>
+ val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq
+ .sorted(OneStageResource.ordering(sortBy))
+ tasks.slice(offset, offset + length)
+ }
+ }
+
+ private case class StageStatusInfoUi(status: StageStatus, info: StageInfo, ui: StageUIData)
+
+ private def withStage[T](stageId: Int)(f: Seq[StageStatusInfoUi] => T): T = {
+ val stageAttempts = findStageStatusUIData(ui.jobProgressListener, stageId)
+ if (stageAttempts.isEmpty) {
+ throw new NotFoundException("unknown stage: " + stageId)
+ } else {
+ f(stageAttempts)
+ }
+ }
+
+ private def findStageStatusUIData(
+ listener: JobProgressListener,
+ stageId: Int): Seq[StageStatusInfoUi] = {
+ listener.synchronized {
+ def getStatusInfoUi(status: StageStatus, infos: Seq[StageInfo]): Seq[StageStatusInfoUi] = {
+ infos.filter { _.stageId == stageId }.map { info =>
+ val ui = listener.stageIdToData.getOrElse((info.stageId, info.attemptId),
+ // this is an internal error -- we should always have uiData
+ throw new SparkException(
+ s"no stage ui data found for stage: ${info.stageId}:${info.attemptId}")
+ )
+ StageStatusInfoUi(status, info, ui)
+ }
+ }
+ getStatusInfoUi(ACTIVE, listener.activeStages.values.toSeq) ++
+ getStatusInfoUi(COMPLETE, listener.completedStages) ++
+ getStatusInfoUi(FAILED, listener.failedStages) ++
+ getStatusInfoUi(PENDING, listener.pendingStages.values.toSeq)
+ }
+ }
+
+ private def withStageAttempt[T](
+ stageId: Int,
+ stageAttemptId: Int)
+ (f: StageStatusInfoUi => T): T = {
+ withStage(stageId) { attempts =>
+ val oneAttempt = attempts.find { stage => stage.info.attemptId == stageAttemptId }
+ oneAttempt match {
+ case Some(stage) =>
+ f(stage)
+ case None =>
+ val stageAttempts = attempts.map { _.info.attemptId }
+ throw new NotFoundException(s"unknown attempt for stage $stageId. " +
+ s"Found attempts: ${stageAttempts.mkString("[", ",", "]")}")
+ }
+ }
+ }
+}
+
+object OneStageResource {
+ def ordering(taskSorting: TaskSorting): Ordering[TaskData] = {
+ val extractor: (TaskData => Long) = td =>
+ taskSorting match {
+ case ID => td.taskId
+ case INCREASING_RUNTIME => td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L)
+ case DECREASING_RUNTIME => -td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L)
+ }
+ Ordering.by(extractor)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala
new file mode 100644
index 0000000000000..95fbd96ade5ab
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+
+import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter}
+
+private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext {
+ def filter(req: ContainerRequest): ContainerRequest = {
+ val user = Option(req.getUserPrincipal).map { _.getName }.orNull
+ if (uiRoot.securityManager.checkUIViewPermissions(user)) {
+ req
+ } else {
+ throw new WebApplicationException(
+ Response
+ .status(Response.Status.FORBIDDEN)
+ .entity(raw"""user "$user"is not authorized""")
+ .build()
+ )
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
new file mode 100644
index 0000000000000..cee29786c3019
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.text.SimpleDateFormat
+import java.util.TimeZone
+import javax.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+import javax.ws.rs.core.Response.Status
+
+import scala.util.Try
+
+private[v1] class SimpleDateParam(val originalValue: String) {
+ val timestamp: Long = {
+ SimpleDateParam.formats.collectFirst {
+ case fmt if Try(fmt.parse(originalValue)).isSuccess =>
+ fmt.parse(originalValue).getTime()
+ }.getOrElse(
+ throw new WebApplicationException(
+ Response
+ .status(Status.BAD_REQUEST)
+ .entity("Couldn't parse date: " + originalValue)
+ .build()
+ )
+ )
+ }
+}
+
+private[v1] object SimpleDateParam {
+
+ val formats: Seq[SimpleDateFormat] = {
+
+ val gmtDay = new SimpleDateFormat("yyyy-MM-dd")
+ gmtDay.setTimeZone(TimeZone.getTimeZone("GMT"))
+
+ Seq(
+ new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz"),
+ gmtDay
+ )
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
new file mode 100644
index 0000000000000..ef3c8570d8186
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.Map
+
+import org.apache.spark.JobExecutionStatus
+
+class ApplicationInfo private[spark](
+ val id: String,
+ val name: String,
+ val attempts: Seq[ApplicationAttemptInfo])
+
+class ApplicationAttemptInfo private[spark](
+ val attemptId: Option[String],
+ val startTime: Date,
+ val endTime: Date,
+ val sparkUser: String,
+ val completed: Boolean = false)
+
+class ExecutorStageSummary private[spark](
+ val taskTime : Long,
+ val failedTasks : Int,
+ val succeededTasks : Int,
+ val inputBytes : Long,
+ val outputBytes : Long,
+ val shuffleRead : Long,
+ val shuffleWrite : Long,
+ val memoryBytesSpilled : Long,
+ val diskBytesSpilled : Long)
+
+class ExecutorSummary private[spark](
+ val id: String,
+ val hostPort: String,
+ val rddBlocks: Int,
+ val memoryUsed: Long,
+ val diskUsed: Long,
+ val activeTasks: Int,
+ val failedTasks: Int,
+ val completedTasks: Int,
+ val totalTasks: Int,
+ val totalDuration: Long,
+ val totalInputBytes: Long,
+ val totalShuffleRead: Long,
+ val totalShuffleWrite: Long,
+ val maxMemory: Long,
+ val executorLogs: Map[String, String])
+
+class JobData private[spark](
+ val jobId: Int,
+ val name: String,
+ val description: Option[String],
+ val submissionTime: Option[Date],
+ val completionTime: Option[Date],
+ val stageIds: Seq[Int],
+ val jobGroup: Option[String],
+ val status: JobExecutionStatus,
+ val numTasks: Int,
+ val numActiveTasks: Int,
+ val numCompletedTasks: Int,
+ val numSkippedTasks: Int,
+ val numFailedTasks: Int,
+ val numActiveStages: Int,
+ val numCompletedStages: Int,
+ val numSkippedStages: Int,
+ val numFailedStages: Int)
+
+// Q: should Tachyon size go in here as well? currently the UI only shows it on the overall storage
+// page ... does anybody pay attention to it?
+class RDDStorageInfo private[spark](
+ val id: Int,
+ val name: String,
+ val numPartitions: Int,
+ val numCachedPartitions: Int,
+ val storageLevel: String,
+ val memoryUsed: Long,
+ val diskUsed: Long,
+ val dataDistribution: Option[Seq[RDDDataDistribution]],
+ val partitions: Option[Seq[RDDPartitionInfo]])
+
+class RDDDataDistribution private[spark](
+ val address: String,
+ val memoryUsed: Long,
+ val memoryRemaining: Long,
+ val diskUsed: Long)
+
+class RDDPartitionInfo private[spark](
+ val blockName: String,
+ val storageLevel: String,
+ val memoryUsed: Long,
+ val diskUsed: Long,
+ val executors: Seq[String])
+
+class StageData private[spark](
+ val status: StageStatus,
+ val stageId: Int,
+ val attemptId: Int,
+ val numActiveTasks: Int ,
+ val numCompleteTasks: Int,
+ val numFailedTasks: Int,
+
+ val executorRunTime: Long,
+
+ val inputBytes: Long,
+ val inputRecords: Long,
+ val outputBytes: Long,
+ val outputRecords: Long,
+ val shuffleReadBytes: Long,
+ val shuffleReadRecords: Long,
+ val shuffleWriteBytes: Long,
+ val shuffleWriteRecords: Long,
+ val memoryBytesSpilled: Long,
+ val diskBytesSpilled: Long,
+
+ val name: String,
+ val details: String,
+ val schedulingPool: String,
+
+ val accumulatorUpdates: Seq[AccumulableInfo],
+ val tasks: Option[Map[Long, TaskData]],
+ val executorSummary:Option[Map[String,ExecutorStageSummary]])
+
+class TaskData private[spark](
+ val taskId: Long,
+ val index: Int,
+ val attempt: Int,
+ val launchTime: Date,
+ val executorId: String,
+ val host: String,
+ val taskLocality: String,
+ val speculative: Boolean,
+ val accumulatorUpdates: Seq[AccumulableInfo],
+ val errorMessage: Option[String] = None,
+ val taskMetrics: Option[TaskMetrics] = None)
+
+class TaskMetrics private[spark](
+ val executorDeserializeTime: Long,
+ val executorRunTime: Long,
+ val resultSize: Long,
+ val jvmGcTime: Long,
+ val resultSerializationTime: Long,
+ val memoryBytesSpilled: Long,
+ val diskBytesSpilled: Long,
+ val inputMetrics: Option[InputMetrics],
+ val outputMetrics: Option[OutputMetrics],
+ val shuffleReadMetrics: Option[ShuffleReadMetrics],
+ val shuffleWriteMetrics: Option[ShuffleWriteMetrics])
+
+class InputMetrics private[spark](
+ val bytesRead: Long,
+ val recordsRead: Long)
+
+class OutputMetrics private[spark](
+ val bytesWritten: Long,
+ val recordsWritten: Long)
+
+class ShuffleReadMetrics private[spark](
+ val remoteBlocksFetched: Int,
+ val localBlocksFetched: Int,
+ val fetchWaitTime: Long,
+ val remoteBytesRead: Long,
+ val totalBlocksFetched: Int,
+ val recordsRead: Long)
+
+class ShuffleWriteMetrics private[spark](
+ val bytesWritten: Long,
+ val writeTime: Long,
+ val recordsWritten: Long)
+
+class TaskMetricDistributions private[spark](
+ val quantiles: IndexedSeq[Double],
+
+ val executorDeserializeTime: IndexedSeq[Double],
+ val executorRunTime: IndexedSeq[Double],
+ val resultSize: IndexedSeq[Double],
+ val jvmGcTime: IndexedSeq[Double],
+ val resultSerializationTime: IndexedSeq[Double],
+ val memoryBytesSpilled: IndexedSeq[Double],
+ val diskBytesSpilled: IndexedSeq[Double],
+
+ val inputMetrics: Option[InputMetricDistributions],
+ val outputMetrics: Option[OutputMetricDistributions],
+ val shuffleReadMetrics: Option[ShuffleReadMetricDistributions],
+ val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions])
+
+class InputMetricDistributions private[spark](
+ val bytesRead: IndexedSeq[Double],
+ val recordsRead: IndexedSeq[Double])
+
+class OutputMetricDistributions private[spark](
+ val bytesWritten: IndexedSeq[Double],
+ val recordsWritten: IndexedSeq[Double])
+
+class ShuffleReadMetricDistributions private[spark](
+ val readBytes: IndexedSeq[Double],
+ val readRecords: IndexedSeq[Double],
+ val remoteBlocksFetched: IndexedSeq[Double],
+ val localBlocksFetched: IndexedSeq[Double],
+ val fetchWaitTime: IndexedSeq[Double],
+ val remoteBytesRead: IndexedSeq[Double],
+ val totalBlocksFetched: IndexedSeq[Double])
+
+class ShuffleWriteMetricDistributions private[spark](
+ val writeBytes: IndexedSeq[Double],
+ val writeRecords: IndexedSeq[Double],
+ val writeTime: IndexedSeq[Double])
+
+class AccumulableInfo private[spark](
+ val id: Long,
+ val name: String,
+ val update: Option[String],
+ val value: String)
diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
index ad53a3edc7cc1..96062626b5045 100644
--- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
+++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
@@ -18,7 +18,7 @@
package org.apache.spark.storage
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{RDDOperationScope, RDD}
import org.apache.spark.util.Utils
@DeveloperApi
@@ -26,7 +26,9 @@ class RDDInfo(
val id: Int,
val name: String,
val numPartitions: Int,
- var storageLevel: StorageLevel)
+ var storageLevel: StorageLevel,
+ val parentIds: Seq[Int],
+ val scope: Option[RDDOperationScope] = None)
extends Ordered[RDDInfo] {
var numCachedPartitions = 0
@@ -52,7 +54,8 @@ class RDDInfo(
private[spark] object RDDInfo {
def fromRdd(rdd: RDD[_]): RDDInfo = {
- val rddName = Option(rdd.name).getOrElse(rdd.id.toString)
- new RDDInfo(rdd.id, rddName, rdd.partitions.length, rdd.getStorageLevel)
+ val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd))
+ val parentIds = rdd.dependencies.map(_.rdd.id)
+ new RDDInfo(rdd.id, rddName, rdd.partitions.length, rdd.getStorageLevel, parentIds, rdd.scope)
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
index 7d75929b96f75..ec711480ebf30 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
@@ -25,13 +25,17 @@ import org.apache.spark.scheduler._
/**
* :: DeveloperApi ::
* A SparkListener that maintains executor storage status.
+ *
+ * This class is thread-safe (unlike JobProgressListener)
*/
@DeveloperApi
class StorageStatusListener extends SparkListener {
// This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE)
private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]()
- def storageStatusList: Seq[StorageStatus] = executorIdToStorageStatus.values.toSeq
+ def storageStatusList: Seq[StorageStatus] = synchronized {
+ executorIdToStorageStatus.values.toSeq
+ }
/** Update storage status list to reflect updated block statuses */
private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) {
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 06fce86bd38d2..bfe4a180e8a6f 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -17,6 +17,9 @@
package org.apache.spark.ui
+import java.util.Date
+
+import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot}
import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
import org.apache.spark.scheduler._
import org.apache.spark.storage.StorageStatusListener
@@ -25,6 +28,7 @@ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab}
import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab}
import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab}
import org.apache.spark.ui.storage.{StorageListener, StorageTab}
+import org.apache.spark.ui.scope.RDDOperationGraphListener
/**
* Top level user interface for a Spark application.
@@ -32,29 +36,35 @@ import org.apache.spark.ui.storage.{StorageListener, StorageTab}
private[spark] class SparkUI private (
val sc: Option[SparkContext],
val conf: SparkConf,
- val securityManager: SecurityManager,
+ securityManager: SecurityManager,
val environmentListener: EnvironmentListener,
val storageStatusListener: StorageStatusListener,
val executorsListener: ExecutorsListener,
val jobProgressListener: JobProgressListener,
val storageListener: StorageListener,
+ val operationGraphListener: RDDOperationGraphListener,
var appName: String,
- val basePath: String)
+ val basePath: String,
+ val startTime: Long)
extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI")
- with Logging {
+ with Logging
+ with UIRoot {
val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false)
+
+ val stagesTab = new StagesTab(this)
+
/** Initialize all components of the server. */
def initialize() {
attachTab(new JobsTab(this))
- val stagesTab = new StagesTab(this)
attachTab(stagesTab)
attachTab(new StorageTab(this))
attachTab(new EnvironmentTab(this))
attachTab(new ExecutorsTab(this))
attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath))
+ attachHandler(JsonRootResource.getJsonServlet(this))
// This should be POST only, but, the YARN AM proxy won't proxy POSTs
attachHandler(createRedirectHandler(
"/stages/stage/kill", "/stages", stagesTab.handleKillRequest,
@@ -81,6 +91,24 @@ private[spark] class SparkUI private (
private[spark] def appUIHostPort = publicHostName + ":" + boundPort
private[spark] def appUIAddress = s"http://$appUIHostPort"
+
+ def getSparkUI(appId: String): Option[SparkUI] = {
+ if (appId == appName) Some(this) else None
+ }
+
+ def getApplicationInfoList: Iterator[ApplicationInfo] = {
+ Iterator(new ApplicationInfo(
+ id = appName,
+ name = appName,
+ attempts = Seq(new ApplicationAttemptInfo(
+ attemptId = None,
+ startTime = new Date(startTime),
+ endTime = new Date(-1),
+ sparkUser = "",
+ completed = false
+ ))
+ ))
+ }
}
private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String)
@@ -93,6 +121,9 @@ private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String)
private[spark] object SparkUI {
val DEFAULT_PORT = 4040
val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
+ val DEFAULT_POOL_NAME = "default"
+ val DEFAULT_RETAINED_STAGES = 1000
+ val DEFAULT_RETAINED_JOBS = 1000
def getUIPort(conf: SparkConf): Int = {
conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT)
@@ -104,9 +135,10 @@ private[spark] object SparkUI {
listenerBus: SparkListenerBus,
jobProgressListener: JobProgressListener,
securityManager: SecurityManager,
- appName: String): SparkUI = {
+ appName: String,
+ startTime: Long): SparkUI = {
create(Some(sc), conf, listenerBus, securityManager, appName,
- jobProgressListener = Some(jobProgressListener))
+ jobProgressListener = Some(jobProgressListener), startTime = startTime)
}
def createHistoryUI(
@@ -114,8 +146,9 @@ private[spark] object SparkUI {
listenerBus: SparkListenerBus,
securityManager: SecurityManager,
appName: String,
- basePath: String): SparkUI = {
- create(None, conf, listenerBus, securityManager, appName, basePath)
+ basePath: String,
+ startTime: Long): SparkUI = {
+ create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime)
}
/**
@@ -132,7 +165,8 @@ private[spark] object SparkUI {
securityManager: SecurityManager,
appName: String,
basePath: String = "",
- jobProgressListener: Option[JobProgressListener] = None): SparkUI = {
+ jobProgressListener: Option[JobProgressListener] = None,
+ startTime: Long): SparkUI = {
val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse {
val listener = new JobProgressListener(conf)
@@ -144,13 +178,16 @@ private[spark] object SparkUI {
val storageStatusListener = new StorageStatusListener
val executorsListener = new ExecutorsListener(storageStatusListener)
val storageListener = new StorageListener(storageStatusListener)
+ val operationGraphListener = new RDDOperationGraphListener(conf)
listenerBus.addListener(environmentListener)
listenerBus.addListener(storageStatusListener)
listenerBus.addListener(executorsListener)
listenerBus.addListener(storageListener)
+ listenerBus.addListener(operationGraphListener)
new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener,
- executorsListener, _jobProgressListener, storageListener, appName, basePath)
+ executorsListener, _jobProgressListener, storageListener, operationGraphListener,
+ appName, basePath, startTime)
}
}
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 395af2ea30b9d..2f3fb181e4026 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -23,6 +23,7 @@ import java.util.{Locale, Date}
import scala.xml.{Node, Text}
import org.apache.spark.Logging
+import org.apache.spark.ui.scope.RDDOperationGraph
/** Utility functions for generating XML pages with spark content. */
private[spark] object UIUtils extends Logging {
@@ -172,13 +173,21 @@ private[spark] object UIUtils extends Logging {
}
+ def vizHeaderNodes: Seq[Node] = {
+
+
+
+
+ }
+
/** Returns a spark page with correctly formatted headers */
def headerSparkPage(
title: String,
content: => Seq[Node],
activeTab: SparkUITab,
refreshInterval: Option[Int] = None,
- helpText: Option[String] = None): Seq[Node] = {
+ helpText: Option[String] = None,
+ showVisualization: Boolean = false): Seq[Node] = {
val appName = activeTab.appName
val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..."
@@ -196,6 +205,7 @@ private[spark] object UIUtils extends Logging {
{commonHeaderNodes}
+ {if (showVisualization) vizHeaderNodes else Seq.empty}
{appName} - {title}
@@ -320,4 +330,47 @@ private[spark] object UIUtils extends Logging {
}
+
+ /** Return a "DAG visualization" DOM element that expands into a visualization for a stage. */
+ def showDagVizForStage(stageId: Int, graph: Option[RDDOperationGraph]): Seq[Node] = {
+ showDagViz(graph.toSeq, forJob = false)
+ }
+
+ /** Return a "DAG visualization" DOM element that expands into a visualization for a job. */
+ def showDagVizForJob(jobId: Int, graphs: Seq[RDDOperationGraph]): Seq[Node] = {
+ showDagViz(graphs, forJob = true)
+ }
+
+ /**
+ * Return a "DAG visualization" DOM element that expands into a visualization on the UI.
+ *
+ * This populates metadata necessary for generating the visualization on the front-end in
+ * a format that is expected by spark-dag-viz.js. Any changes in the format here must be
+ * reflected there.
+ */
+ private def showDagViz(graphs: Seq[RDDOperationGraph], forJob: Boolean): Seq[Node] = {
+
+
+
+ DAG visualization
+
+
+
+
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index f9860d1a5ce76..384f2ad26e281 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -37,7 +37,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf}
* pages. The use of tabs is optional, however; a WebUI may choose to include pages directly.
*/
private[spark] abstract class WebUI(
- securityManager: SecurityManager,
+ val securityManager: SecurityManager,
port: Int,
conf: SparkConf,
basePath: String = "",
@@ -77,15 +77,9 @@ private[spark] abstract class WebUI(
val pagePath = "/" + page.prefix
val renderHandler = createServletHandler(pagePath,
(request: HttpServletRequest) => page.render(request), securityManager, basePath)
- val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json",
- (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)
attachHandler(renderHandler)
- attachHandler(renderJsonHandler)
pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
.append(renderHandler)
- pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
- .append(renderJsonHandler)
-
}
/** Attach a handler to this UI. */
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
index 956608d7c0cbe..b247e4cdc3bd4 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
@@ -22,11 +22,11 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
+import org.apache.spark.status.api.v1.ExecutorSummary
import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage}
import org.apache.spark.util.Utils
-/** Summary information about an executor to display in the UI. */
-// Needs to be private[ui] because of a false positive MiMa failure.
+// This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive
private[ui] case class ExecutorSummaryInfo(
id: String,
hostPort: String,
@@ -44,6 +44,7 @@ private[ui] case class ExecutorSummaryInfo(
maxMemory: Long,
executorLogs: Map[String, String])
+
private[ui] class ExecutorsPage(
parent: ExecutorsTab,
threadDumpEnabled: Boolean)
@@ -55,7 +56,8 @@ private[ui] class ExecutorsPage(
val maxMem = storageStatusList.map(_.maxMem).sum
val memUsed = storageStatusList.map(_.memUsed).sum
val diskUsed = storageStatusList.map(_.diskUsed).sum
- val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId)
+ val execInfo = for (statusId <- 0 until storageStatusList.size) yield
+ ExecutorsPage.getExecInfo(listener, statusId)
val execInfoSorted = execInfo.sortBy(_.id)
val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty
@@ -111,7 +113,7 @@ private[ui] class ExecutorsPage(
}
/** Render an HTML row representing an executor */
- private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = {
+ private def execRow(info: ExecutorSummary, logsExist: Boolean): Seq[Node] = {
val maximumMemory = info.maxMemory
val memoryUsed = info.memoryUsed
val diskUsed = info.diskUsed
@@ -170,8 +172,11 @@ private[ui] class ExecutorsPage(
}
+}
+
+private[spark] object ExecutorsPage {
/** Represent an executor's info as a map given a storage status index */
- private def getExecInfo(statusId: Int): ExecutorSummaryInfo = {
+ def getExecInfo(listener: ExecutorsListener, statusId: Int): ExecutorSummary = {
val status = listener.storageStatusList(statusId)
val execId = status.blockManagerId.executorId
val hostPort = status.blockManagerId.hostPort
@@ -189,7 +194,7 @@ private[ui] class ExecutorsPage(
val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L)
val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty)
- new ExecutorSummaryInfo(
+ new ExecutorSummary(
execId,
hostPort,
rddBlocks,
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index a7ea12b1655fe..09323d1d80ad6 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -179,7 +179,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
- Event Timeline
+ Event timeline
++
@@ -271,6 +271,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
val shouldShowCompletedJobs = completedJobs.nonEmpty
val shouldShowFailedJobs = failedJobs.nonEmpty
+ val completedJobNumStr = if (completedJobs.size == listener.numCompletedJobs) {
+ s"${completedJobs.size}"
+ } else {
+ s"${listener.numCompletedJobs}, only showing ${completedJobs.size}"
+ }
+
val summary: NodeSeq =
@@ -295,9 +301,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
}
{
if (shouldShowCompletedJobs) {
-
+
Completed Jobs:
- {completedJobs.size}
+ {completedJobNumStr}
}
}
@@ -305,7 +311,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
if (shouldShowFailedJobs) {
Failed Jobs:
- {failedJobs.size}
+ {listener.numFailedJobs}
}
}
@@ -322,7 +328,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
activeJobsTable
}
if (shouldShowCompletedJobs) {
- content ++= Completed Jobs ({completedJobs.size}) ++
+ content ++= Completed Jobs ({completedJobNumStr}) ++
completedJobsTable
}
if (shouldShowFailedJobs) {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
index 527f960af2dfc..a37f739ab9c66 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
@@ -27,7 +27,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils}
/** Page showing list of all ongoing and recently finished stages and pools */
private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
private val sc = parent.sc
- private val listener = parent.listener
+ private val listener = parent.progressListener
private def isFairScheduler = parent.isFairScheduler
def render(request: HttpServletRequest): Seq[Node] = {
@@ -42,18 +42,18 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
val activeStagesTable =
new StageTableBase(activeStages.sortBy(_.submissionTime).reverse,
- parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler,
+ parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler,
killEnabled = parent.killEnabled)
val pendingStagesTable =
new StageTableBase(pendingStages.sortBy(_.submissionTime).reverse,
- parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler,
+ parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler,
killEnabled = false)
val completedStagesTable =
new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath,
- parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false)
+ parent.progressListener, isFairScheduler = parent.isFairScheduler, killEnabled = false)
val failedStagesTable =
new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath,
- parent.listener, isFairScheduler = parent.isFairScheduler)
+ parent.progressListener, isFairScheduler = parent.isFairScheduler)
// For now, pool information is only accessible in live UIs
val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable])
@@ -64,6 +64,12 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
val shouldShowCompletedStages = completedStages.nonEmpty
val shouldShowFailedStages = failedStages.nonEmpty
+ val completedStageNumStr = if (numCompletedStages == completedStages.size) {
+ s"$numCompletedStages"
+ } else {
+ s"$numCompletedStages, only showing ${completedStages.size}"
+ }
+
val summary: NodeSeq =
@@ -98,9 +104,9 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
}
{
if (shouldShowCompletedStages) {
-
+
Completed Stages:
- {numCompletedStages}
+ {completedStageNumStr}
}
}
@@ -132,7 +138,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
pendingStagesTable.toNodeSeq
}
if (shouldShowCompletedStages) {
- content ++= Completed Stages ({numCompletedStages}) ++
+ content ++= Completed Stages ({completedStageNumStr}) ++
completedStagesTable.toNodeSeq
}
if (shouldShowFailedStages) {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
index 1f8536d1b7195..d5cdbfac104f8 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
@@ -26,7 +26,7 @@ import org.apache.spark.util.Utils
/** Stage summary grouped by executors. */
private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) {
- private val listener = parent.listener
+ private val listener = parent.progressListener
def toNodeSeq: Seq[Node] = {
listener.synchronized {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index dd968e124738e..7163217e1fed0 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -31,6 +31,7 @@ import org.apache.spark.ui.jobs.UIData.ExecutorUIData
/** Page showing statistics and stage list for a given job */
private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
+
private val STAGES_LEGEND =
- Event Timeline
+ Event timeline
++
@@ -186,7 +187,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
val jobDataOption = listener.jobIdToData.get(jobId)
if (jobDataOption.isEmpty) {
val content =
-
+
No information to display for job {jobId}
return UIUtils.headerSparkPage(
@@ -198,7 +199,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
// This could be empty if the JobProgressListener hasn't received information about the
// stage or if the stage information has been garbage collected
listener.stageIdToInfo.getOrElse(stageId,
- new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown"))
+ new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown"))
}
val activeStages = Buffer[StageInfo]()
@@ -303,9 +304,14 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
var content = summary
val appStartTime = listener.startTime
val executorListener = parent.executorListener
+ val operationGraphListener = parent.operationGraphListener
+
content ++= makeTimeline(activeStages ++ completedStages ++ failedStages,
executorListener.executorIdToData, appStartTime)
+ content ++= UIUtils.showDagVizForJob(
+ jobId, operationGraphListener.getOperationGraphForJob(jobId))
+
if (shouldShowActiveStages) {
content ++=
Active Stages ({activeStages.size}) ++
activeStagesTable.toNodeSeq
@@ -326,7 +332,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
content ++=
Failed Stages ({failedStages.size}) ++
failedStagesTable.toNodeSeq
}
- UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent)
+ UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent, showVisualization = true)
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index d6d716de25920..246e191d64776 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -25,6 +25,7 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.ui.SparkUI
import org.apache.spark.ui.jobs.UIData._
/**
@@ -38,8 +39,6 @@ import org.apache.spark.ui.jobs.UIData._
@DeveloperApi
class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
- import JobProgressListener._
-
// Define a handful of type aliases so that data structures' types can serve as documentation.
// These type aliases are public because they're used in the types of public fields:
@@ -75,6 +74,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
// JobProgressListener's retention limits.
var numCompletedStages = 0
var numFailedStages = 0
+ var numCompletedJobs = 0
+ var numFailedJobs = 0
// Misc:
val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]()
@@ -86,8 +87,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
// To limit the total memory usage of JobProgressListener, we only track information for a fixed
// number of non-active jobs and stages (there is no limit for active jobs and stages):
- val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES)
- val retainedJobs = conf.getInt("spark.ui.retainedJobs", DEFAULT_RETAINED_JOBS)
+ val retainedStages = conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES)
+ val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS)
// We can test for memory leaks by ensuring that collections that track non-active jobs and
// stages do not grow without bound and that collections for active jobs/stages eventually become
@@ -218,10 +219,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
completedJobs += jobData
trimJobsIfNecessary(completedJobs)
jobData.status = JobExecutionStatus.SUCCEEDED
+ numCompletedJobs += 1
case JobFailed(exception) =>
failedJobs += jobData
trimJobsIfNecessary(failedJobs)
jobData.status = JobExecutionStatus.FAILED
+ numFailedJobs += 1
}
for (stageId <- jobData.stageIds) {
stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage =>
@@ -288,8 +291,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
activeStages(stage.stageId) = stage
pendingStages.remove(stage.stageId)
val poolName = Option(stageSubmitted.properties).map {
- p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME)
- }.getOrElse(DEFAULT_POOL_NAME)
+ p => p.getProperty("spark.scheduler.pool", SparkUI.DEFAULT_POOL_NAME)
+ }.getOrElse(SparkUI.DEFAULT_POOL_NAME)
stageIdToInfo(stage.stageId) = stage
val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData)
@@ -524,9 +527,3 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
startTime = appStarted.time
}
}
-
-private object JobProgressListener {
- val DEFAULT_POOL_NAME = "default"
- val DEFAULT_RETAINED_STAGES = 1000
- val DEFAULT_RETAINED_JOBS = 1000
-}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
index 342787f1a3c2d..77ca60b000a9b 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
@@ -24,10 +24,12 @@ import org.apache.spark.ui.{SparkUI, SparkUITab}
private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") {
val sc = parent.sc
val killEnabled = parent.killEnabled
- def isFairScheduler: Boolean =
- jobProgresslistener.schedulingMode.exists(_ == SchedulingMode.FAIR)
val jobProgresslistener = parent.jobProgressListener
val executorListener = parent.executorsListener
+ val operationGraphListener = parent.operationGraphListener
+
+ def isFairScheduler: Boolean =
+ jobProgresslistener.schedulingMode.exists(_ == SchedulingMode.FAIR)
attachPage(new AllJobsPage(this))
attachPage(new JobPage(this))
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index f47cdc935e539..f3e0b38523f32 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
@@ -21,13 +21,13 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import org.apache.spark.scheduler.{Schedulable, StageInfo}
+import org.apache.spark.scheduler.StageInfo
import org.apache.spark.ui.{WebUIPage, UIUtils}
/** Page showing specific pool details */
private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") {
private val sc = parent.sc
- private val listener = parent.listener
+ private val listener = parent.progressListener
def render(request: HttpServletRequest): Seq[Node] = {
listener.synchronized {
@@ -40,7 +40,7 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") {
case None => Seq[StageInfo]()
}
val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse,
- parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler,
+ parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler,
killEnabled = parent.killEnabled)
// For now, pool information is only accessible in live UIs
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
index df1899e7a9b84..9ba2af54dacf4 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
@@ -25,7 +25,7 @@ import org.apache.spark.ui.UIUtils
/** Table showing list of pools */
private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) {
- private val listener = parent.listener
+ private val listener = parent.progressListener
def toNodeSeq: Seq[Node] = {
listener.synchronized {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 797c9404bc449..89d175b06b947 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -25,17 +25,19 @@ import scala.xml.{Elem, Node, Unparsed}
import org.apache.commons.lang3.StringEscapeUtils
import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo}
import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils}
import org.apache.spark.ui.jobs.UIData._
+import org.apache.spark.ui.scope.RDDOperationGraph
import org.apache.spark.util.{Utils, Distribution}
-import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo}
/** Page showing statistics and task list for a given stage */
private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
- private val listener = parent.listener
+ private val progressListener = parent.progressListener
+ private val operationGraphListener = parent.operationGraphListener
def render(request: HttpServletRequest): Seq[Node] = {
- listener.synchronized {
+ progressListener.synchronized {
val parameterId = request.getParameter("id")
require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
@@ -44,23 +46,31 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
val stageId = parameterId.toInt
val stageAttemptId = parameterAttempt.toInt
- val stageDataOption = listener.stageIdToData.get((stageId, stageAttemptId))
+ val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId))
- if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) {
+ val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)"
+ if (stageDataOption.isEmpty) {
+ val content =
+
+
No information to display for Stage {stageId} (Attempt {stageAttemptId})
+
+ return UIUtils.headerSparkPage(stageHeader, content, parent)
+
+ }
+ if (stageDataOption.get.taskData.isEmpty) {
val content =
Summary Metrics No tasks have started yet
Tasks No tasks have started yet
- return UIUtils.headerSparkPage(
- s"Details for Stage $stageId (Attempt $stageAttemptId)", content, parent)
+ return UIUtils.headerSparkPage(stageHeader, content, parent)
}
val stageData = stageDataOption.get
val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime)
val numCompleted = tasks.count(_.taskInfo.finished)
- val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables
+ val accumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables
val hasAccumulators = accumulables.size > 0
val summary =
@@ -169,6 +179,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
+ val dagViz = UIUtils.showDagVizForStage(
+ stageId, operationGraphListener.getOperationGraphForStage(stageId))
+
val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
def accumulableRow(acc: AccumulableInfo): Elem =
{acc.name} {acc.value}
@@ -434,13 +447,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
val content =
summary ++
showAdditionalMetrics ++
+ dagViz ++
Summary Metrics for {numCompleted} Completed Tasks ++
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++
Aggregated Metrics by Executor ++ executorTable.toNodeSeq ++
maybeAccumulableTable ++
Tasks ++ taskTable
- UIUtils.headerSparkPage("Details for Stage %d".format(stageId), content, parent)
+ UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true)
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
index 1bd2d87e00796..55169956d8304 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
@@ -26,19 +26,20 @@ import org.apache.spark.ui.{SparkUI, SparkUITab}
private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") {
val sc = parent.sc
val killEnabled = parent.killEnabled
- val listener = parent.jobProgressListener
+ val progressListener = parent.jobProgressListener
+ val operationGraphListener = parent.operationGraphListener
attachPage(new AllStagesPage(this))
attachPage(new StagePage(this))
attachPage(new PoolPage(this))
- def isFairScheduler: Boolean = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+ def isFairScheduler: Boolean = progressListener.schedulingMode.exists(_ == SchedulingMode.FAIR)
def handleKillRequest(request: HttpServletRequest): Unit = {
if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) {
val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean
val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt
- if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) {
+ if (stageId >= 0 && killFlag && progressListener.activeStages.contains(stageId)) {
sc.get.cancelStage(stageId)
}
// Do a quick pause here to give Spark time to kill the stage so it shows up as
diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
new file mode 100644
index 0000000000000..a18c193540ce3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ui.scope
+
+import scala.collection.mutable
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.Logging
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * A representation of a generic cluster graph used for storing information on RDD operations.
+ *
+ * Each graph is defined with a set of edges and a root cluster, which may contain children
+ * nodes and children clusters. Additionally, a graph may also have edges that enter or exit
+ * the graph from nodes that belong to adjacent graphs.
+ */
+private[ui] case class RDDOperationGraph(
+ edges: Seq[RDDOperationEdge],
+ outgoingEdges: Seq[RDDOperationEdge],
+ incomingEdges: Seq[RDDOperationEdge],
+ rootCluster: RDDOperationCluster)
+
+/** A node in an RDDOperationGraph. This represents an RDD. */
+private[ui] case class RDDOperationNode(id: Int, name: String, cached: Boolean)
+
+/**
+ * A directed edge connecting two nodes in an RDDOperationGraph.
+ * This represents an RDD dependency.
+ */
+private[ui] case class RDDOperationEdge(fromId: Int, toId: Int)
+
+/**
+ * A cluster that groups nodes together in an RDDOperationGraph.
+ *
+ * This represents any grouping of RDDs, including operation scopes (e.g. textFile, flatMap),
+ * stages, jobs, or any higher level construct. A cluster may be nested inside of other clusters.
+ */
+private[ui] class RDDOperationCluster(val id: String, val name: String) {
+ private val _childNodes = new ListBuffer[RDDOperationNode]
+ private val _childClusters = new ListBuffer[RDDOperationCluster]
+
+ def childNodes: Seq[RDDOperationNode] = _childNodes.iterator.toSeq
+ def childClusters: Seq[RDDOperationCluster] = _childClusters.iterator.toSeq
+ def attachChildNode(childNode: RDDOperationNode): Unit = { _childNodes += childNode }
+ def attachChildCluster(childCluster: RDDOperationCluster): Unit = {
+ _childClusters += childCluster
+ }
+
+ /** Return all the nodes container in this cluster, including ones nested in other clusters. */
+ def getAllNodes: Seq[RDDOperationNode] = {
+ _childNodes ++ _childClusters.flatMap(_.childNodes)
+ }
+}
+
+private[ui] object RDDOperationGraph extends Logging {
+
+ /**
+ * Construct a RDDOperationGraph for a given stage.
+ *
+ * The root cluster represents the stage, and all children clusters represent RDD operations.
+ * Each node represents an RDD, and each edge represents a dependency between two RDDs pointing
+ * from the parent to the child.
+ *
+ * This does not currently merge common operation scopes across stages. This may be worth
+ * supporting in the future if we decide to group certain stages within the same job under
+ * a common scope (e.g. part of a SQL query).
+ */
+ def makeOperationGraph(stage: StageInfo): RDDOperationGraph = {
+ val edges = new ListBuffer[RDDOperationEdge]
+ val nodes = new mutable.HashMap[Int, RDDOperationNode]
+ val clusters = new mutable.HashMap[String, RDDOperationCluster] // indexed by cluster ID
+
+ // Root cluster is the stage cluster
+ val stageClusterId = s"stage_${stage.stageId}"
+ val stageClusterName = s"Stage ${stage.stageId}" +
+ { if (stage.attemptId == 0) "" else s" (attempt ${stage.attemptId})" }
+ val rootCluster = new RDDOperationCluster(stageClusterId, stageClusterName)
+
+ // Find nodes, edges, and operation scopes that belong to this stage
+ stage.rddInfos.foreach { rdd =>
+ edges ++= rdd.parentIds.map { parentId => RDDOperationEdge(parentId, rdd.id) }
+
+ // TODO: differentiate between the intention to cache an RDD and whether it's actually cached
+ val node = nodes.getOrElseUpdate(
+ rdd.id, RDDOperationNode(rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE))
+
+ if (rdd.scope == null) {
+ // This RDD has no encompassing scope, so we put it directly in the root cluster
+ // This should happen only if an RDD is instantiated outside of a public RDD API
+ rootCluster.attachChildNode(node)
+ } else {
+ // Otherwise, this RDD belongs to an inner cluster,
+ // which may be nested inside of other clusters
+ val rddScopes = rdd.scope.map { scope => scope.getAllScopes }.getOrElse(Seq.empty)
+ val rddClusters = rddScopes.map { scope =>
+ val clusterId = scope.name + "_" + scope.id
+ val clusterName = scope.name
+ clusters.getOrElseUpdate(clusterId, new RDDOperationCluster(clusterId, clusterName))
+ }
+ // Build the cluster hierarchy for this RDD
+ rddClusters.sliding(2).foreach { pc =>
+ if (pc.size == 2) {
+ val parentCluster = pc(0)
+ val childCluster = pc(1)
+ parentCluster.attachChildCluster(childCluster)
+ }
+ }
+ // Attach the outermost cluster to the root cluster, and the RDD to the innermost cluster
+ rddClusters.headOption.foreach { cluster => rootCluster.attachChildCluster(cluster) }
+ rddClusters.lastOption.foreach { cluster => cluster.attachChildNode(node) }
+ }
+ }
+
+ // Classify each edge as internal, outgoing or incoming
+ // This information is needed to reason about how stages relate to each other
+ val internalEdges = new ListBuffer[RDDOperationEdge]
+ val outgoingEdges = new ListBuffer[RDDOperationEdge]
+ val incomingEdges = new ListBuffer[RDDOperationEdge]
+ edges.foreach { case e: RDDOperationEdge =>
+ val fromThisGraph = nodes.contains(e.fromId)
+ val toThisGraph = nodes.contains(e.toId)
+ (fromThisGraph, toThisGraph) match {
+ case (true, true) => internalEdges += e
+ case (true, false) => outgoingEdges += e
+ case (false, true) => incomingEdges += e
+ // should never happen
+ case _ => logWarning(s"Found an orphan edge in stage ${stage.stageId}: $e")
+ }
+ }
+
+ RDDOperationGraph(internalEdges, outgoingEdges, incomingEdges, rootCluster)
+ }
+
+ /**
+ * Generate the content of a dot file that describes the specified graph.
+ *
+ * Note that this only uses a minimal subset of features available to the DOT specification.
+ * Part of the styling must be done here because the rendering library must take certain
+ * attributes into account when arranging the graph elements. More style is added in the
+ * visualization later through post-processing in JavaScript.
+ *
+ * For the complete DOT specification, see http://www.graphviz.org/Documentation/dotguide.pdf.
+ */
+ def makeDotFile(graph: RDDOperationGraph, forJob: Boolean): String = {
+ val dotFile = new StringBuilder
+ dotFile.append("digraph G {\n")
+ dotFile.append(makeDotSubgraph(graph.rootCluster, forJob, indent = " "))
+ graph.edges.foreach { edge =>
+ dotFile.append(s""" ${edge.fromId}->${edge.toId} [lineInterpolate="basis"];\n""")
+ }
+ dotFile.append("}")
+ val result = dotFile.toString()
+ logDebug(result)
+ result
+ }
+
+ /**
+ * Return the dot representation of a node in an RDDOperationGraph.
+ *
+ * On the job page, is displayed as a small circle without labels.
+ * On the stage page, it is displayed as a box with an embedded label.
+ */
+ private def makeDotNode(node: RDDOperationNode, forJob: Boolean): String = {
+ if (forJob) {
+ s"""${node.id} [label=" " shape="circle" padding="5" labelStyle="font-size: 0"]"""
+ } else {
+ s"""${node.id} [label="${node.name} (${node.id})"]"""
+ }
+ }
+
+ /** Return the dot representation of a subgraph in an RDDOperationGraph. */
+ private def makeDotSubgraph(
+ scope: RDDOperationCluster,
+ forJob: Boolean,
+ indent: String): String = {
+ val subgraph = new StringBuilder
+ subgraph.append(indent + s"subgraph cluster${scope.id} {\n")
+ subgraph.append(indent + s""" label="${scope.name}";\n""")
+ scope.childNodes.foreach { node =>
+ subgraph.append(indent + s" ${makeDotNode(node, forJob)};\n")
+ }
+ scope.childClusters.foreach { cscope =>
+ subgraph.append(makeDotSubgraph(cscope, forJob, indent + " "))
+ }
+ subgraph.append(indent + "}\n")
+ subgraph.toString()
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala
new file mode 100644
index 0000000000000..2884a49f31122
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ui.scope
+
+import scala.collection.mutable
+
+import org.apache.spark.SparkConf
+import org.apache.spark.scheduler._
+import org.apache.spark.ui.SparkUI
+
+/**
+ * A SparkListener that constructs a DAG of RDD operations.
+ */
+private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListener {
+ private val jobIdToStageIds = new mutable.HashMap[Int, Seq[Int]]
+ private val stageIdToGraph = new mutable.HashMap[Int, RDDOperationGraph]
+ private val stageIds = new mutable.ArrayBuffer[Int]
+
+ // How many jobs or stages to retain graph metadata for
+ private val retainedStages =
+ conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES)
+
+ /** Return the graph metadata for the given stage, or None if no such information exists. */
+ def getOperationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = {
+ jobIdToStageIds.get(jobId)
+ .map { sids => sids.flatMap { sid => stageIdToGraph.get(sid) } }
+ .getOrElse { Seq.empty }
+ }
+
+ /** Return the graph metadata for the given stage, or None if no such information exists. */
+ def getOperationGraphForStage(stageId: Int): Option[RDDOperationGraph] = {
+ stageIdToGraph.get(stageId)
+ }
+
+ /** On job start, construct a RDDOperationGraph for each stage in the job for display later. */
+ override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized {
+ val jobId = jobStart.jobId
+ val stageInfos = jobStart.stageInfos
+
+ stageInfos.foreach { stageInfo =>
+ stageIds += stageInfo.stageId
+ stageIdToGraph(stageInfo.stageId) = RDDOperationGraph.makeOperationGraph(stageInfo)
+ }
+ jobIdToStageIds(jobId) = stageInfos.map(_.stageId).sorted
+
+ // Remove graph metadata for old stages
+ if (stageIds.size >= retainedStages) {
+ val toRemove = math.max(retainedStages / 10, 1)
+ stageIds.take(toRemove).foreach { id => stageIdToGraph.remove(id) }
+ stageIds.trimStart(toRemove)
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 199f731b92bcc..05f94a7507f4f 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -21,8 +21,8 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils}
-import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.status.api.v1.{AllRDDResource, RDDDataDistribution, RDDPartitionInfo}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
import org.apache.spark.util.Utils
/** Page showing storage details for a given RDD */
@@ -32,28 +32,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
def render(request: HttpServletRequest): Seq[Node] = {
val parameterId = request.getParameter("id")
require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
-
val rddId = parameterId.toInt
- val storageStatusList = listener.storageStatusList
- val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse {
- // Rather than crashing, render an "RDD Not Found" page
- return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent)
- }
+ val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener,includeDetails = true)
+ .getOrElse {
+ // Rather than crashing, render an "RDD Not Found" page
+ return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent)
+ }
// Worker table
- val workers = storageStatusList.map((rddId, _))
- val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers,
- id = Some("rdd-storage-by-worker-table"))
+ val workerTable = UIUtils.listingTable(workerHeader, workerRow,
+ rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table"))
// Block table
- val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList)
- val blocks = storageStatusList
- .flatMap(_.rddBlocksById(rddId))
- .sortWith(_._1.name < _._1.name)
- .map { case (blockId, status) =>
- (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown")))
- }
- val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks,
+ val blockTable = UIUtils.listingTable(blockHeader, blockRow, rddStorageInfo.partitions.get,
id = Some("rdd-storage-by-block-table"))
val content =
@@ -62,23 +53,23 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
Storage Level:
- {rddInfo.storageLevel.description}
+ {rddStorageInfo.storageLevel}
Cached Partitions:
- {rddInfo.numCachedPartitions}
+ {rddStorageInfo.numCachedPartitions}
Total Partitions:
- {rddInfo.numPartitions}
+ {rddStorageInfo.numPartitions}
Memory Size:
- {Utils.bytesToString(rddInfo.memSize)}
+ {Utils.bytesToString(rddStorageInfo.memoryUsed)}
Disk Size:
- {Utils.bytesToString(rddInfo.diskSize)}
+ {Utils.bytesToString(rddStorageInfo.diskUsed)}
@@ -86,19 +77,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
-
Data Distribution on {workers.size} Executors
+ Data Distribution on {rddStorageInfo.dataDistribution.size} Executors
{workerTable}
-
{blocks.size} Partitions
+ {rddStorageInfo.partitions.size} Partitions
{blockTable}
;
- UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent)
+ UIUtils.headerSparkPage("RDD Storage Info for " + rddStorageInfo.name, content, parent)
}
/** Header fields for the worker table */
@@ -116,34 +107,32 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
"Executors")
/** Render an HTML row representing a worker */
- private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = {
- val (rddId, status) = worker
+ private def workerRow(worker: RDDDataDistribution): Seq[Node] = {
- {status.blockManagerId.host + ":" + status.blockManagerId.port}
+ {worker.address}
- {Utils.bytesToString(status.memUsedByRdd(rddId))}
- ({Utils.bytesToString(status.memRemaining)} Remaining)
+ {Utils.bytesToString(worker.memoryUsed)}
+ ({Utils.bytesToString(worker.memoryRemaining)} Remaining)
- {Utils.bytesToString(status.diskUsedByRdd(rddId))}
+ {Utils.bytesToString(worker.diskUsed)}
}
/** Render an HTML row representing a block */
- private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = {
- val (id, block, locations) = row
+ private def blockRow(row: RDDPartitionInfo): Seq[Node] = {
- {id}
+ {row.blockName}
- {block.storageLevel.description}
+ {row.storageLevel}
-
- {Utils.bytesToString(block.memSize)}
+
+ {Utils.bytesToString(row.memoryUsed)}
-
- {Utils.bytesToString(block.diskSize)}
+
+ {Utils.bytesToString(row.diskUsed)}
- {locations.map(l => {l} )}
+ {row.executors.map(l => {l} )}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
index 59dc6b547c1d8..07db783c572cf 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
@@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
import org.apache.spark.storage.RDDInfo
-import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
import org.apache.spark.util.Utils
/** Page showing list of RDD's currently stored in the cluster */
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
index 045bd784990d1..0351749700962 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
@@ -35,6 +35,8 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag
/**
* :: DeveloperApi ::
* A SparkListener that prepares information to be displayed on the BlockManagerUI.
+ *
+ * This class is thread-safe (unlike JobProgressListener)
*/
@DeveloperApi
class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener {
@@ -43,7 +45,9 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar
def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList
/** Filter RDD info to include only those with cached partitions */
- def rddInfoList: Seq[RDDInfo] = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq
+ def rddInfoList: Seq[RDDInfo] = synchronized {
+ _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq
+ }
/** Update the storage info of the RDDs whose blocks are among the given updated blocks */
private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index b725df3b44596..de3316d083a22 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -183,7 +183,9 @@ private[spark] object AkkaUtils extends Logging {
lastException = e
logWarning(s"Error sending message [message = $message] in $attempts attempts", e)
}
- Thread.sleep(retryInterval)
+ if (attempts < maxAttempts) {
+ Thread.sleep(retryInterval)
+ }
}
throw new SparkException(
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index 4ac0382d80815..6fe32e469c732 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -179,6 +179,11 @@ private[spark] object ClosureCleaner extends Logging {
cleanTransitively: Boolean,
accessedFields: Map[Class[_], Set[String]]): Unit = {
+ if (!isClosure(func.getClass)) {
+ logWarning("Expected a closure; got " + func.getClass.getName)
+ return
+ }
+
// TODO: clean all inner closures first. This requires us to find the inner objects.
// TODO: cache outerClasses / innerClasses / accessedFields
@@ -312,7 +317,9 @@ private[spark] object ClosureCleaner extends Logging {
private def ensureSerializable(func: AnyRef) {
try {
- SparkEnv.get.closureSerializer.newInstance().serialize(func)
+ if (SparkEnv.get != null) {
+ SparkEnv.get.closureSerializer.newInstance().serialize(func)
+ }
} catch {
case ex: Exception => throw new SparkException("Task not serializable", ex)
}
@@ -347,6 +354,9 @@ private[spark] object ClosureCleaner extends Logging {
}
}
+private[spark] class ReturnStatementInClosureException
+ extends SparkException("Return statements aren't allowed in Spark closures")
+
private class ReturnStatementFinder extends ClassVisitor(ASM4) {
override def visitMethod(access: Int, name: String, desc: String,
sig: String, exceptions: Array[String]): MethodVisitor = {
@@ -354,7 +364,7 @@ private class ReturnStatementFinder extends ClassVisitor(ASM4) {
new MethodVisitor(ASM4) {
override def visitTypeInsn(op: Int, tp: String) {
if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) {
- throw new SparkException("Return statements aren't allowed in Spark closures")
+ throw new ReturnStatementInClosureException
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index ee02fbd9ce069..3f162d1f6c3eb 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -28,10 +28,11 @@ import org.json4s.DefaultFormats
import org.json4s.JsonDSL._
import org.json4s.JsonAST._
+import org.apache.spark._
import org.apache.spark.executor._
+import org.apache.spark.rdd.RDDOperationScope
import org.apache.spark.scheduler._
import org.apache.spark.storage._
-import org.apache.spark._
/**
* Serializes SparkListener events to/from JSON. This protocol provides strong backwards-
@@ -228,6 +229,7 @@ private[spark] object JsonProtocol {
def stageInfoToJson(stageInfo: StageInfo): JValue = {
val rddInfo = JArray(stageInfo.rddInfos.map(rddInfoToJson).toList)
+ val parentIds = JArray(stageInfo.parentIds.map(JInt(_)).toList)
val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing)
val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing)
val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing)
@@ -236,6 +238,7 @@ private[spark] object JsonProtocol {
("Stage Name" -> stageInfo.name) ~
("Number of Tasks" -> stageInfo.numTasks) ~
("RDD Info" -> rddInfo) ~
+ ("Parent IDs" -> parentIds) ~
("Details" -> stageInfo.details) ~
("Submission Time" -> submissionTime) ~
("Completion Time" -> completionTime) ~
@@ -368,8 +371,11 @@ private[spark] object JsonProtocol {
def rddInfoToJson(rddInfo: RDDInfo): JValue = {
val storageLevel = storageLevelToJson(rddInfo.storageLevel)
+ val parentIds = JArray(rddInfo.parentIds.map(JInt(_)).toList)
("RDD ID" -> rddInfo.id) ~
("Name" -> rddInfo.name) ~
+ ("Scope" -> rddInfo.scope.map(_.toJson)) ~
+ ("Parent IDs" -> parentIds) ~
("Storage Level" -> storageLevel) ~
("Number of Partitions" -> rddInfo.numPartitions) ~
("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~
@@ -519,7 +525,7 @@ private[spark] object JsonProtocol {
// The "Stage Infos" field was added in Spark 1.2.0
val stageInfos = Utils.jsonOption(json \ "Stage Infos")
.map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse {
- stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown"))
+ stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown"))
}
SparkListenerJobStart(jobId, submissionTime, stageInfos, properties)
}
@@ -599,7 +605,10 @@ private[spark] object JsonProtocol {
val attemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0)
val stageName = (json \ "Stage Name").extract[String]
val numTasks = (json \ "Number of Tasks").extract[Int]
- val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson(_))
+ val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson)
+ val parentIds = Utils.jsonOption(json \ "Parent IDs")
+ .map { l => l.extract[List[JValue]].map(_.extract[Int]) }
+ .getOrElse(Seq.empty)
val details = (json \ "Details").extractOpt[String].getOrElse("")
val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long])
val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long])
@@ -609,7 +618,8 @@ private[spark] object JsonProtocol {
case None => Seq[AccumulableInfo]()
}
- val stageInfo = new StageInfo(stageId, attemptId, stageName, numTasks, rddInfos, details)
+ val stageInfo = new StageInfo(
+ stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details)
stageInfo.submissionTime = submissionTime
stageInfo.completionTime = completionTime
stageInfo.failureReason = failureReason
@@ -785,16 +795,22 @@ private[spark] object JsonProtocol {
def rddInfoFromJson(json: JValue): RDDInfo = {
val rddId = (json \ "RDD ID").extract[Int]
val name = (json \ "Name").extract[String]
+ val scope = Utils.jsonOption(json \ "Scope")
+ .map(_.extract[String])
+ .map(RDDOperationScope.fromJson)
+ val parentIds = Utils.jsonOption(json \ "Parent IDs")
+ .map { l => l.extract[List[JValue]].map(_.extract[Int]) }
+ .getOrElse(Seq.empty)
val storageLevel = storageLevelFromJson(json \ "Storage Level")
val numPartitions = (json \ "Number of Partitions").extract[Int]
val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int]
val memSize = (json \ "Memory Size").extract[Long]
- // fallback to tachyon for backward compatability
+ // fallback to tachyon for backward compatibility
val externalBlockStoreSize = (json \ "ExternalBlockStore Size").toSome
.getOrElse(json \ "Tachyon Size").extract[Long]
val diskSize = (json \ "Disk Size").extract[Long]
- val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel)
+ val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel, parentIds, scope)
rddInfo.numCachedPartitions = numCachedPartitions
rddInfo.memSize = memSize
rddInfo.externalBlockStoreSize = externalBlockStoreSize
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation
new file mode 100644
index 0000000000000..6101177e4bb85
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation
@@ -0,0 +1,53 @@
+[ {
+ "id" : "local-1427397477963",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2015-03-26T19:17:57.184GMT",
+ "endTime" : "2015-03-26T19:20:02.949GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ } ]
+}, {
+ "id" : "local-1426533911241",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "attemptId" : "2",
+ "startTime" : "2015-03-17T23:11:50.242GMT",
+ "endTime" : "2015-03-17T23:12:25.177GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ }, {
+ "attemptId" : "1",
+ "startTime" : "2015-03-16T19:25:10.242GMT",
+ "endTime" : "2015-03-16T19:25:45.177GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ } ]
+}, {
+ "id" : "local-1425081759269",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2015-02-28T00:02:38.277GMT",
+ "endTime" : "2015-02-28T00:02:46.912GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ } ]
+}, {
+ "id" : "local-1422981780767",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2015-02-03T16:42:59.720GMT",
+ "endTime" : "2015-02-03T16:43:08.731GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ } ]
+}, {
+ "id" : "local-1422981759269",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2015-02-03T16:42:38.277GMT",
+ "endTime" : "2015-02-03T16:42:46.912GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ } ]
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation
new file mode 100644
index 0000000000000..cb622e147249e
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation
@@ -0,0 +1,17 @@
+[ {
+ "id" : "",
+ "hostPort" : "localhost:57971",
+ "rddBlocks" : 8,
+ "memoryUsed" : 28000128,
+ "diskUsed" : 0,
+ "activeTasks" : 0,
+ "failedTasks" : 1,
+ "completedTasks" : 31,
+ "totalTasks" : 32,
+ "totalDuration" : 8820,
+ "totalInputBytes" : 28000288,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 13180,
+ "maxMemory" : 278302556,
+ "executorLogs" : { }
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation
new file mode 100644
index 0000000000000..4a29072bdb6e4
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation
@@ -0,0 +1,15 @@
+{
+ "jobId" : 0,
+ "name" : "count at :15",
+ "stageIds" : [ 0 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+}
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation
new file mode 100644
index 0000000000000..cab4750270dfa
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation
@@ -0,0 +1,43 @@
+[ {
+ "jobId" : 2,
+ "name" : "count at :17",
+ "stageIds" : [ 3 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+}, {
+ "jobId" : 1,
+ "name" : "count at :20",
+ "stageIds" : [ 1, 2 ],
+ "status" : "FAILED",
+ "numTasks" : 16,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 15,
+ "numSkippedTasks" : 15,
+ "numFailedTasks" : 1,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 1
+}, {
+ "jobId" : 0,
+ "name" : "count at :15",
+ "stageIds" : [ 0 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation
new file mode 100644
index 0000000000000..cab4750270dfa
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation
@@ -0,0 +1,43 @@
+[ {
+ "jobId" : 2,
+ "name" : "count at :17",
+ "stageIds" : [ 3 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+}, {
+ "jobId" : 1,
+ "name" : "count at :20",
+ "stageIds" : [ 1, 2 ],
+ "status" : "FAILED",
+ "numTasks" : 16,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 15,
+ "numSkippedTasks" : 15,
+ "numFailedTasks" : 1,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 1
+}, {
+ "jobId" : 0,
+ "name" : "count at :15",
+ "stageIds" : [ 0 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation
new file mode 100644
index 0000000000000..6fd25befbf7e8
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation
@@ -0,0 +1,29 @@
+[ {
+ "jobId" : 2,
+ "name" : "count at :17",
+ "stageIds" : [ 3 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+}, {
+ "jobId" : 0,
+ "name" : "count at :15",
+ "stageIds" : [ 0 ],
+ "status" : "SUCCEEDED",
+ "numTasks" : 8,
+ "numActiveTasks" : 0,
+ "numCompletedTasks" : 8,
+ "numSkippedTasks" : 8,
+ "numFailedTasks" : 0,
+ "numActiveStages" : 0,
+ "numCompletedStages" : 1,
+ "numSkippedStages" : 0,
+ "numFailedStages" : 0
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation
new file mode 100644
index 0000000000000..07489ad96414a
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation
@@ -0,0 +1,10 @@
+{
+ "id" : "local-1422981780767",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2015-02-03T16:42:59.720GMT",
+ "endTime" : "2015-02-03T16:43:08.731GMT",
+ "sparkUser" : "irashid",
+ "completed" : true
+ } ]
+}
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation
new file mode 100644
index 0000000000000..111cb8163eb3d
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation
@@ -0,0 +1,270 @@
+{
+ "status" : "COMPLETE",
+ "stageId" : 1,
+ "attemptId" : 0,
+ "numActiveTasks" : 0,
+ "numCompleteTasks" : 8,
+ "numFailedTasks" : 0,
+ "executorRunTime" : 3476,
+ "inputBytes" : 28000128,
+ "inputRecords" : 0,
+ "outputBytes" : 0,
+ "outputRecords" : 0,
+ "shuffleReadBytes" : 0,
+ "shuffleReadRecords" : 0,
+ "shuffleWriteBytes" : 13180,
+ "shuffleWriteRecords" : 0,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "name" : "map at :14",
+ "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+ "schedulingPool" : "default",
+ "accumulatorUpdates" : [ ],
+ "tasks" : {
+ "8" : {
+ "taskId" : 8,
+ "index" : 0,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.829GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 1,
+ "executorRunTime" : 435,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 2,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 94000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "11" : {
+ "taskId" : 11,
+ "index" : 3,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.830GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1647,
+ "writeTime" : 83000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "14" : {
+ "taskId" : 14,
+ "index" : 6,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.832GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 88000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "13" : {
+ "taskId" : 13,
+ "index" : 5,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.831GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 2,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 73000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "10" : {
+ "taskId" : 10,
+ "index" : 2,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.830GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 76000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "9" : {
+ "taskId" : 9,
+ "index" : 1,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.830GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 1,
+ "executorRunTime" : 436,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 0,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 98000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "12" : {
+ "taskId" : 12,
+ "index" : 4,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.831GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1645,
+ "writeTime" : 101000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "15" : {
+ "taskId" : 15,
+ "index" : 7,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.833GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 1,
+ "executorRunTime" : 435,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 79000,
+ "recordsWritten" : 0
+ }
+ }
+ }
+ },
+ "executorSummary" : {
+ "" : {
+ "taskTime" : 3624,
+ "failedTasks" : 0,
+ "succeededTasks" : 8,
+ "inputBytes" : 28000128,
+ "outputBytes" : 0,
+ "shuffleRead" : 0,
+ "shuffleWrite" : 13180,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0
+ }
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation
new file mode 100644
index 0000000000000..ef339f89afa45
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation
@@ -0,0 +1,270 @@
+[ {
+ "status" : "COMPLETE",
+ "stageId" : 1,
+ "attemptId" : 0,
+ "numActiveTasks" : 0,
+ "numCompleteTasks" : 8,
+ "numFailedTasks" : 0,
+ "executorRunTime" : 3476,
+ "inputBytes" : 28000128,
+ "inputRecords" : 0,
+ "outputBytes" : 0,
+ "outputRecords" : 0,
+ "shuffleReadBytes" : 0,
+ "shuffleReadRecords" : 0,
+ "shuffleWriteBytes" : 13180,
+ "shuffleWriteRecords" : 0,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "name" : "map at :14",
+ "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+ "schedulingPool" : "default",
+ "accumulatorUpdates" : [ ],
+ "tasks" : {
+ "8" : {
+ "taskId" : 8,
+ "index" : 0,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.829GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 1,
+ "executorRunTime" : 435,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 2,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 94000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "11" : {
+ "taskId" : 11,
+ "index" : 3,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.830GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1647,
+ "writeTime" : 83000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "14" : {
+ "taskId" : 14,
+ "index" : 6,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.832GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 88000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "13" : {
+ "taskId" : 13,
+ "index" : 5,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.831GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 2,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 73000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "10" : {
+ "taskId" : 10,
+ "index" : 2,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.830GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 76000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "9" : {
+ "taskId" : 9,
+ "index" : 1,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.830GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 1,
+ "executorRunTime" : 436,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 0,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 98000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "12" : {
+ "taskId" : 12,
+ "index" : 4,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.831GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 2,
+ "executorRunTime" : 434,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1645,
+ "writeTime" : 101000,
+ "recordsWritten" : 0
+ }
+ }
+ },
+ "15" : {
+ "taskId" : 15,
+ "index" : 7,
+ "attempt" : 0,
+ "launchTime" : "2015-02-03T16:43:05.833GMT",
+ "executorId" : "",
+ "host" : "localhost",
+ "taskLocality" : "PROCESS_LOCAL",
+ "speculative" : false,
+ "accumulatorUpdates" : [ ],
+ "taskMetrics" : {
+ "executorDeserializeTime" : 1,
+ "executorRunTime" : 435,
+ "resultSize" : 1902,
+ "jvmGcTime" : 19,
+ "resultSerializationTime" : 1,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "inputMetrics" : {
+ "bytesRead" : 3500016,
+ "recordsRead" : 0
+ },
+ "shuffleWriteMetrics" : {
+ "bytesWritten" : 1648,
+ "writeTime" : 79000,
+ "recordsWritten" : 0
+ }
+ }
+ }
+ },
+ "executorSummary" : {
+ "" : {
+ "taskTime" : 3624,
+ "failedTasks" : 0,
+ "succeededTasks" : 8,
+ "inputBytes" : 28000128,
+ "outputBytes" : 0,
+ "shuffleRead" : 0,
+ "shuffleWrite" : 13180,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0
+ }
+ }
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation
new file mode 100644
index 0000000000000..056fac7088594
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation
@@ -0,0 +1,89 @@
+[ {
+ "status" : "COMPLETE",
+ "stageId" : 3,
+ "attemptId" : 0,
+ "numActiveTasks" : 0,
+ "numCompleteTasks" : 8,
+ "numFailedTasks" : 0,
+ "executorRunTime" : 162,
+ "inputBytes" : 160,
+ "inputRecords" : 0,
+ "outputBytes" : 0,
+ "outputRecords" : 0,
+ "shuffleReadBytes" : 0,
+ "shuffleReadRecords" : 0,
+ "shuffleWriteBytes" : 0,
+ "shuffleWriteRecords" : 0,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "name" : "count at :17",
+ "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+ "schedulingPool" : "default",
+ "accumulatorUpdates" : [ ]
+}, {
+ "status" : "COMPLETE",
+ "stageId" : 1,
+ "attemptId" : 0,
+ "numActiveTasks" : 0,
+ "numCompleteTasks" : 8,
+ "numFailedTasks" : 0,
+ "executorRunTime" : 3476,
+ "inputBytes" : 28000128,
+ "inputRecords" : 0,
+ "outputBytes" : 0,
+ "outputRecords" : 0,
+ "shuffleReadBytes" : 0,
+ "shuffleReadRecords" : 0,
+ "shuffleWriteBytes" : 13180,
+ "shuffleWriteRecords" : 0,
+ "memoryBytesSpilled" : 0,
+ "diskBytesSpilled" : 0,
+ "name" : "map at :14",
+ "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(