Commit 3d89043
[SPARK-1210] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executo...
...r.
Constructor of `org.apache.spark.executor.Executor` should not set context class loader of current thread, which is backend Actor's thread.
Run the following code in local-mode REPL.
```
scala> case class Foo(i: Int)
scala> val ret = sc.parallelize((1 to 100).map(Foo), 10).collect
```
This causes errors as follows:
```
ERROR actor.OneForOneStrategy: [L$line5.$read$$iwC$$iwC$$iwC$$iwC$Foo;
java.lang.ArrayStoreException: [L$line5.$read$$iwC$$iwC$$iwC$$iwC$Foo;
at scala.runtime.ScalaRunTime$.array_update(ScalaRunTime.scala:88)
at org.apache.spark.SparkContext$$anonfun$runJob$3.apply(SparkContext.scala:870)
at org.apache.spark.SparkContext$$anonfun$runJob$3.apply(SparkContext.scala:870)
at org.apache.spark.scheduler.JobWaiter.taskSucceeded(JobWaiter.scala:56)
at org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:859)
at org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:616)
at org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:207)
at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
at akka.actor.ActorCell.invoke(ActorCell.scala:456)
at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
at akka.dispatch.Mailbox.run(Mailbox.scala:219)
at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
```
This is because the class loaders to deserialize result `Foo` instances might be different from backend Actor's, and the Actor's class loader should be the same as Driver's.
Author: Takuya UESHIN <[email protected]>
Closes #15 from ueshin/wip/wrongcontextclassloader and squashes the following commits:
d79e8c0 [Takuya UESHIN] Change a parent class loader of ExecutorURLClassLoader.
c6c09b6 [Takuya UESHIN] Add a test to collect objects of class defined in repl.
43e0feb [Takuya UESHIN] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executor.1 parent 6f986f0 commit 3d89043
File tree
2 files changed
+13
-3
lines changed- core/src/main/scala/org/apache/spark/executor
- repl/src/test/scala/org/apache/spark/repl
2 files changed
+13
-3
lines changedLines changed: 2 additions & 3 deletions
| Original file line number | Diff line number | Diff line change | |
|---|---|---|---|
| |||
112 | 112 | | |
113 | 113 | | |
114 | 114 | | |
115 | | - | |
| 115 | + | |
116 | 116 | | |
117 | 117 | | |
118 | 118 | | |
119 | | - | |
120 | 119 | | |
121 | 120 | | |
122 | 121 | | |
| |||
294 | 293 | | |
295 | 294 | | |
296 | 295 | | |
297 | | - | |
| 296 | + | |
298 | 297 | | |
299 | 298 | | |
300 | 299 | | |
| |||
Lines changed: 11 additions & 0 deletions
| Original file line number | Diff line number | Diff line change | |
|---|---|---|---|
| |||
242 | 242 | | |
243 | 243 | | |
244 | 244 | | |
| 245 | + | |
| 246 | + | |
| 247 | + | |
| 248 | + | |
| 249 | + | |
| 250 | + | |
| 251 | + | |
| 252 | + | |
| 253 | + | |
| 254 | + | |
| 255 | + | |
245 | 256 | | |
0 commit comments