Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Move classLoader initialization to SparkContext
SparkEnv is used by Executor as well; we want this change to affect driver only.
  • Loading branch information
Evan Chan committed Dec 29, 2013
commit d3df2415eaed4da50fc1ffeef42e0140f6b49d88
11 changes: 9 additions & 2 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
import org.apache.mesos.MesosNativeLibrary

import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
import org.apache.spark.executor.ExecutorURLClassLoader
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd._
import org.apache.spark.scheduler._
Expand Down Expand Up @@ -101,6 +102,12 @@ class SparkContext(

val isLocal = (master == "local" || master.startsWith("local["))

// Create a classLoader for use by the driver so that jars added via addJar are available to the driver
// Do this before all other initialization so that any thread pools created for this SparkContext
// uses the class loader
private[spark] val classLoader = new ExecutorURLClassLoader(Array.empty[URL], this.getClass.getClassLoader)
Thread.currentThread.setContextClassLoader(classLoader)

// Create the Spark execution environment (cache, map output tracker, etc)
private[spark] val env = SparkEnv.createFromSystemProperties(
"<driver>",
Expand Down Expand Up @@ -667,9 +674,9 @@ class SparkContext(
}

private def addUrlToDriverLoader(url: URL) {
if (!env.classLoader.getURLs.contains(url)) {
if (!classLoader.getURLs.contains(url)) {
logInfo("Adding JAR " + url + " to driver class loader")
env.classLoader.addURL(url)
classLoader.addURL(url)
}
}

Expand Down
19 changes: 2 additions & 17 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import akka.actor._
import akka.remote.RemoteActorRefProvider

import org.apache.spark.broadcast.BroadcastManager
import org.apache.spark.executor.ExecutorURLClassLoader
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster}
import org.apache.spark.network.ConnectionManager
Expand Down Expand Up @@ -57,8 +56,7 @@ class SparkEnv (
val connectionManager: ConnectionManager,
val httpFileServer: HttpFileServer,
val sparkFilesDir: String,
val metricsSystem: MetricsSystem,
val classLoader: ExecutorURLClassLoader) {
val metricsSystem: MetricsSystem) {

private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()

Expand Down Expand Up @@ -121,12 +119,6 @@ object SparkEnv extends Logging {
isDriver: Boolean,
isLocal: Boolean): SparkEnv = {

// Create a classLoader for use by the driver so that jars added via addJar are available to the driver
// Do this before all other initialization so that any thread pools created for this SparkContext
// uses the class loader
val driverLoader = getDriverClassLoader()
Thread.currentThread.setContextClassLoader(driverLoader)

val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port)

// Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port),
Expand Down Expand Up @@ -241,13 +233,6 @@ object SparkEnv extends Logging {
connectionManager,
httpFileServer,
sparkFilesDir,
metricsSystem,
driverLoader)
}

private def getDriverClassLoader(): ExecutorURLClassLoader = {
// Initially there are no jars
val parentLoader = this.getClass.getClassLoader
new ExecutorURLClassLoader(Array.empty[URL], parentLoader)
metricsSystem)
}
}