Skip to content

Commit 15616f4

Browse files
Ngone51dongjoon-hyun
authored andcommitted
[SPARK-33173][CORE][TESTS][FOLLOWUP] Use local[2] and AtomicInteger
### What changes were proposed in this pull request? Use `local[2]` to let tasks launch at the same time. And change counters (`numOnTaskXXX`) to `AtomicInteger` type to ensure thread safe. ### Why are the changes needed? The test is still flaky after the fix #30072. See: https://github.com/apache/spark/pull/30728/checks?check_run_id=1557987642 And it's easy to reproduce if you test it multiple times (e.g. 100) locally. The test sets up a stage with 2 tasks to run on an executor with 1 core. So these 2 tasks have to be launched one by one. The task-2 will be launched after task-1 fails. However, since we don't retry failed task in local mode (MAX_LOCAL_TASK_FAILURES = 1), the stage will abort right away after task-1 fail and cancels the running task-2 at the same time. There's a chance that task-2 gets canceled before calling `PluginContainer.onTaskStart`, which leads to the test failure. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tested manually after the fix and the test is no longer flaky. Closes #30823 from Ngone51/debug-flaky-spark-33088. Authored-by: yi.wu <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent 8c81cf7 commit 15616f4

File tree

1 file changed

+14
-13
lines changed

1 file changed

+14
-13
lines changed

core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala

Lines changed: 14 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.internal.plugin
2020
import java.io.File
2121
import java.nio.charset.StandardCharsets
2222
import java.util.{Map => JMap}
23+
import java.util.concurrent.atomic.AtomicInteger
2324

2425
import scala.collection.JavaConverters._
2526
import scala.concurrent.duration._
@@ -138,15 +139,15 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo
138139
sc = new SparkContext(conf)
139140
sc.parallelize(1 to 10, 2).count()
140141

141-
assert(TestSparkPlugin.executorPlugin.numOnTaskStart == 2)
142-
assert(TestSparkPlugin.executorPlugin.numOnTaskSucceeded == 2)
143-
assert(TestSparkPlugin.executorPlugin.numOnTaskFailed == 0)
142+
assert(TestSparkPlugin.executorPlugin.numOnTaskStart.get() == 2)
143+
assert(TestSparkPlugin.executorPlugin.numOnTaskSucceeded.get() == 2)
144+
assert(TestSparkPlugin.executorPlugin.numOnTaskFailed.get() == 0)
144145
}
145146

146147
test("SPARK-33088: executor failed tasks trigger plugin calls") {
147148
val conf = new SparkConf()
148149
.setAppName(getClass().getName())
149-
.set(SparkLauncher.SPARK_MASTER, "local[1]")
150+
.set(SparkLauncher.SPARK_MASTER, "local[2]")
150151
.set(PLUGINS, Seq(classOf[TestSparkPlugin].getName()))
151152

152153
sc = new SparkContext(conf)
@@ -157,9 +158,9 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo
157158
}
158159

159160
eventually(timeout(10.seconds), interval(100.millis)) {
160-
assert(TestSparkPlugin.executorPlugin.numOnTaskStart == 2)
161-
assert(TestSparkPlugin.executorPlugin.numOnTaskSucceeded == 0)
162-
assert(TestSparkPlugin.executorPlugin.numOnTaskFailed == 2)
161+
assert(TestSparkPlugin.executorPlugin.numOnTaskStart.get() == 2)
162+
assert(TestSparkPlugin.executorPlugin.numOnTaskSucceeded.get() == 0)
163+
assert(TestSparkPlugin.executorPlugin.numOnTaskFailed.get() == 2)
163164
}
164165
}
165166

@@ -343,9 +344,9 @@ private class TestDriverPlugin extends DriverPlugin {
343344

344345
private class TestExecutorPlugin extends ExecutorPlugin {
345346

346-
var numOnTaskStart: Int = 0
347-
var numOnTaskSucceeded: Int = 0
348-
var numOnTaskFailed: Int = 0
347+
val numOnTaskStart = new AtomicInteger(0)
348+
val numOnTaskSucceeded = new AtomicInteger(0)
349+
val numOnTaskFailed = new AtomicInteger(0)
349350

350351
override def init(ctx: PluginContext, extraConf: JMap[String, String]): Unit = {
351352
ctx.metricRegistry().register("executorMetric", new Gauge[Int] {
@@ -355,15 +356,15 @@ private class TestExecutorPlugin extends ExecutorPlugin {
355356
}
356357

357358
override def onTaskStart(): Unit = {
358-
numOnTaskStart += 1
359+
numOnTaskStart.incrementAndGet()
359360
}
360361

361362
override def onTaskSucceeded(): Unit = {
362-
numOnTaskSucceeded += 1
363+
numOnTaskSucceeded.incrementAndGet()
363364
}
364365

365366
override def onTaskFailed(failureReason: TaskFailedReason): Unit = {
366-
numOnTaskFailed += 1
367+
numOnTaskFailed.incrementAndGet()
367368
}
368369
}
369370

0 commit comments

Comments
 (0)