aboutsummaryrefslogtreecommitdiff
path: root/repl
diff options
context:
space:
mode:
authorMarcelo Vanzin <vanzin@cloudera.com>2015-12-10 13:26:30 -0800
committerMarcelo Vanzin <vanzin@cloudera.com>2015-12-10 13:26:30 -0800
commit4a46b8859d3314b5b45a67cdc5c81fecb6e9e78c (patch)
treee675378fe850f9fbcf2cafea7cae589876f147a8 /repl
parent2ecbe02d5b28ee562d10c1735244b90a08532c9e (diff)
downloadspark-4a46b8859d3314b5b45a67cdc5c81fecb6e9e78c.tar.gz
spark-4a46b8859d3314b5b45a67cdc5c81fecb6e9e78c.tar.bz2
spark-4a46b8859d3314b5b45a67cdc5c81fecb6e9e78c.zip
[SPARK-11563][CORE][REPL] Use RpcEnv to transfer REPL-generated classes.
This avoids bringing up yet another HTTP server on the driver, and instead reuses the file server already managed by the driver's RpcEnv. As a bonus, the repl now inherits the security features of the network library. There's also a small change to create the directory for storing classes under the root temp dir for the application (instead of directly under java.io.tmpdir). Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9923 from vanzin/SPARK-11563.
Diffstat (limited to 'repl')
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala17
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala28
-rw-r--r--repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala23
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala36
-rw-r--r--repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala48
5 files changed, 85 insertions, 67 deletions
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 304b1e8cdb..22749c4609 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -253,7 +253,7 @@ class SparkILoop(
case xs => xs find (_.name == cmd)
}
}
- private var fallbackMode = false
+ private var fallbackMode = false
private def toggleFallbackMode() {
val old = fallbackMode
@@ -261,9 +261,9 @@ class SparkILoop(
System.setProperty("spark.repl.fallback", fallbackMode.toString)
echo(s"""
|Switched ${if (old) "off" else "on"} fallback mode without restarting.
- | If you have defined classes in the repl, it would
+ | If you have defined classes in the repl, it would
|be good to redefine them incase you plan to use them. If you still run
- |into issues it would be good to restart the repl and turn on `:fallback`
+ |into issues it would be good to restart the repl and turn on `:fallback`
|mode as first command.
""".stripMargin)
}
@@ -350,7 +350,7 @@ class SparkILoop(
shCommand,
nullary("silent", "disable/enable automatic printing of results", verbosity),
nullary("fallback", """
- |disable/enable advanced repl changes, these fix some issues but may introduce others.
+ |disable/enable advanced repl changes, these fix some issues but may introduce others.
|This mode will be removed once these fixes stablize""".stripMargin, toggleFallbackMode),
cmd("type", "[-v] <expr>", "display the type of an expression without evaluating it", typeCommand),
nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand)
@@ -1009,8 +1009,13 @@ class SparkILoop(
val conf = new SparkConf()
.setMaster(getMaster())
.setJars(jars)
- .set("spark.repl.class.uri", intp.classServerUri)
.setIfMissing("spark.app.name", "Spark shell")
+ // SparkContext will detect this configuration and register it with the RpcEnv's
+ // file server, setting spark.repl.class.uri to the actual URI for executors to
+ // use. This is sort of ugly but since executors are started as part of SparkContext
+ // initialization in certain cases, there's an initialization order issue that prevents
+ // this from being set after SparkContext is instantiated.
+ .set("spark.repl.class.outputDir", intp.outputDir.getAbsolutePath())
if (execUri != null) {
conf.set("spark.executor.uri", execUri)
}
@@ -1025,7 +1030,7 @@ class SparkILoop(
val loader = Utils.getContextOrSparkClassLoader
try {
sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext])
- .newInstance(sparkContext).asInstanceOf[SQLContext]
+ .newInstance(sparkContext).asInstanceOf[SQLContext]
logInfo("Created sql context (with Hive support)..")
}
catch {
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index 829b12269f..7fcb423575 100644
--- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -37,7 +37,7 @@ import scala.reflect.{ ClassTag, classTag }
import scala.tools.reflect.StdRuntimeTags._
import scala.util.control.ControlThrowable
-import org.apache.spark.{Logging, HttpServer, SecurityManager, SparkConf}
+import org.apache.spark.{Logging, SparkConf, SparkContext}
import org.apache.spark.util.Utils
import org.apache.spark.annotation.DeveloperApi
@@ -96,10 +96,9 @@ import org.apache.spark.annotation.DeveloperApi
private val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
/** Local directory to save .class files too */
- private lazy val outputDir = {
- val tmp = System.getProperty("java.io.tmpdir")
- val rootDir = conf.get("spark.repl.classdir", tmp)
- Utils.createTempDir(rootDir)
+ private[repl] val outputDir = {
+ val rootDir = conf.getOption("spark.repl.classdir").getOrElse(Utils.getLocalDir(conf))
+ Utils.createTempDir(root = rootDir, namePrefix = "repl")
}
if (SPARK_DEBUG_REPL) {
echo("Output directory: " + outputDir)
@@ -114,8 +113,6 @@ import org.apache.spark.annotation.DeveloperApi
private val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles
/** Jetty server that will serve our classes to worker nodes */
- private val classServerPort = conf.getInt("spark.replClassServer.port", 0)
- private val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server")
private var currentSettings: Settings = initialSettings
private var printResults = true // whether to print result lines
private var totalSilence = false // whether to print anything
@@ -124,22 +121,6 @@ import org.apache.spark.annotation.DeveloperApi
private var bindExceptions = true // whether to bind the lastException variable
private var _executionWrapper = "" // code to be wrapped around all lines
-
- // Start the classServer and store its URI in a spark system property
- // (which will be passed to executors so that they can connect to it)
- classServer.start()
- if (SPARK_DEBUG_REPL) {
- echo("Class server started, URI = " + classServer.uri)
- }
-
- /**
- * URI of the class server used to feed REPL compiled classes.
- *
- * @return The string representing the class server uri
- */
- @DeveloperApi
- def classServerUri = classServer.uri
-
/** We're going to go to some trouble to initialize the compiler asynchronously.
* It's critical that nothing call into it until it's been initialized or we will
* run into unrecoverable issues, but the perceived repl startup time goes
@@ -994,7 +975,6 @@ import org.apache.spark.annotation.DeveloperApi
@DeveloperApi
def close() {
reporter.flush()
- classServer.stop()
}
/**
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
index 455a6b9a93..44650f25f7 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
@@ -28,11 +28,13 @@ import org.apache.spark.sql.SQLContext
object Main extends Logging {
val conf = new SparkConf()
- val tmp = System.getProperty("java.io.tmpdir")
- val rootDir = conf.get("spark.repl.classdir", tmp)
- val outputDir = Utils.createTempDir(rootDir)
+ val rootDir = conf.getOption("spark.repl.classdir").getOrElse(Utils.getLocalDir(conf))
+ val outputDir = Utils.createTempDir(root = rootDir, namePrefix = "repl")
+ val s = new Settings()
+ s.processArguments(List("-Yrepl-class-based",
+ "-Yrepl-outdir", s"${outputDir.getAbsolutePath}",
+ "-classpath", getAddedJars.mkString(File.pathSeparator)), true)
// the creation of SecurityManager has to be lazy so SPARK_YARN_MODE is set if needed
- lazy val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf))
var sparkContext: SparkContext = _
var sqlContext: SQLContext = _
var interp = new SparkILoop // this is a public var because tests reset it.
@@ -45,7 +47,6 @@ object Main extends Logging {
}
def main(args: Array[String]) {
-
val interpArguments = List(
"-Yrepl-class-based",
"-Yrepl-outdir", s"${outputDir.getAbsolutePath}",
@@ -57,11 +58,7 @@ object Main extends Logging {
if (!hasErrors) {
if (getMaster == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true")
- // Start the classServer and store its URI in a spark system property
- // (which will be passed to executors so that they can connect to it)
- classServer.start()
interp.process(settings) // Repl starts and goes in loop of R.E.P.L
- classServer.stop()
Option(sparkContext).map(_.stop)
}
}
@@ -82,9 +79,13 @@ object Main extends Logging {
val conf = new SparkConf()
.setMaster(getMaster)
.setJars(jars)
- .set("spark.repl.class.uri", classServer.uri)
.setIfMissing("spark.app.name", "Spark shell")
- logInfo("Spark class server started at " + classServer.uri)
+ // SparkContext will detect this configuration and register it with the RpcEnv's
+ // file server, setting spark.repl.class.uri to the actual URI for executors to
+ // use. This is sort of ugly but since executors are started as part of SparkContext
+ // initialization in certain cases, there's an initialization order issue that prevents
+ // this from being set after SparkContext is instantiated.
+ .set("spark.repl.class.outputDir", outputDir.getAbsolutePath())
if (execUri != null) {
conf.set("spark.executor.uri", execUri)
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
index a8859fcd45..da8f0aa1e3 100644
--- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
@@ -19,6 +19,7 @@ package org.apache.spark.repl
import java.io.{IOException, ByteArrayOutputStream, InputStream}
import java.net.{HttpURLConnection, URI, URL, URLEncoder}
+import java.nio.channels.Channels
import scala.util.control.NonFatal
@@ -38,7 +39,11 @@ import org.apache.spark.util.ParentClassLoader
* This class loader delegates getting/finding resources to parent loader,
* which makes sense until REPL never provide resource dynamically.
*/
-class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader,
+class ExecutorClassLoader(
+ conf: SparkConf,
+ env: SparkEnv,
+ classUri: String,
+ parent: ClassLoader,
userClassPathFirst: Boolean) extends ClassLoader with Logging {
val uri = new URI(classUri)
val directory = uri.getPath
@@ -48,13 +53,12 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader
// Allows HTTP connect and read timeouts to be controlled for testing / debugging purposes
private[repl] var httpUrlConnectionTimeoutMillis: Int = -1
- // Hadoop FileSystem object for our URI, if it isn't using HTTP
- var fileSystem: FileSystem = {
- if (Set("http", "https", "ftp").contains(uri.getScheme)) {
- null
- } else {
- FileSystem.get(uri, SparkHadoopUtil.get.newConfiguration(conf))
- }
+ private val fetchFn: (String) => InputStream = uri.getScheme() match {
+ case "spark" => getClassFileInputStreamFromSparkRPC
+ case "http" | "https" | "ftp" => getClassFileInputStreamFromHttpServer
+ case _ =>
+ val fileSystem = FileSystem.get(uri, SparkHadoopUtil.get.newConfiguration(conf))
+ getClassFileInputStreamFromFileSystem(fileSystem)
}
override def getResource(name: String): URL = {
@@ -90,6 +94,11 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader
}
}
+ private def getClassFileInputStreamFromSparkRPC(path: String): InputStream = {
+ val channel = env.rpcEnv.openChannel(s"$classUri/$path")
+ Channels.newInputStream(channel)
+ }
+
private def getClassFileInputStreamFromHttpServer(pathInDirectory: String): InputStream = {
val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) {
val uri = new URI(classUri + "/" + urlEncode(pathInDirectory))
@@ -126,7 +135,8 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader
}
}
- private def getClassFileInputStreamFromFileSystem(pathInDirectory: String): InputStream = {
+ private def getClassFileInputStreamFromFileSystem(fileSystem: FileSystem)(
+ pathInDirectory: String): InputStream = {
val path = new Path(directory, pathInDirectory)
if (fileSystem.exists(path)) {
fileSystem.open(path)
@@ -139,13 +149,7 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader
val pathInDirectory = name.replace('.', '/') + ".class"
var inputStream: InputStream = null
try {
- inputStream = {
- if (fileSystem != null) {
- getClassFileInputStreamFromFileSystem(pathInDirectory)
- } else {
- getClassFileInputStreamFromHttpServer(pathInDirectory)
- }
- }
+ inputStream = fetchFn(pathInDirectory)
val bytes = readAndTransformClass(name, inputStream)
Some(defineClass(name, bytes, 0, bytes.length))
} catch {
diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala
index c1211f7596..1360f09e7f 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala
@@ -18,24 +18,29 @@
package org.apache.spark.repl
import java.io.File
-import java.net.{URL, URLClassLoader}
+import java.net.{URI, URL, URLClassLoader}
+import java.nio.channels.{FileChannel, ReadableByteChannel}
import java.nio.charset.StandardCharsets
+import java.nio.file.{Paths, StandardOpenOption}
import java.util
-import com.google.common.io.Files
-
import scala.concurrent.duration._
import scala.io.Source
import scala.language.implicitConversions
import scala.language.postfixOps
+import com.google.common.io.Files
import org.scalatest.BeforeAndAfterAll
import org.scalatest.concurrent.Interruptor
import org.scalatest.concurrent.Timeouts._
import org.scalatest.mock.MockitoSugar
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.mockito.Matchers.anyString
import org.mockito.Mockito._
import org.apache.spark._
+import org.apache.spark.rpc.RpcEnv
import org.apache.spark.util.Utils
class ExecutorClassLoaderSuite
@@ -78,7 +83,7 @@ class ExecutorClassLoaderSuite
test("child first") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorClassLoader(new SparkConf(), url1, parentLoader, true)
+ val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true)
val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "1")
@@ -86,7 +91,7 @@ class ExecutorClassLoaderSuite
test("parent first") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorClassLoader(new SparkConf(), url1, parentLoader, false)
+ val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, false)
val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
@@ -94,7 +99,7 @@ class ExecutorClassLoaderSuite
test("child first can fall back") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorClassLoader(new SparkConf(), url1, parentLoader, true)
+ val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true)
val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance()
val fakeClassVersion = fakeClass.toString
assert(fakeClassVersion === "2")
@@ -102,7 +107,7 @@ class ExecutorClassLoaderSuite
test("child first can fail") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorClassLoader(new SparkConf(), url1, parentLoader, true)
+ val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true)
intercept[java.lang.ClassNotFoundException] {
classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance()
}
@@ -110,7 +115,7 @@ class ExecutorClassLoaderSuite
test("resource from parent") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorClassLoader(new SparkConf(), url1, parentLoader, true)
+ val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true)
val resourceName: String = parentResourceNames.head
val is = classLoader.getResourceAsStream(resourceName)
assert(is != null, s"Resource $resourceName not found")
@@ -120,7 +125,7 @@ class ExecutorClassLoaderSuite
test("resources from parent") {
val parentLoader = new URLClassLoader(urls2, null)
- val classLoader = new ExecutorClassLoader(new SparkConf(), url1, parentLoader, true)
+ val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true)
val resourceName: String = parentResourceNames.head
val resources: util.Enumeration[URL] = classLoader.getResources(resourceName)
assert(resources.hasMoreElements, s"Resource $resourceName not found")
@@ -142,7 +147,7 @@ class ExecutorClassLoaderSuite
SparkEnv.set(mockEnv)
// Create an ExecutorClassLoader that's configured to load classes from the HTTP server
val parentLoader = new URLClassLoader(Array.empty, null)
- val classLoader = new ExecutorClassLoader(conf, classServer.uri, parentLoader, false)
+ val classLoader = new ExecutorClassLoader(conf, null, classServer.uri, parentLoader, false)
classLoader.httpUrlConnectionTimeoutMillis = 500
// Check that this class loader can actually load classes that exist
val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance()
@@ -177,4 +182,27 @@ class ExecutorClassLoaderSuite
failAfter(10 seconds)(tryAndFailToLoadABunchOfClasses())(interruptor)
}
+ test("fetch classes using Spark's RpcEnv") {
+ val env = mock[SparkEnv]
+ val rpcEnv = mock[RpcEnv]
+ when(env.rpcEnv).thenReturn(rpcEnv)
+ when(rpcEnv.openChannel(anyString())).thenAnswer(new Answer[ReadableByteChannel]() {
+ override def answer(invocation: InvocationOnMock): ReadableByteChannel = {
+ val uri = new URI(invocation.getArguments()(0).asInstanceOf[String])
+ val path = Paths.get(tempDir1.getAbsolutePath(), uri.getPath().stripPrefix("/"))
+ FileChannel.open(path, StandardOpenOption.READ)
+ }
+ })
+
+ val classLoader = new ExecutorClassLoader(new SparkConf(), env, "spark://localhost:1234",
+ getClass().getClassLoader(), false)
+
+ val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance()
+ val fakeClassVersion = fakeClass.toString
+ assert(fakeClassVersion === "1")
+ intercept[java.lang.ClassNotFoundException] {
+ classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance()
+ }
+ }
+
}