
org.apache.spark.repl.SparkIMain.scala Maven / Gradle / Ivy
The newest version!
/* NSC -- new Scala compiler
* Copyright 2005-2011 LAMP/EPFL
* @author Martin Odersky
*/
package org.apache.spark.repl
import scala.tools.nsc._
import scala.tools.nsc.interpreter._
import Predef.{ println => _, _ }
import java.io.{ PrintWriter }
import java.lang.reflect
import java.net.URL
import util.{ Set => _, _ }
import io.{ AbstractFile, PlainFile, VirtualDirectory }
import reporters.{ ConsoleReporter, Reporter }
import symtab.{ Flags, Names }
import scala.tools.nsc.interpreter.{ Results => IR }
import scala.tools.util.PathResolver
import scala.tools.nsc.util.{ ScalaClassLoader, Exceptional }
import ScalaClassLoader.URLClassLoader
import Exceptional.unwrap
import scala.collection.{ mutable, immutable }
import scala.PartialFunction.{ cond, condOpt }
import scala.util.control.Exception.{ ultimately }
import scala.reflect.NameTransformer
import SparkIMain._
import org.apache.spark.HttpServer
import org.apache.spark.util.Utils
import org.apache.spark.SparkEnv
/** An interpreter for Scala code.
*
* The main public entry points are compile(), interpret(), and bind().
* The compile() method loads a complete Scala file. The interpret() method
* executes one line of Scala code at the request of the user. The bind()
* method binds an object to a variable that can then be used by later
* interpreted code.
*
* The overall approach is based on compiling the requested code and then
* using a Java classloader and Java reflection to run the code
* and access its results.
*
* In more detail, a single compiler instance is used
* to accumulate all successfully compiled or interpreted Scala code. To
* "interpret" a line of code, the compiler generates a fresh object that
* includes the line of code and which has public member(s) to export
* all variables defined by that code. To extract the result of an
* interpreted line to show the user, a second "result object" is created
* which imports the variables exported by the above object and then
* exports a single member named "$export". To accomodate user expressions
* that read from variables or methods defined in previous statements, "import"
* statements are used.
*
* This interpreter shares the strengths and weaknesses of using the
* full compiler-to-Java. The main strength is that interpreted code
* behaves exactly as does compiled code, including running at full speed.
* The main weakness is that redefining classes and methods is not handled
* properly, because rebinding at the Java level is technically difficult.
*
* @author Moez A. Abdel-Gawad
* @author Lex Spoon
*/
class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends SparkImports {
imain =>
/** construct an interpreter that reports to Console */
def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true))
def this() = this(new Settings())
/** whether to print out result lines */
var printResults: Boolean = true
/** whether to print errors */
var totalSilence: Boolean = false
private val RESULT_OBJECT_PREFIX = "RequestResult$"
lazy val formatting: Formatting = new Formatting {
val prompt = Properties.shellPromptString
}
import formatting._
val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
/** Local directory to save .class files too */
val outputDir = {
val tmp = System.getProperty("java.io.tmpdir")
val rootDir = System.getProperty("spark.repl.classdir", tmp)
Utils.createTempDir(rootDir)
}
if (SPARK_DEBUG_REPL) {
echo("Output directory: " + outputDir)
}
/** Scala compiler virtual directory for outputDir */
val virtualDirectory = new PlainFile(outputDir)
/** Jetty server that will serve our classes to worker nodes */
val classServer = new HttpServer(outputDir)
// 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()
System.setProperty("spark.repl.class.uri", classServer.uri)
if (SPARK_DEBUG_REPL) {
echo("Class server started, URI = " + classServer.uri)
}
/*
// directory to save .class files to
val virtualDirectory = new VirtualDirectory("(memory)", None) {
private def pp(root: io.AbstractFile, indentLevel: Int) {
val spaces = " " * indentLevel
out.println(spaces + root.name)
if (root.isDirectory)
root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1))
}
// print the contents hierarchically
def show() = pp(this, 0)
}
*/
/** reporter */
lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this)
import reporter.{ printMessage, withoutTruncating }
// not sure if we have some motivation to print directly to console
private def echo(msg: String) { Console println msg }
// protected def defaultImports: List[String] = List("_root_.scala.sys.exit")
/** 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
* through the roof if we wait for it. So we initialize it with a future and
* use a lazy val to ensure that any attempt to use the compiler object waits
* on the future.
*/
private val _compiler: Global = newCompiler(settings, reporter)
private var _initializeComplete = false
def isInitializeComplete = _initializeComplete
private def _initialize(): Boolean = {
val source = """
|class $repl_$init {
| List(1) map (_ + 1)
|}
|""".stripMargin
val result = try {
new _compiler.Run() compileSources List(new BatchSourceFile("", source))
if (isReplDebug || settings.debug.value) {
// Can't use printMessage here, it deadlocks
Console.println("Repl compiler initialized.")
}
// addImports(defaultImports: _*)
true
}
catch {
case x: AbstractMethodError =>
printMessage("""
|Failed to initialize compiler: abstract method error.
|This is most often remedied by a full clean and recompile.
|""".stripMargin
)
x.printStackTrace()
false
case x: MissingRequirementError => printMessage("""
|Failed to initialize compiler: %s not found.
|** Note that as of 2.8 scala does not assume use of the java classpath.
|** For the old behavior pass -usejavacp to scala, or if using a Settings
|** object programatically, settings.usejavacp.value = true.""".stripMargin.format(x.req)
)
false
}
try result
finally _initializeComplete = result
}
// set up initialization future
private var _isInitialized: () => Boolean = null
def initialize() = synchronized {
if (_isInitialized == null)
_isInitialized = scala.concurrent.ops future _initialize()
}
/** the public, go through the future compiler */
lazy val global: Global = {
initialize()
// blocks until it is ; false means catastrophic failure
if (_isInitialized()) _compiler
else null
}
@deprecated("Use `global` for access to the compiler instance.", "2.9.0")
lazy val compiler: global.type = global
import global._
object naming extends {
val global: imain.global.type = imain.global
} with Naming {
// make sure we don't overwrite their unwisely named res3 etc.
override def freshUserVarName(): String = {
val name = super.freshUserVarName()
if (definedNameMap contains name) freshUserVarName()
else name
}
}
import naming._
// object dossiers extends {
// val intp: imain.type = imain
// } with Dossiers { }
// import dossiers._
lazy val memberHandlers = new {
val intp: imain.type = imain
} with SparkMemberHandlers
import memberHandlers._
def atPickler[T](op: => T): T = atPhase(currentRun.picklerPhase)(op)
def afterTyper[T](op: => T): T = atPhase(currentRun.typerPhase.next)(op)
/** Temporarily be quiet */
def beQuietDuring[T](operation: => T): T = {
val wasPrinting = printResults
ultimately(printResults = wasPrinting) {
if (isReplDebug) echo(">> beQuietDuring")
else printResults = false
operation
}
}
def beSilentDuring[T](operation: => T): T = {
val saved = totalSilence
totalSilence = true
try operation
finally totalSilence = saved
}
def quietRun[T](code: String) = beQuietDuring(interpret(code))
/** whether to bind the lastException variable */
private var bindLastException = true
/** A string representing code to be wrapped around all lines. */
private var _executionWrapper: String = ""
def executionWrapper = _executionWrapper
def setExecutionWrapper(code: String) = _executionWrapper = code
def clearExecutionWrapper() = _executionWrapper = ""
/** Temporarily stop binding lastException */
def withoutBindingLastException[T](operation: => T): T = {
val wasBinding = bindLastException
ultimately(bindLastException = wasBinding) {
bindLastException = false
operation
}
}
protected def createLineManager(): Line.Manager = new Line.Manager
lazy val lineManager = createLineManager()
/** interpreter settings */
lazy val isettings = new SparkISettings(this)
/** Instantiate a compiler. Subclasses can override this to
* change the compiler class used by this interpreter. */
protected def newCompiler(settings: Settings, reporter: Reporter) = {
settings.outputDirs setSingleOutput virtualDirectory
settings.exposeEmptyPackage.value = true
new Global(settings, reporter)
}
/** the compiler's classpath, as URL's */
lazy val compilerClasspath: List[URL] = new PathResolver(settings) asURLs
/* A single class loader is used for all commands interpreted by this Interpreter.
It would also be possible to create a new class loader for each command
to interpret. The advantages of the current approach are:
- Expressions are only evaluated one time. This is especially
significant for I/O, e.g. "val x = Console.readLine"
The main disadvantage is:
- Objects, classes, and methods cannot be rebound. Instead, definitions
shadow the old ones, and old code objects refer to the old
definitions.
*/
private var _classLoader: AbstractFileClassLoader = null
def resetClassLoader() = _classLoader = makeClassLoader()
def classLoader: AbstractFileClassLoader = {
if (_classLoader == null)
resetClassLoader()
_classLoader
}
private def makeClassLoader(): AbstractFileClassLoader = {
val parent =
if (parentClassLoader == null) ScalaClassLoader fromURLs compilerClasspath
else new URLClassLoader(compilerClasspath, parentClassLoader)
new AbstractFileClassLoader(virtualDirectory, parent) {
/** Overridden here to try translating a simple name to the generated
* class name if the original attempt fails. This method is used by
* getResourceAsStream as well as findClass.
*/
override protected def findAbstractFile(name: String): AbstractFile = {
super.findAbstractFile(name) match {
// deadlocks on startup if we try to translate names too early
case null if isInitializeComplete => generatedName(name) map (x => super.findAbstractFile(x)) orNull
case file => file
}
}
}
}
private def loadByName(s: String): JClass =
(classLoader tryToInitializeClass s) getOrElse sys.error("Failed to load expected class: '" + s + "'")
protected def parentClassLoader: ClassLoader =
SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() )
def getInterpreterClassLoader() = classLoader
// Set the current Java "context" class loader to this interpreter's class loader
def setContextClassLoader() = classLoader.setAsContext()
/** Given a simple repl-defined name, returns the real name of
* the class representing it, e.g. for "Bippy" it may return
*
* $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy
*/
def generatedName(simpleName: String): Option[String] = {
if (simpleName endsWith "$") optFlatName(simpleName.init) map (_ + "$")
else optFlatName(simpleName)
}
def flatName(id: String) = optFlatName(id) getOrElse id
def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id)
def allDefinedNames = definedNameMap.keys.toList sortBy (_.toString)
def pathToType(id: String): String = pathToName(newTypeName(id))
def pathToTerm(id: String): String = pathToName(newTermName(id))
def pathToName(name: Name): String = {
if (definedNameMap contains name)
definedNameMap(name) fullPath name
else name.toString
}
/** Most recent tree handled which wasn't wholly synthetic. */
private def mostRecentlyHandledTree: Option[Tree] = {
prevRequests.reverse foreach { req =>
req.handlers.reverse foreach {
case x: MemberDefHandler if x.definesValue && !isInternalVarName(x.name.toString) => return Some(x.member)
case _ => ()
}
}
None
}
/** Stubs for work in progress. */
def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = {
for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) {
DBG("Redefining type '%s'\n %s -> %s".format(name, t1, t2))
}
}
def handleTermRedefinition(name: TermName, old: Request, req: Request) = {
for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) {
// Printing the types here has a tendency to cause assertion errors, like
// assertion failed: fatal: has owner value x, but a class owner is required
// so DBG is by-name now to keep it in the family. (It also traps the assertion error,
// but we don't want to unnecessarily risk hosing the compiler's internal state.)
DBG("Redefining term '%s'\n %s -> %s".format(name, t1, t2))
}
}
def recordRequest(req: Request) {
if (req == null || referencedNameMap == null)
return
prevRequests += req
req.referencedNames foreach (x => referencedNameMap(x) = req)
// warning about serially defining companions. It'd be easy
// enough to just redefine them together but that may not always
// be what people want so I'm waiting until I can do it better.
if (!settings.nowarnings.value) {
for {
name <- req.definedNames filterNot (x => req.definedNames contains x.companionName)
oldReq <- definedNameMap get name.companionName
newSym <- req.definedSymbols get name
oldSym <- oldReq.definedSymbols get name.companionName
} {
printMessage("warning: previously defined %s is not a companion to %s.".format(oldSym, newSym))
printMessage("Companions must be defined together; you may wish to use :paste mode for this.")
}
}
// Updating the defined name map
req.definedNames foreach { name =>
if (definedNameMap contains name) {
if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req)
else handleTermRedefinition(name.toTermName, definedNameMap(name), req)
}
definedNameMap(name) = req
}
}
/** Parse a line into a sequence of trees. Returns None if the input is incomplete. */
def parse(line: String): Option[List[Tree]] = {
var justNeedsMore = false
reporter.withIncompleteHandler((pos,msg) => {justNeedsMore = true}) {
// simple parse: just parse it, nothing else
def simpleParse(code: String): List[Tree] = {
reporter.reset()
val unit = new CompilationUnit(new BatchSourceFile("", code))
val scanner = new syntaxAnalyzer.UnitParser(unit)
scanner.templateStatSeq(false)._2
}
val trees = simpleParse(line)
if (reporter.hasErrors) Some(Nil) // the result did not parse, so stop
else if (justNeedsMore) None
else Some(trees)
}
}
def isParseable(line: String): Boolean = {
beSilentDuring {
parse(line) match {
case Some(xs) => xs.nonEmpty // parses as-is
case None => true // incomplete
}
}
}
/** Compile an nsc SourceFile. Returns true if there are
* no compilation errors, or false otherwise.
*/
def compileSources(sources: SourceFile*): Boolean = {
reporter.reset()
new Run() compileSources sources.toList
!reporter.hasErrors
}
/** Compile a string. Returns true if there are no
* compilation errors, or false otherwise.
*/
def compileString(code: String): Boolean =
compileSources(new BatchSourceFile("