|
17 | 17 |
|
18 | 18 | package org.apache.spark
|
19 | 19 |
|
20 |
| -import java.util.concurrent.Semaphore |
| 20 | +import java.util.concurrent.{TimeUnit, Semaphore} |
21 | 21 | import java.util.concurrent.atomic.AtomicBoolean
|
22 | 22 | import java.util.concurrent.atomic.AtomicInteger
|
23 | 23 |
|
| 24 | +import org.apache.spark.scheduler._ |
24 | 25 | import org.scalatest.FunSuite
|
25 | 26 |
|
26 | 27 | /**
|
@@ -189,4 +190,47 @@ class ThreadingSuite extends FunSuite with LocalSparkContext {
|
189 | 190 | assert(sc.getLocalProperty("test") === "parent")
|
190 | 191 | assert(sc.getLocalProperty("Foo") === null)
|
191 | 192 | }
|
| 193 | + |
| 194 | + test("mutations to local properties should not affect submitted jobs") { |
| 195 | + val jobStarted = new Semaphore(0) |
| 196 | + val jobEnded = new Semaphore(0) |
| 197 | + @volatile var jobResult: JobResult = null |
| 198 | + |
| 199 | + sc = new SparkContext("local", "test") |
| 200 | + sc.setJobGroup("originalJobGroupId", "description") |
| 201 | + sc.addSparkListener(new SparkListener { |
| 202 | + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { |
| 203 | + jobStarted.release() |
| 204 | + } |
| 205 | + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { |
| 206 | + jobResult = jobEnd.jobResult |
| 207 | + jobEnded.release() |
| 208 | + } |
| 209 | + }) |
| 210 | + |
| 211 | + // Create a new thread which will inherit the current thread's properties |
| 212 | + val thread = new Thread() { |
| 213 | + override def run(): Unit = { |
| 214 | + assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId") |
| 215 | + // Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task |
| 216 | + try { |
| 217 | + sc.parallelize(1 to 100).foreach { x => |
| 218 | + Thread.sleep(100) |
| 219 | + } |
| 220 | + } catch { |
| 221 | + case s: SparkException => // ignored so that we don't print noise in test logs |
| 222 | + } |
| 223 | + } |
| 224 | + } |
| 225 | + thread.start() |
| 226 | + // Wait for the job to start, then mutate the original properties, which should have been |
| 227 | + // inherited by the running job but hopefully defensively copied or snapshotted: |
| 228 | + jobStarted.tryAcquire(10, TimeUnit.SECONDS) |
| 229 | + sc.setJobGroup("modifiedJobGroupId", "description") |
| 230 | + // Canceling the original job group should cancel the running job. In other words, the |
| 231 | + // modification of the properties object should not affect the properties of running jobs |
| 232 | + sc.cancelJobGroup("originalJobGroupId") |
| 233 | + jobEnded.tryAcquire(10, TimeUnit.SECONDS) |
| 234 | + assert(jobResult.isInstanceOf[JobFailed]) |
| 235 | + } |
192 | 236 | }
|
0 commit comments