Skip to content

Commit 0b7e5ab

Browse files
author
Andrew Or
committed
Clone parent local properties on inherit
SPARK-10548 is caused by the fact that mutating local properties in the parent thread is reflected in the children threads. Instead, we should just make a clone of the parent properties.
1 parent 5b7067c commit 0b7e5ab

File tree

3 files changed

+134
-38
lines changed

3 files changed

+134
-38
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ import scala.collection.mutable.HashMap
3333
import scala.reflect.{ClassTag, classTag}
3434
import scala.util.control.NonFatal
3535

36+
import org.apache.commons.lang.SerializationUtils
3637
import org.apache.hadoop.conf.Configuration
3738
import org.apache.hadoop.fs.Path
3839
import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable,
@@ -347,8 +348,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
347348
private[spark] var checkpointDir: Option[String] = None
348349

349350
// Thread Local variable that can be used by users to pass information down the stack
350-
private val localProperties = new InheritableThreadLocal[Properties] {
351-
override protected def childValue(parent: Properties): Properties = new Properties(parent)
351+
protected[spark] val localProperties = new InheritableThreadLocal[Properties] {
352+
override protected def childValue(parent: Properties): Properties = {
353+
// Note: make a clone such that changes in the parent properties aren't reflected in
354+
// the those of the children threads, which has confusing semantics (SPARK-10564).
355+
SerializationUtils.clone(parent).asInstanceOf[Properties]
356+
}
352357
override protected def initialValue(): Properties = new Properties()
353358
}
354359

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

Lines changed: 26 additions & 36 deletions
Original file line numberDiff line numberDiff line change
@@ -152,7 +152,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
152152
ThreadingSuiteState.runningThreads.get() + "); failing test")
153153
fail("One or more threads didn't see runningThreads = 4")
154154
}
155-
throwable.foreach { t => throw t }
155+
throwable.foreach { t => throw improveStackTrace(t) }
156156
}
157157

