+ * 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..b5f040ceb15ca 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -407,15 +407,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true")
+ // "_jobProgressListener" should be set up before creating SparkEnv because when creating
+ // "SparkEnv", some messages will be posted to "listenerBus" and we should not miss them.
+ _jobProgressListener = new JobProgressListener(_conf)
+ listenerBus.addListener(jobProgressListener)
+
// Create the Spark execution environment (cache, map output tracker, etc)
_env = createSparkEnv(_conf, isLocal, listenerBus)
SparkEnv.set(_env)
_metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf)
- _jobProgressListener = new JobProgressListener(_conf)
- listenerBus.addListener(jobProgressListener)
-
_statusTracker = new SparkStatusTracker(this)
_progressBar =
@@ -537,6 +539,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 +664,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 +682,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 +693,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 +712,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 +747,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 +796,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 +827,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 +871,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 +891,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 +922,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 +945,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 +975,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 +1009,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 +1029,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 +1043,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 +1076,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 +1099,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 +1120,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 +1681,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 +1736,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 +2086,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/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/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/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
index 499dd97c0656a..8bc4e205bc3c6 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -59,7 +59,7 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) extends Ou
def write(key: Any, value: Any)
/**
- * Notify the writer that a record worth of bytes has been written with writeBytes.
+ * Notify the writer that a record worth of bytes has been written with OutputStream#write.
*/
def recordWritten()
@@ -215,12 +215,7 @@ private[spark] class DiskBlockObjectWriter(
objOut.writeKey(key)
objOut.writeValue(value)
- numRecordsWritten += 1
- writeMetrics.incShuffleRecordsWritten(1)
-
- if (numRecordsWritten % 32 == 0) {
- updateBytesWritten()
- }
+ recordWritten()
}
override def write(b: Int): Unit = throw new UnsupportedOperationException()
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/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 06fce86bd38d2..a5271f0574e6c 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -25,6 +25,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.
@@ -38,6 +39,7 @@ private[spark] class SparkUI private (
val executorsListener: ExecutorsListener,
val jobProgressListener: JobProgressListener,
val storageListener: StorageListener,
+ val operationGraphListener: RDDOperationGraphListener,
var appName: String,
val basePath: String)
extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI")
@@ -93,6 +95,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)
@@ -144,13 +149,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)
}
}
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/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index a7ea12b1655fe..f6abf27db49dd 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
++
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..236bc8ea92879 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])
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..96cc3d78d0f15 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 =
|