Skip to content

Commit 75a8d90

Browse files
author
Andrew Or
committed
Merge branch 'master' of github.com:apache/spark into concurrent-sql-executions
Conflicts: core/src/test/scala/org/apache/spark/ThreadingSuite.scala
2 parents fce3819 + 7b6c856 commit 75a8d90

File tree

28 files changed

+813
-170
lines changed

28 files changed

+813
-170
lines changed

core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -368,7 +368,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
368368
val peakExecutionMemory = validTasks.map { case TaskUIData(info, _, _) =>
369369
info.accumulables
370370
.find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY }
371-
.map { acc => acc.value.toLong }
371+
.map { acc => acc.update.getOrElse("0").toLong }
372372
.getOrElse(0L)
373373
.toDouble
374374
}

core/src/test/scala/org/apache/spark/ThreadingSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -147,12 +147,12 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
147147
}.start()
148148
}
149149
sem.acquire(2)
150+
throwable.foreach { t => throw improveStackTrace(t) }
150151
if (ThreadingSuiteState.failed.get()) {
151152
logError("Waited 1 second without seeing runningThreads = 4 (it was " +
152153
ThreadingSuiteState.runningThreads.get() + "); failing test")
153154
fail("One or more threads didn't see runningThreads = 4")
154155
}
155-
throwable.foreach { t => throw improveStackTrace(t) }
156156
}
157157

158158
test("set local properties in different thread") {
@@ -178,8 +178,8 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
178178
threads.foreach(_.start())
179179

180180
sem.acquire(5)
181-
assert(sc.getLocalProperty("test") === null)
182181
throwable.foreach { t => throw improveStackTrace(t) }
182+
assert(sc.getLocalProperty("test") === null)
183183
}
184184

185185
test("set and get local properties in parent-children thread") {
@@ -207,9 +207,9 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
207207
threads.foreach(_.start())
208208

209209
sem.acquire(5)
210+
throwable.foreach { t => throw improveStackTrace(t) }
210211
assert(sc.getLocalProperty("test") === "parent")
211212
assert(sc.getLocalProperty("Foo") === null)
212-
throwable.foreach { t => throw improveStackTrace(t) }
213213
}
214214

215215
test("mutation in parent local property does not affect child (SPARK-10563)") {

core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala

Lines changed: 22 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import scala.xml.Node
2323

2424
import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS}
2525

26-
import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite, Success}
26+
import org.apache.spark._
2727
import org.apache.spark.executor.TaskMetrics
2828
import org.apache.spark.scheduler._
2929
import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab}
@@ -47,6 +47,14 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext {
4747
assert(html3.contains(targetString))
4848
}
4949

50+
test("SPARK-10543: peak execution memory should be per-task rather than cumulative") {
51+
val unsafeConf = "spark.sql.unsafe.enabled"
52+
val conf = new SparkConf(false).set(unsafeConf, "true")
53+
val html = renderStagePage(conf).toString().toLowerCase
54+
// verify min/25/50/75/max show task value not cumulative values
55+
assert(html.contains("<td>10.0 b</td>" * 5))
56+
}
57+
5058
/**
5159
* Render a stage page started with the given conf and return the HTML.
5260
* This also runs a dummy stage to populate the page with useful content.
@@ -67,12 +75,19 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext {
6775

6876
// Simulate a stage in job progress listener
6977
val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details")
70-
val taskInfo = new TaskInfo(0, 0, 0, 0, "0", "localhost", TaskLocality.ANY, false)
71-
jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo))
72-
jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo))
73-
taskInfo.markSuccessful()
74-
jobListener.onTaskEnd(
75-
SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, TaskMetrics.empty))
78+
// Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness
79+
(1 to 2).foreach {
80+
taskId =>
81+
val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false)
82+
val peakExecutionMemory = 10
83+
taskInfo.accumulables += new AccumulableInfo(0, InternalAccumulator.PEAK_EXECUTION_MEMORY,
84+
Some(peakExecutionMemory.toString), (peakExecutionMemory * taskId).toString, true)
85+
jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo))
86+
jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo))
87+
taskInfo.markSuccessful()
88+
jobListener.onTaskEnd(
89+
SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, TaskMetrics.empty))
90+
}
7691
jobListener.onStageCompleted(SparkListenerStageCompleted(stageInfo))
7792
page.render(request)
7893
}

repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,8 @@ object Main extends Logging {
3535
s.processArguments(List("-Yrepl-class-based",
3636
"-Yrepl-outdir", s"${outputDir.getAbsolutePath}",
3737
"-classpath", getAddedJars.mkString(File.pathSeparator)), true)
38-
val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf))
38+
// the creation of SecurityManager has to be lazy so SPARK_YARN_MODE is set if needed
39+
lazy val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf))
3940
var sparkContext: SparkContext = _
4041
var sqlContext: SQLContext = _
4142
var interp = new SparkILoop // this is a public var because tests reset it.

0 commit comments

Comments
 (0)