158158
test("set local properties in different thread") {
@@ -179,7 +179,7 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
179179

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

185185
test("set and get local properties in parent-children thread") {
@@ -209,49 +209,39 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging {
209209
sem.acquire(5)
210210
assert(sc.getLocalProperty("test") === "parent")
211211
assert(sc.getLocalProperty("Foo") === null)
212-
throwable.foreach { t => throw t }
212+
throwable.foreach { t => throw improveStackTrace(t) }
213213
}
214214

215-
test("mutations to local properties should not affect submitted jobs (SPARK-6629)") {
216-
val jobStarted = new Semaphore(0)
217-
val jobEnded = new Semaphore(0)
218-
@volatile var jobResult: JobResult = null
219-
215+
test("mutation in parent local property does not affect child (SPARK-10563)") {
220216
sc = new SparkContext("local", "test")
221-
sc.setJobGroup("originalJobGroupId", "description")
222-
sc.addSparkListener(new SparkListener {
223-
override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
224-
jobStarted.release()
225-
}
226-
override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
227-
jobResult = jobEnd.jobResult
228-
jobEnded.release()
229-
}
230-
})
231-
232-
// Create a new thread which will inherit the current thread's properties
233-
val thread = new Thread() {
217+
val originalTestValue: String = "original-value"
218+
var threadTestValue: String = null
219+
sc.setLocalProperty("test", originalTestValue)
220+
var throwable: Option[Throwable] = None
221+
val thread = new Thread {
234222
override def run(): Unit = {
235-
assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId")
236-
// Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task
237223
try {
238-
sc.parallelize(1 to 100).foreach { x =>
239-
Thread.sleep(100)
240-
}
224+
threadTestValue = sc.getLocalProperty("test")
241225
} catch {
242-
case s: SparkException => // ignored so that we don't print noise in test logs
226+
case t: Throwable =>
227+
throwable = Some(t)
243228
}
244229
}
245230
}
231+
sc.setLocalProperty("test", "this-should-not-be-inherited")
246232
thread.start()
247-
// Wait for the job to start, then mutate the original properties, which should have been
248-
// inherited by the running job but hopefully defensively copied or snapshotted:
249-
jobStarted.tryAcquire(10, TimeUnit.SECONDS)
250-
sc.setJobGroup("modifiedJobGroupId", "description")
251-
// Canceling the original job group should cancel the running job. In other words, the
252-
// modification of the properties object should not affect the properties of running jobs
253-
sc.cancelJobGroup("originalJobGroupId")
254-
jobEnded.tryAcquire(10, TimeUnit.SECONDS)
255-
assert(jobResult.isInstanceOf[JobFailed])
233+
thread.join()
234+
throwable.foreach { t => throw improveStackTrace(t) }
235+
assert(threadTestValue === originalTestValue)
236+
}
237+
238+
/**
239+
* Improve the stack trace of an error thrown from within a thread.
240+
* Otherwise it's difficult to tell which line in the test the error came from.
241+
*/
242+
private def improveStackTrace(t: Throwable): Throwable = {
243+
t.setStackTrace(t.getStackTrace ++ Thread.currentThread.getStackTrace)
244+
t
256245
}
246+
257247
}
Lines changed: 101 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,101 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution
19+
20+
import java.util.Properties
21+
22+
import scala.collection.parallel.CompositeThrowable
23+
24+
import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
25+
import org.apache.spark.sql.SQLContext
26+
27+
class SQLExecutionSuite extends SparkFunSuite {
28+
29+
test("concurrent query execution (SPARK-10548)") {
30+
// Try to reproduce the issue with the old SparkContext
31+
val conf = new SparkConf()
32+
.setMaster("local[*]")
33+
.setAppName("test")
34+
val badSparkContext = new BadSparkContext(conf)
35+
try {
36+
testConcurrentQueryExecution(badSparkContext)
37+
fail("unable to reproduce SPARK-10548")
38+
} catch {
39+
case e: IllegalArgumentException =>
40+
assert(e.getMessage.contains(SQLExecution.EXECUTION_ID_KEY))
41+
} finally {
42+
badSparkContext.stop()
43+
}
44+
45+
// Verify that the issue is fixed with the latest SparkContext
46+
val goodSparkContext = new SparkContext(conf)
47+
try {
48+
testConcurrentQueryExecution(goodSparkContext)
49+
} finally {
50+
goodSparkContext.stop()
51+
}
52+
}
53+
54+
/**
55+
* Trigger SPARK-10548 by mocking a parent and its child thread executing queries concurrently.
56+
*/
57+
private def testConcurrentQueryExecution(sc: SparkContext): Unit = {
58+
val sqlContext = new SQLContext(sc)
59+
import sqlContext.implicits._
60+
61+
// Initialize local properties. This is necessary for the test to pass.
62+
sc.getLocalProperties
63+
64+
// Set up a thread that runs executes a simple SQL query.
65+
// Before starting the thread, mutate the execution ID in the parent.
66+
// The child thread should not see the effect of this change.
67+
var throwable: Option[Throwable] = None
68+
val child = new Thread {
69+
override def run(): Unit = {
70+
try {
71+
sc.parallelize(1 to 100).map { i => (i, i) }.toDF("a", "b").collect()
72+
} catch {
73+
case t: Throwable =>
74+
throwable = Some(t)
75+
}
76+
77+
}
78+
}
79+
sc.setLocalProperty(SQLExecution.EXECUTION_ID_KEY, "anything")
80+
child.start()
81+
child.join()
82+
83+
// The throwable is thrown from the child thread so it doesn't have a helpful stack trace
84+
throwable.foreach { t =>
85+
t.setStackTrace(t.getStackTrace ++ Thread.currentThread.getStackTrace)
86+
throw t
87+
}
88+
}
89+
90+
}
91+
92+
/**
93+
* A bad [[SparkContext]] that does not clone the inheritable thread local properties
94+
* when passing them to children threads.
95+
*/
96+
private class BadSparkContext(conf: SparkConf) extends SparkContext(conf) {
97+
protected[spark] override val localProperties = new InheritableThreadLocal[Properties] {
98+
override protected def childValue(parent: Properties): Properties = new Properties(parent)
99+
override protected def initialValue(): Properties = new Properties()
100+
}
101+
}

0 commit comments

Comments
 (0)