diff options
11 files changed, 644 insertions, 195 deletions
diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala index 05816941b5..6480e2d24e 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala @@ -19,14 +19,21 @@ package org.apache.spark.repl import scala.tools.nsc.{Settings, CompilerCommand} import scala.Predef._ +import org.apache.spark.annotation.DeveloperApi /** * Command class enabling Spark-specific command line options (provided by * <i>org.apache.spark.repl.SparkRunnerSettings</i>). + * + * @example new SparkCommandLine(Nil).settings + * + * @param args The list of command line arguments + * @param settings The underlying settings to associate with this set of + * command-line options */ +@DeveloperApi class SparkCommandLine(args: List[String], override val settings: Settings) extends CompilerCommand(args, settings) { - def this(args: List[String], error: String => Unit) { this(args, new SparkRunnerSettings(error)) } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index f8432c8af6..5fb378112e 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -15,7 +15,7 @@ import scala.tools.nsc.ast.parser.Tokens.EOF import org.apache.spark.Logging -trait SparkExprTyper extends Logging { +private[repl] trait SparkExprTyper extends Logging { val repl: SparkIMain import repl._ diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala index 5340951d91..955be17a73 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala @@ -17,6 +17,23 @@ package scala.tools.nsc +import org.apache.spark.annotation.DeveloperApi + +// NOTE: Forced to be public (and in scala.tools.nsc package) to access the +// settings "explicitParentLoader" method + +/** + * Provides exposure for the explicitParentLoader method on settings instances. + */ +@DeveloperApi object SparkHelper { + /** + * Retrieves the explicit parent loader for the provided settings. + * + * @param settings The settings whose explicit parent loader to retrieve + * + * @return The Optional classloader representing the explicit parent loader + */ + @DeveloperApi def explicitParentLoader(settings: Settings) = settings.explicitParentLoader } 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 e56b74edba..72c1a98999 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 @@ -10,6 +10,8 @@ package org.apache.spark.repl import java.net.URL +import org.apache.spark.annotation.DeveloperApi + import scala.reflect.io.AbstractFile import scala.tools.nsc._ import scala.tools.nsc.backend.JavaPlatform @@ -57,20 +59,22 @@ import org.apache.spark.util.Utils * @author Lex Spoon * @version 1.2 */ -class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, - val master: Option[String]) - extends AnyRef - with LoopCommands - with SparkILoopInit - with Logging -{ +@DeveloperApi +class SparkILoop( + private val in0: Option[BufferedReader], + protected val out: JPrintWriter, + val master: Option[String] +) extends AnyRef with LoopCommands with SparkILoopInit with Logging { def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master)) def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) def this() = this(None, new JPrintWriter(Console.out, true), None) - var in: InteractiveReader = _ // the input stream from which commands come - var settings: Settings = _ - var intp: SparkIMain = _ + private var in: InteractiveReader = _ // the input stream from which commands come + + // NOTE: Exposed in package for testing + private[repl] var settings: Settings = _ + + private[repl] var intp: SparkIMain = _ @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i @@ -123,6 +127,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } + // NOTE: Must be public for visibility + @DeveloperApi var sparkContext: SparkContext = _ override def echoCommandMessage(msg: String) { @@ -130,45 +136,45 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } // def isAsync = !settings.Yreplsync.value - def isAsync = false + private[repl] def isAsync = false // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) - def history = in.history + private def history = in.history /** The context class loader at the time this object was created */ protected val originalClassLoader = Utils.getContextOrSparkClassLoader // classpath entries added via :cp - var addedClasspath: String = "" + private var addedClasspath: String = "" /** A reverse list of commands to replay if the user requests a :replay */ - var replayCommandStack: List[String] = Nil + private var replayCommandStack: List[String] = Nil /** A list of commands to replay if the user requests a :replay */ - def replayCommands = replayCommandStack.reverse + private def replayCommands = replayCommandStack.reverse /** Record a command for replay should the user request a :replay */ - def addReplay(cmd: String) = replayCommandStack ::= cmd + private def addReplay(cmd: String) = replayCommandStack ::= cmd - def savingReplayStack[T](body: => T): T = { + private def savingReplayStack[T](body: => T): T = { val saved = replayCommandStack try body finally replayCommandStack = saved } - def savingReader[T](body: => T): T = { + private def savingReader[T](body: => T): T = { val saved = in try body finally in = saved } - def sparkCleanUp(){ + private def sparkCleanUp(){ echo("Stopping spark context.") intp.beQuietDuring { command("sc.stop()") } } /** Close the interpreter and set the var to null. */ - def closeInterpreter() { + private def closeInterpreter() { if (intp ne null) { sparkCleanUp() intp.close() @@ -179,14 +185,16 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, class SparkILoopInterpreter extends SparkIMain(settings, out) { outer => - override lazy val formatting = new Formatting { + override private[repl] lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } - /** Create a new interpreter. */ - def createInterpreter() { + /** + * Constructs a new interpreter. + */ + protected def createInterpreter() { require(settings != null) if (addedClasspath != "") settings.classpath.append(addedClasspath) @@ -207,7 +215,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** print a friendly help message */ - def helpCommand(line: String): Result = { + private def helpCommand(line: String): Result = { if (line == "") helpSummary() else uniqueCommand(line) match { case Some(lc) => echo("\n" + lc.longHelp) @@ -258,7 +266,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** Show the history */ - lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { + private lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { override def usage = "[num]" def defaultLines = 20 @@ -279,21 +287,21 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // When you know you are most likely breaking into the middle // of a line being typed. This softens the blow. - protected def echoAndRefresh(msg: String) = { + private[repl] def echoAndRefresh(msg: String) = { echo("\n" + msg) in.redrawLine() } - protected def echo(msg: String) = { + private[repl] def echo(msg: String) = { out println msg out.flush() } - protected def echoNoNL(msg: String) = { + private def echoNoNL(msg: String) = { out print msg out.flush() } /** Search the history */ - def searchHistory(_cmdline: String) { + private def searchHistory(_cmdline: String) { val cmdline = _cmdline.toLowerCase val offset = history.index - history.size + 1 @@ -302,14 +310,27 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } private var currentPrompt = Properties.shellPromptString + + /** + * Sets the prompt string used by the REPL. + * + * @param prompt The new prompt string + */ + @DeveloperApi def setPrompt(prompt: String) = currentPrompt = prompt - /** Prompt to print when awaiting input */ + + /** + * Represents the current prompt string used by the REPL. + * + * @return The current prompt string + */ + @DeveloperApi def prompt = currentPrompt import LoopCommand.{ cmd, nullary } /** Standard commands */ - lazy val standardCommands = List( + private lazy val standardCommands = List( cmd("cp", "<path>", "add a jar or directory to the classpath", addClasspath), cmd("help", "[command]", "print this summary or command-specific help", helpCommand), historyCommand, @@ -333,7 +354,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, ) /** Power user commands */ - lazy val powerCommands: List[LoopCommand] = List( + private lazy val powerCommands: List[LoopCommand] = List( // cmd("phase", "<phase>", "set the implicit phase for power commands", phaseCommand) ) @@ -459,7 +480,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - protected def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { + private def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { override def tryClass(path: String): Array[Byte] = { val hd :: rest = path split '.' toList; // If there are dots in the name, the first segment is the @@ -581,7 +602,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // } // } - /** Available commands */ + /** + * Provides a list of available commands. + * + * @return The list of commands + */ + @DeveloperApi def commands: List[LoopCommand] = standardCommands /*++ ( if (isReplPower) powerCommands else Nil )*/ @@ -613,7 +639,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, * command() for each line of input, and stops when * command() returns false. */ - def loop() { + private def loop() { def readOneLine() = { out.flush() in readLine prompt @@ -642,7 +668,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** interpret all lines from a specified file */ - def interpretAllFrom(file: File) { + private def interpretAllFrom(file: File) { savingReader { savingReplayStack { file applyReader { reader => @@ -655,7 +681,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** create a new interpreter and replay the given commands */ - def replay() { + private def replay() { reset() if (replayCommandStack.isEmpty) echo("Nothing to replay.") @@ -665,7 +691,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, echo("") } } - def resetCommand() { + private def resetCommand() { echo("Resetting repl state.") if (replayCommandStack.nonEmpty) { echo("Forgetting this session history:\n") @@ -681,13 +707,13 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, reset() } - def reset() { + private def reset() { intp.reset() // unleashAndSetPhase() } /** fork a shell and run a command */ - lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { + private lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { override def usage = "<command line>" def apply(line: String): Result = line match { case "" => showUsage() @@ -698,14 +724,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - def withFile(filename: String)(action: File => Unit) { + private def withFile(filename: String)(action: File => Unit) { val f = File(filename) if (f.exists) action(f) else echo("That file does not exist") } - def loadCommand(arg: String) = { + private def loadCommand(arg: String) = { var shouldReplay: Option[String] = None withFile(arg)(f => { interpretAllFrom(f) @@ -714,7 +740,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, Result(true, shouldReplay) } - def addAllClasspath(args: Seq[String]): Unit = { + private def addAllClasspath(args: Seq[String]): Unit = { var added = false var totalClasspath = "" for (arg <- args) { @@ -729,7 +755,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - def addClasspath(arg: String): Unit = { + private def addClasspath(arg: String): Unit = { val f = File(arg).normalize if (f.exists) { addedClasspath = ClassPath.join(addedClasspath, f.path) @@ -741,12 +767,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } - def powerCmd(): Result = { + private def powerCmd(): Result = { if (isReplPower) "Already in power mode." else enablePowerMode(false) } - def enablePowerMode(isDuringInit: Boolean) = { + private[repl] def enablePowerMode(isDuringInit: Boolean) = { // replProps.power setValue true // unleashAndSetPhase() // asyncEcho(isDuringInit, power.banner) @@ -759,12 +785,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // } // } - def asyncEcho(async: Boolean, msg: => String) { + private def asyncEcho(async: Boolean, msg: => String) { if (async) asyncMessage(msg) else echo(msg) } - def verbosity() = { + private def verbosity() = { // val old = intp.printResults // intp.printResults = !old // echo("Switched " + (if (old) "off" else "on") + " result printing.") @@ -773,7 +799,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, /** Run one command submitted by the user. Two values are returned: * (1) whether to keep running, (2) the line to record for replay, * if any. */ - def command(line: String): Result = { + private[repl] def command(line: String): Result = { if (line startsWith ":") { val cmd = line.tail takeWhile (x => !x.isWhitespace) uniqueCommand(cmd) match { @@ -789,7 +815,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) } - def pasteCommand(): Result = { + private def pasteCommand(): Result = { echo("// Entering paste mode (ctrl-D to finish)\n") val code = readWhile(_ => true) mkString "\n" echo("\n// Exiting paste mode, now interpreting.\n") @@ -820,7 +846,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, * read, go ahead and interpret it. Return the full string * to be recorded for replay, if any. */ - def interpretStartingWith(code: String): Option[String] = { + private def interpretStartingWith(code: String): Option[String] = { // signal completion non-completion input has been received in.completion.resetVerbosity() @@ -874,7 +900,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } // runs :load `file` on any files passed via -i - def loadFiles(settings: Settings) = settings match { + private def loadFiles(settings: Settings) = settings match { case settings: SparkRunnerSettings => for (filename <- settings.loadfiles.value) { val cmd = ":load " + filename @@ -889,7 +915,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, * unless settings or properties are such that it should start * with SimpleReader. */ - def chooseReader(settings: Settings): InteractiveReader = { + private def chooseReader(settings: Settings): InteractiveReader = { if (settings.Xnojline.value || Properties.isEmacsShell) SimpleReader() else try new SparkJLineReader( @@ -903,8 +929,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - val m = u.runtimeMirror(Utils.getSparkClassLoader) + private val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + private val m = u.runtimeMirror(Utils.getSparkClassLoader) private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = u.TypeTag[T]( m, @@ -913,7 +939,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] }) - def process(settings: Settings): Boolean = savingContextLoader { + private def process(settings: Settings): Boolean = savingContextLoader { if (getMaster() == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") this.settings = settings @@ -972,6 +998,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, true } + // NOTE: Must be public for visibility + @DeveloperApi def createSparkContext(): SparkContext = { val execUri = System.getenv("SPARK_EXECUTOR_URI") val jars = SparkILoop.getAddedJars @@ -979,7 +1007,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, .setMaster(getMaster()) .setAppName("Spark shell") .setJars(jars) - .set("spark.repl.class.uri", intp.classServer.uri) + .set("spark.repl.class.uri", intp.classServerUri) if (execUri != null) { conf.set("spark.executor.uri", execUri) } @@ -1014,7 +1042,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } @deprecated("Use `process` instead", "2.9.0") - def main(settings: Settings): Unit = process(settings) + private def main(settings: Settings): Unit = process(settings) } object SparkILoop { @@ -1033,7 +1061,7 @@ object SparkILoop { // Designed primarily for use by test code: take a String with a // bunch of code, and prints out a transcript of what it would look // like if you'd just typed it into the repl. - def runForTranscript(code: String, settings: Settings): String = { + private[repl] def runForTranscript(code: String, settings: Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } stringFromStream { ostream => @@ -1071,7 +1099,7 @@ object SparkILoop { /** Creates an interpreter loop with default settings and feeds * the given code to it as input. */ - def run(code: String, sets: Settings = new Settings): String = { + private[repl] def run(code: String, sets: Settings = new Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } stringFromStream { ostream => @@ -1087,5 +1115,5 @@ object SparkILoop { } } } - def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) + private[repl] def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index da4286c5e4..99bd777c04 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -19,7 +19,7 @@ import org.apache.spark.SPARK_VERSION /** * Machinery for the asynchronous initialization of the repl. */ -trait SparkILoopInit { +private[repl] trait SparkILoopInit { self: SparkILoop => /** Print a welcome message */ 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 b646f0b6f0..35fb625645 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 @@ -39,6 +39,7 @@ import scala.util.control.ControlThrowable import org.apache.spark.{Logging, HttpServer, SecurityManager, SparkConf} import org.apache.spark.util.Utils +import org.apache.spark.annotation.DeveloperApi // /** directory to save .class files to */ // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { @@ -84,17 +85,18 @@ import org.apache.spark.util.Utils * @author Moez A. Abdel-Gawad * @author Lex Spoon */ + @DeveloperApi class SparkIMain( initialSettings: Settings, val out: JPrintWriter, propagateExceptions: Boolean = false) extends SparkImports with Logging { imain => - val conf = new SparkConf() + private val conf = new SparkConf() - val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") + private val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") /** Local directory to save .class files too */ - lazy val outputDir = { + private lazy val outputDir = { val tmp = System.getProperty("java.io.tmpdir") val rootDir = conf.get("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) @@ -103,13 +105,20 @@ import org.apache.spark.util.Utils echo("Output directory: " + outputDir) } - val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles + /** + * Returns the path to the output directory containing all generated + * class files that will be served by the REPL class server. + */ + @DeveloperApi + lazy val getClassOutputDirectory = outputDir + + private val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles /** Jetty server that will serve our classes to worker nodes */ - val classServerPort = conf.getInt("spark.replClassServer.port", 0) - val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") + 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 - var printResults = true // whether to print result lines - var totalSilence = false // whether to print anything + private var printResults = true // whether to print result lines + private var totalSilence = false // whether to print anything private var _initializeComplete = false // compiler is initialized private var _isInitialized: Future[Boolean] = null // set up initialization future private var bindExceptions = true // whether to bind the lastException variable @@ -123,6 +132,14 @@ import org.apache.spark.util.Utils 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 @@ -141,17 +158,18 @@ import org.apache.spark.util.Utils () => { counter += 1 ; counter } } - def compilerClasspath: Seq[URL] = ( + private def compilerClasspath: Seq[URL] = ( if (isInitializeComplete) global.classPath.asURLs else new PathResolver(settings).result.asURLs // the compiler's classpath ) - def settings = currentSettings - def mostRecentLine = prevRequestList match { + // NOTE: Exposed to repl package since accessed indirectly from SparkIMain + private[repl] def settings = currentSettings + private def mostRecentLine = prevRequestList match { case Nil => "" case req :: _ => req.originalLine } // Run the code body with the given boolean settings flipped to true. - def withoutWarnings[T](body: => T): T = beQuietDuring { + private def withoutWarnings[T](body: => T): T = beQuietDuring { val saved = settings.nowarn.value if (!saved) settings.nowarn.value = true @@ -164,16 +182,28 @@ import org.apache.spark.util.Utils def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) def this() = this(new Settings()) - lazy val repllog: Logger = new Logger { + private lazy val repllog: Logger = new Logger { val out: JPrintWriter = imain.out val isInfo: Boolean = BooleanProp keyExists "scala.repl.info" val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug" val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace" } - lazy val formatting: Formatting = new Formatting { + private[repl] lazy val formatting: Formatting = new Formatting { val prompt = Properties.shellPromptString } - lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + // NOTE: Exposed to repl package since used by SparkExprTyper and SparkILoop + private[repl] lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + /** + * Determines if errors were reported (typically during compilation). + * + * @note This is not for runtime errors + * + * @return True if had errors, otherwise false + */ + @DeveloperApi + def isReportingErrors = reporter.hasErrors import formatting._ import reporter.{ printMessage, withoutTruncating } @@ -193,7 +223,8 @@ import org.apache.spark.util.Utils private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" // argument is a thunk to execute after init is done - def initialize(postInitSignal: => Unit) { + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def initialize(postInitSignal: => Unit) { synchronized { if (_isInitialized == null) { _isInitialized = io.spawn { @@ -203,15 +234,27 @@ import org.apache.spark.util.Utils } } } + + /** + * Initializes the underlying compiler/interpreter in a blocking fashion. + * + * @note Must be executed before using SparkIMain! + */ + @DeveloperApi def initializeSynchronous(): Unit = { if (!isInitializeComplete) { _initialize() assert(global != null, global) } } - def isInitializeComplete = _initializeComplete + private def isInitializeComplete = _initializeComplete /** the public, go through the future compiler */ + + /** + * The underlying compiler used to generate ASTs and execute code. + */ + @DeveloperApi lazy val global: Global = { if (isInitializeComplete) _compiler else { @@ -226,13 +269,13 @@ import org.apache.spark.util.Utils } } @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - lazy val compiler: global.type = global + private lazy val compiler: global.type = global import global._ import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember} import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass} - implicit class ReplTypeOps(tp: Type) { + private implicit class ReplTypeOps(tp: Type) { def orElse(other: => Type): Type = if (tp ne NoType) tp else other def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) } @@ -240,7 +283,8 @@ import org.apache.spark.util.Utils // TODO: If we try to make naming a lazy val, we run into big time // scalac unhappiness with what look like cycles. It has not been easy to // reduce, but name resolution clearly takes different paths. - object naming extends { + // NOTE: Exposed to repl package since used by SparkExprTyper + private[repl] object naming extends { val global: imain.global.type = imain.global } with Naming { // make sure we don't overwrite their unwisely named res3 etc. @@ -254,22 +298,43 @@ import org.apache.spark.util.Utils } import naming._ - object deconstruct extends { + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] object deconstruct extends { val global: imain.global.type = imain.global } with StructuredTypeStrings - lazy val memberHandlers = new { + // NOTE: Exposed to repl package since used by SparkImports + private[repl] lazy val memberHandlers = new { val intp: imain.type = imain } with SparkMemberHandlers import memberHandlers._ - /** Temporarily be quiet */ + /** + * Suppresses overwriting print results during the operation. + * + * @param body The block to execute + * @tparam T The return type of the block + * + * @return The result from executing the block + */ + @DeveloperApi def beQuietDuring[T](body: => T): T = { val saved = printResults printResults = false try body finally printResults = saved } + + /** + * Completely masks all output during the operation (minus JVM standard + * out and error). + * + * @param operation The block to execute + * @tparam T The return type of the block + * + * @return The result from executing the block + */ + @DeveloperApi def beSilentDuring[T](operation: => T): T = { val saved = totalSilence totalSilence = true @@ -277,10 +342,10 @@ import org.apache.spark.util.Utils finally totalSilence = saved } - def quietRun[T](code: String) = beQuietDuring(interpret(code)) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { + private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { case t: ControlThrowable => throw t case t: Throwable => logDebug(label + ": " + unwrap(t)) @@ -298,14 +363,44 @@ import org.apache.spark.util.Utils finally bindExceptions = true } + /** + * Contains the code (in string form) representing a wrapper around all + * code executed by this instance. + * + * @return The wrapper code as a string + */ + @DeveloperApi def executionWrapper = _executionWrapper + + /** + * Sets the code to use as a wrapper around all code executed by this + * instance. + * + * @param code The wrapper code as a string + */ + @DeveloperApi def setExecutionWrapper(code: String) = _executionWrapper = code + + /** + * Clears the code used as a wrapper around all code executed by + * this instance. + */ + @DeveloperApi def clearExecutionWrapper() = _executionWrapper = "" /** interpreter settings */ - lazy val isettings = new SparkISettings(this) + private lazy val isettings = new SparkISettings(this) - /** Instantiate a compiler. Overridable. */ + /** + * Instantiates a new compiler used by SparkIMain. Overridable to provide + * own instance of a compiler. + * + * @param settings The settings to provide the compiler + * @param reporter The reporter to use for compiler output + * + * @return The compiler as a Global + */ + @DeveloperApi protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { settings.outputDirs setSingleOutput virtualDirectory settings.exposeEmptyPackage.value = true @@ -320,13 +415,14 @@ import org.apache.spark.util.Utils * @note Currently only supports jars, not directories * @param urls The list of items to add to the compile and runtime classpaths */ + @DeveloperApi def addUrlsToClassPath(urls: URL*): Unit = { new Run // Needed to force initialization of "something" to correctly load Scala classes from jars urls.foreach(_runtimeClassLoader.addNewUrl) // Add jars/classes to runtime for execution updateCompilerClassPath(urls: _*) // Add jars/classes to compile time for compiling } - protected def updateCompilerClassPath(urls: URL*): Unit = { + private def updateCompilerClassPath(urls: URL*): Unit = { require(!global.forMSIL) // Only support JavaPlatform val platform = global.platform.asInstanceOf[JavaPlatform] @@ -342,7 +438,7 @@ import org.apache.spark.util.Utils global.invalidateClassPathEntries(urls.map(_.getPath): _*) } - protected def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = { + private def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = { // Collect our new jars/directories and add them to the existing set of classpaths val allClassPaths = ( platform.classPath.asInstanceOf[MergedClassPath[AbstractFile]].entries ++ @@ -365,7 +461,13 @@ import org.apache.spark.util.Utils new MergedClassPath(allClassPaths, platform.classPath.context) } - /** Parent classloader. Overridable. */ + /** + * Represents the parent classloader used by this instance. Can be + * overridden to provide alternative classloader. + * + * @return The classloader used as the parent loader of this instance + */ + @DeveloperApi protected def parentClassLoader: ClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) @@ -382,16 +484,18 @@ import org.apache.spark.util.Utils shadow the old ones, and old code objects refer to the old definitions. */ - def resetClassLoader() = { + private def resetClassLoader() = { logDebug("Setting new classloader: was " + _classLoader) _classLoader = null ensureClassLoader() } - final def ensureClassLoader() { + private final def ensureClassLoader() { if (_classLoader == null) _classLoader = makeClassLoader() } - def classLoader: AbstractFileClassLoader = { + + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def classLoader: AbstractFileClassLoader = { ensureClassLoader() _classLoader } @@ -418,27 +522,58 @@ import org.apache.spark.util.Utils _runtimeClassLoader }) - def getInterpreterClassLoader() = classLoader + private def getInterpreterClassLoader() = classLoader // Set the current Java "context" class loader to this interpreter's class loader - def setContextClassLoader() = classLoader.setAsContext() + // NOTE: Exposed to repl package since used by SparkILoopInit + private[repl] 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 - * }}} + /** + * Returns the real name of a class based on its repl-defined name. + * + * ==Example== + * 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 + * }}} + * + * @param simpleName The repl-defined name whose real name to retrieve + * + * @return Some real name if the simple name exists, else None */ + @DeveloperApi def generatedName(simpleName: String): Option[String] = { if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING) else optFlatName(simpleName) } - def flatName(id: String) = optFlatName(id) getOrElse id - def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def flatName(id: String) = optFlatName(id) getOrElse id + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) + + /** + * Retrieves all simple names contained in the current instance. + * + * @return A list of sorted names + */ + @DeveloperApi def allDefinedNames = definedNameMap.keys.toList.sorted - def pathToType(id: String): String = pathToName(newTypeName(id)) - def pathToTerm(id: String): String = pathToName(newTermName(id)) + + private def pathToType(id: String): String = pathToName(newTypeName(id)) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def pathToTerm(id: String): String = pathToName(newTermName(id)) + + /** + * Retrieves the full code path to access the specified simple name + * content. + * + * @param name The simple name of the target whose path to determine + * + * @return The full path used to access the specified target (name) + */ + @DeveloperApi def pathToName(name: Name): String = { if (definedNameMap contains name) definedNameMap(name) fullPath name @@ -457,13 +592,13 @@ import org.apache.spark.util.Utils } /** Stubs for work in progress. */ - def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { + private def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) } } - def handleTermRedefinition(name: TermName, old: Request, req: Request) = { + private 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: <refinement> has owner value x, but a class owner is required @@ -473,7 +608,7 @@ import org.apache.spark.util.Utils } } - def recordRequest(req: Request) { + private def recordRequest(req: Request) { if (req == null || referencedNameMap == null) return @@ -504,12 +639,12 @@ import org.apache.spark.util.Utils } } - def replwarn(msg: => String) { + private def replwarn(msg: => String) { if (!settings.nowarnings.value) printMessage(msg) } - def isParseable(line: String): Boolean = { + private def isParseable(line: String): Boolean = { beSilentDuring { try parse(line) match { case Some(xs) => xs.nonEmpty // parses as-is @@ -522,22 +657,32 @@ import org.apache.spark.util.Utils } } - def compileSourcesKeepingRun(sources: SourceFile*) = { + private def compileSourcesKeepingRun(sources: SourceFile*) = { val run = new Run() reporter.reset() run compileSources sources.toList (!reporter.hasErrors, run) } - /** Compile an nsc SourceFile. Returns true if there are - * no compilation errors, or false otherwise. + /** + * Compiles specified source files. + * + * @param sources The sequence of source files to compile + * + * @return True if successful, otherwise false */ + @DeveloperApi def compileSources(sources: SourceFile*): Boolean = compileSourcesKeepingRun(sources: _*)._1 - /** Compile a string. Returns true if there are no - * compilation errors, or false otherwise. + /** + * Compiles a string of code. + * + * @param code The string of code to compile + * + * @return True if successful, otherwise false */ + @DeveloperApi def compileString(code: String): Boolean = compileSources(new BatchSourceFile("<script>", code)) @@ -562,7 +707,7 @@ import org.apache.spark.util.Utils private def safePos(t: Tree, alt: Int): Int = try t.pos.startOrPoint - catch { case _: UnsupportedOperationException => alt } + catch { case _: UnsupportedOperationException => alt } // Given an expression like 10 * 10 * 10 we receive the parent tree positioned // at a '*'. So look at each subtree and find the earliest of all positions. @@ -653,22 +798,43 @@ import org.apache.spark.util.Utils } // normalize non-public types so we don't see protected aliases like Self - def normalizeNonPublic(tp: Type) = tp match { + private def normalizeNonPublic(tp: Type) = tp match { case TypeRef(_, sym, _) if sym.isAliasType && !sym.isPublic => tp.dealias case _ => tp } /** - * Interpret one line of input. All feedback, including parse errors - * and evaluation results, are printed via the supplied compiler's - * reporter. Values defined are available for future interpreted strings. + * Interpret one line of input. All feedback, including parse errors + * and evaluation results, are printed via the supplied compiler's + * reporter. Values defined are available for future interpreted strings. + * + * @note This assigns variables with user name structure like "res0" + * + * @param line The line representing the code to interpret * - * The return value is whether the line was interpreter successfully, - * e.g. that there were no parse errors. + * @return Whether the line was interpreted successfully, or failed due to + * incomplete code, compilation error, or runtime error */ + @DeveloperApi def interpret(line: String): IR.Result = interpret(line, false) + + /** + * Interpret one line of input. All feedback, including parse errors + * and evaluation results, are printed via the supplied compiler's + * reporter. Values defined are available for future interpreted strings. + * + * @note This assigns variables with synthetic (generated) name structure + * like "$ires0" + * + * @param line The line representing the code to interpret + * + * @return Whether the line was interpreted successfully, or failed due to + * incomplete code, compilation error, or runtime error + */ + @DeveloperApi def interpretSynthetic(line: String): IR.Result = interpret(line, true) - def interpret(line: String, synthetic: Boolean): IR.Result = { + + private def interpret(line: String, synthetic: Boolean): IR.Result = { def loadAndRunReq(req: Request) = { classLoader.setAsContext() val (result, succeeded) = req.loadAndRun @@ -706,14 +872,20 @@ import org.apache.spark.util.Utils } } - /** Bind a specified name to a specified value. The name may - * later be used by expressions passed to interpret. + /** + * Bind a specified name to a specified value. The name may + * later be used by expressions passed to interpret. * - * @param name the variable name to bind - * @param boundType the type of the variable, as a string - * @param value the object value to bind to it - * @return an indication of whether the binding succeeded + * @note This binds via compilation and interpretation + * + * @param name The variable name to bind + * @param boundType The type of the variable, as a string + * @param value The object value to bind to it + * + * @return An indication of whether the binding succeeded or failed + * using interpreter results */ + @DeveloperApi def bind(name: String, boundType: String, value: Any, modifiers: List[String] = Nil): IR.Result = { val bindRep = new ReadEvalPrint() val run = bindRep.compile(""" @@ -735,15 +907,38 @@ import org.apache.spark.util.Utils interpret(line) } } + + /** + * Bind a specified name to a specified value directly. + * + * @note This updates internal bound names directly + * + * @param name The variable name to bind + * @param boundType The type of the variable, as a string + * @param value The object value to bind to it + * + * @return An indication of whether the binding succeeded or failed + * using interpreter results + */ + @DeveloperApi def directBind(name: String, boundType: String, value: Any): IR.Result = { val result = bind(name, boundType, value) if (result == IR.Success) directlyBoundNames += newTermName(name) result } - def directBind(p: NamedParam): IR.Result = directBind(p.name, p.tpe, p.value) - def directBind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = directBind((name, value)) + private def directBind(p: NamedParam): IR.Result = directBind(p.name, p.tpe, p.value) + private def directBind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = directBind((name, value)) + + /** + * Overwrites previously-bound val with a new instance. + * + * @param p The named parameters used to provide the name, value, and type + * + * @return The results of rebinding the named val + */ + @DeveloperApi def rebind(p: NamedParam): IR.Result = { val name = p.name val oldType = typeOfTerm(name) orElse { return IR.Error } @@ -753,19 +948,34 @@ import org.apache.spark.util.Utils quietRun("val %s = %s".format(tempName, name)) quietRun("val %s = %s.asInstanceOf[%s]".format(name, tempName, newType)) } - def quietImport(ids: String*): IR.Result = beQuietDuring(addImports(ids: _*)) + private def quietImport(ids: String*): IR.Result = beQuietDuring(addImports(ids: _*)) + + /** + * Executes an import statement per "id" provided + * + * @example addImports("org.apache.spark.SparkContext") + * + * @param ids The series of "id" strings used for import statements + * + * @return The results of importing the series of "id" strings + */ + @DeveloperApi def addImports(ids: String*): IR.Result = if (ids.isEmpty) IR.Success else interpret("import " + ids.mkString(", ")) - def quietBind(p: NamedParam): IR.Result = beQuietDuring(bind(p)) - def bind(p: NamedParam): IR.Result = bind(p.name, p.tpe, p.value) - def bind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = bind((name, value)) - def bindSyntheticValue(x: Any): IR.Result = bindValue(freshInternalVarName(), x) - def bindValue(x: Any): IR.Result = bindValue(freshUserVarName(), x) - def bindValue(name: String, x: Any): IR.Result = bind(name, TypeStrings.fromValue(x), x) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def quietBind(p: NamedParam): IR.Result = beQuietDuring(bind(p)) + private def bind(p: NamedParam): IR.Result = bind(p.name, p.tpe, p.value) + private def bind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = bind((name, value)) + private def bindSyntheticValue(x: Any): IR.Result = bindValue(freshInternalVarName(), x) + private def bindValue(x: Any): IR.Result = bindValue(freshUserVarName(), x) + private def bindValue(name: String, x: Any): IR.Result = bind(name, TypeStrings.fromValue(x), x) - /** Reset this interpreter, forgetting all user-specified requests. */ + /** + * Reset this interpreter, forgetting all user-specified requests. + */ + @DeveloperApi def reset() { clearExecutionWrapper() resetClassLoader() @@ -777,9 +987,11 @@ import org.apache.spark.util.Utils virtualDirectory.create() } - /** This instance is no longer needed, so release any resources - * it is using. The reporter's output gets flushed. + /** + * Stops the underlying REPL class server and flushes the reporter used + * for compiler output. */ + @DeveloperApi def close() { reporter.flush() classServer.stop() @@ -788,6 +1000,7 @@ import org.apache.spark.util.Utils /** * Captures the session names (which are set by system properties) once, instead of for each line. */ + @DeveloperApi object FixedSessionNames { val lineName = sessionNames.line val readName = sessionNames.read @@ -1129,10 +1342,13 @@ import org.apache.spark.util.Utils override def toString = "Request(line=%s, %s trees)".format(line, trees.size) } - /** Returns the name of the most recent interpreter result. - * Mostly this exists so you can conveniently invoke methods on - * the previous result. + /** + * Returns the name of the most recent interpreter result. Useful for + * for extracting information regarding the previous result. + * + * @return The simple name of the result (such as res0) */ + @DeveloperApi def mostRecentVar: String = if (mostRecentlyHandledTree.isEmpty) "" else "" + (mostRecentlyHandledTree.get match { @@ -1143,6 +1359,13 @@ import org.apache.spark.util.Utils }) private var mostRecentWarnings: List[(global.Position, String)] = Nil + + /** + * Returns a list of recent warnings from compiler execution. + * + * @return The list of tuples (compiler position, warning) + */ + @DeveloperApi def lastWarnings = mostRecentWarnings def treesForRequestId(id: Int): List[Tree] = @@ -1169,23 +1392,75 @@ import org.apache.spark.util.Utils req.handlers find (_.definedNames contains name) } + /** + * Retrieves the object representing the id (variable name, method name, + * class name, etc) provided. + * + * @param id The id (variable name, method name, class name, etc) whose + * associated content to retrieve + * + * @return Some containing term name (id) representation if exists, else None + */ + @DeveloperApi def valueOfTerm(id: String): Option[AnyRef] = requestForName(newTermName(id)) flatMap (_.getEval) + /** + * Retrieves the class representing the id (variable name, method name, + * class name, etc) provided. + * + * @param id The id (variable name, method name, class name, etc) whose + * associated class to retrieve + * + * @return Some containing term name (id) class if exists, else None + */ + @DeveloperApi def classOfTerm(id: String): Option[JClass] = valueOfTerm(id) map (_.getClass) + /** + * Retrieves the type representing the id (variable name, method name, + * class name, etc) provided. + * + * @param id The id (variable name, method name, class name, etc) whose + * associated type to retrieve + * + * @return The Type information about the term name (id) provided + */ + @DeveloperApi def typeOfTerm(id: String): Type = newTermName(id) match { case nme.ROOTPKG => RootClass.tpe case name => requestForName(name).fold(NoType: Type)(_ compilerTypeOf name) } - def symbolOfType(id: String): Symbol = - requestForName(newTypeName(id)).fold(NoSymbol: Symbol)(_ definedTypeSymbol id) - + /** + * Retrieves the symbol representing the id (variable name, method name, + * class name, etc) provided. + * + * @param id The id (variable name, method name, class name, etc) whose + * associated symbol to retrieve + * + * @return The Symbol information about the term name (id) provided + */ + @DeveloperApi def symbolOfTerm(id: String): Symbol = requestForIdent(newTermName(id)).fold(NoSymbol: Symbol)(_ definedTermSymbol id) + // TODO: No use yet, but could be exposed as a DeveloperApi + private def symbolOfType(id: String): Symbol = + requestForName(newTypeName(id)).fold(NoSymbol: Symbol)(_ definedTypeSymbol id) + + /** + * Retrieves the runtime class and type representing the id (variable name, + * method name, class name, etc) provided. + * + * @param id The id (variable name, method name, class name, etc) whose + * associated runtime class and type to retrieve + * + * @return Some runtime class and Type information as a tuple for the + * provided term name if it exists, else None + */ + @DeveloperApi def runtimeClassAndTypeOfTerm(id: String): Option[(JClass, Type)] = { classOfTerm(id) flatMap { clazz => new RichClass(clazz).supers find(c => !(new RichClass(c).isScalaAnonymous)) map { nonAnon => @@ -1194,6 +1469,16 @@ import org.apache.spark.util.Utils } } + /** + * Retrieves the runtime type representing the id (variable name, + * method name, class name, etc) provided. + * + * @param id The id (variable name, method name, class name, etc) whose + * associated runtime type to retrieve + * + * @return The runtime Type information about the term name (id) provided + */ + @DeveloperApi def runtimeTypeOfTerm(id: String): Type = { typeOfTerm(id) andAlso { tpe => val clazz = classOfTerm(id) getOrElse { return NoType } @@ -1205,7 +1490,8 @@ import org.apache.spark.util.Utils else NoType } } - def cleanMemberDecl(owner: Symbol, member: Name): Type = afterTyper { + + private def cleanMemberDecl(owner: Symbol, member: Name): Type = afterTyper { normalizeNonPublic { owner.info.nonPrivateDecl(member).tpe match { case NullaryMethodType(tp) => tp @@ -1214,50 +1500,125 @@ import org.apache.spark.util.Utils } } - object exprTyper extends { + private object exprTyper extends { val repl: SparkIMain.this.type = imain } with SparkExprTyper { } + /** + * Constructs a list of abstract syntax trees representing the provided code. + * + * @param line The line of code to parse and construct into ASTs + * + * @return Some list of ASTs if the line is valid, else None + */ + @DeveloperApi def parse(line: String): Option[List[Tree]] = exprTyper.parse(line) + /** + * Constructs a Symbol representing the final result of the expression + * provided or representing the definition provided. + * + * @param code The line of code + * + * @return The Symbol or NoSymbol (found under scala.reflect.internal) + */ + @DeveloperApi def symbolOfLine(code: String): Symbol = exprTyper.symbolOfLine(code) + /** + * Constucts type information based on the provided expression's final + * result or the definition provided. + * + * @param expr The expression or definition + * + * @param silent Whether to output information while constructing the type + * + * @return The type information or an error + */ + @DeveloperApi def typeOfExpression(expr: String, silent: Boolean = true): Type = exprTyper.typeOfExpression(expr, silent) protected def onlyTerms(xs: List[Name]) = xs collect { case x: TermName => x } protected def onlyTypes(xs: List[Name]) = xs collect { case x: TypeName => x } + /** + * Retrieves the defined, public names in the compiler. + * + * @return The list of matching "term" names + */ + @DeveloperApi def definedTerms = onlyTerms(allDefinedNames) filterNot isInternalTermName + + /** + * Retrieves the defined type names in the compiler. + * + * @return The list of matching type names + */ + @DeveloperApi def definedTypes = onlyTypes(allDefinedNames) + + /** + * Retrieves the defined symbols in the compiler. + * + * @return The set of matching Symbol instances + */ + @DeveloperApi def definedSymbols = prevRequestList.flatMap(_.definedSymbols.values).toSet[Symbol] + + /** + * Retrieves the list of public symbols in the compiler. + * + * @return The list of public Symbol instances + */ + @DeveloperApi def definedSymbolList = prevRequestList flatMap (_.definedSymbolList) filterNot (s => isInternalTermName(s.name)) // Terms with user-given names (i.e. not res0 and not synthetic) - def namedDefinedTerms = definedTerms filterNot (x => isUserVarName("" + x) || directlyBoundNames(x)) + + /** + * Retrieves defined, public names that are not res0 or the result of a direct bind. + * + * @return The list of matching "term" names + */ + @DeveloperApi + def namedDefinedTerms = definedTerms filterNot (x => isUserVarName("" + x) || directlyBoundNames(x)) private def findName(name: Name) = definedSymbols find (_.name == name) getOrElse NoSymbol /** Translate a repl-defined identifier into a Symbol. */ - def apply(name: String): Symbol = + private def apply(name: String): Symbol = types(name) orElse terms(name) - def types(name: String): Symbol = { + private def types(name: String): Symbol = { val tpname = newTypeName(name) findName(tpname) orElse getClassIfDefined(tpname) } - def terms(name: String): Symbol = { + private def terms(name: String): Symbol = { val termname = newTypeName(name) findName(termname) orElse getModuleIfDefined(termname) } // [Eugene to Paul] possibly you could make use of TypeTags here - def types[T: ClassTag] : Symbol = types(classTag[T].runtimeClass.getName) - def terms[T: ClassTag] : Symbol = terms(classTag[T].runtimeClass.getName) - def apply[T: ClassTag] : Symbol = apply(classTag[T].runtimeClass.getName) + private def types[T: ClassTag] : Symbol = types(classTag[T].runtimeClass.getName) + private def terms[T: ClassTag] : Symbol = terms(classTag[T].runtimeClass.getName) + private def apply[T: ClassTag] : Symbol = apply(classTag[T].runtimeClass.getName) + /** + * Retrieves the Symbols representing classes in the compiler. + * + * @return The list of matching ClassSymbol instances + */ + @DeveloperApi def classSymbols = allDefSymbols collect { case x: ClassSymbol => x } + + /** + * Retrieves the Symbols representing methods in the compiler. + * + * @return The list of matching MethodSymbol instances + */ + @DeveloperApi def methodSymbols = allDefSymbols collect { case x: MethodSymbol => x } /** the previous requests this interpreter has processed */ @@ -1267,32 +1628,41 @@ import org.apache.spark.util.Utils private val definedNameMap = mutable.Map[Name, Request]() private val directlyBoundNames = mutable.Set[Name]() - def allHandlers = prevRequestList flatMap (_.handlers) - def allDefHandlers = allHandlers collect { case x: MemberDefHandler => x } - def allDefSymbols = allDefHandlers map (_.symbol) filter (_ ne NoSymbol) + private def allHandlers = prevRequestList flatMap (_.handlers) + private def allDefHandlers = allHandlers collect { case x: MemberDefHandler => x } + private def allDefSymbols = allDefHandlers map (_.symbol) filter (_ ne NoSymbol) - def lastRequest = if (prevRequests.isEmpty) null else prevRequests.last - def prevRequestList = prevRequests.toList - def allSeenTypes = prevRequestList flatMap (_.typeOf.values.toList) distinct - def allImplicits = allHandlers filter (_.definesImplicit) flatMap (_.definedNames) - def importHandlers = allHandlers collect { case x: ImportHandler => x } + private def lastRequest = if (prevRequests.isEmpty) null else prevRequests.last + // NOTE: Exposed to repl package since used by SparkImports + private[repl] def prevRequestList = prevRequests.toList + private def allSeenTypes = prevRequestList flatMap (_.typeOf.values.toList) distinct + private def allImplicits = allHandlers filter (_.definesImplicit) flatMap (_.definedNames) + // NOTE: Exposed to repl package since used by SparkILoop and SparkImports + private[repl] def importHandlers = allHandlers collect { case x: ImportHandler => x } + /** + * Retrieves a list of unique defined and imported names in the compiler. + * + * @return The list of "term" names + */ def visibleTermNames: List[Name] = definedTerms ++ importedTerms distinct /** Another entry point for tab-completion, ids in scope */ - def unqualifiedIds = visibleTermNames map (_.toString) filterNot (_ contains "$") sorted + // NOTE: Exposed to repl package since used by SparkJLineCompletion + private[repl] def unqualifiedIds = visibleTermNames map (_.toString) filterNot (_ contains "$") sorted /** Parse the ScalaSig to find type aliases */ - def aliasForType(path: String) = ByteCode.aliasForType(path) + private def aliasForType(path: String) = ByteCode.aliasForType(path) - def withoutUnwrapping(op: => Unit): Unit = { + private def withoutUnwrapping(op: => Unit): Unit = { val saved = isettings.unwrapStrings isettings.unwrapStrings = false try op finally isettings.unwrapStrings = saved } - def symbolDefString(sym: Symbol) = { + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def symbolDefString(sym: Symbol) = { TypeStrings.quieter( afterTyper(sym.defString), sym.owner.name + ".this.", @@ -1300,7 +1670,7 @@ import org.apache.spark.util.Utils ) } - def showCodeIfDebugging(code: String) { + private def showCodeIfDebugging(code: String) { /** Secret bookcase entrance for repl debuggers: end the line * with "// show" and see what's going on. */ @@ -1319,7 +1689,9 @@ import org.apache.spark.util.Utils } // debugging - def debugging[T](msg: String)(res: T) = { + // NOTE: Exposed to repl package since accessed indirectly from SparkIMain + // and SparkJLineCompletion + private[repl] def debugging[T](msg: String)(res: T) = { logDebug(msg + " " + res) res } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala index 193a42dcde..1d0fe10d3d 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -12,7 +12,7 @@ import scala.tools.nsc.interpreter._ import scala.collection.{ mutable, immutable } -trait SparkImports { +private[repl] trait SparkImports { self: SparkIMain => import global._ diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala index 3159b70008..f24d6da724 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala @@ -7,6 +7,8 @@ package org.apache.spark.repl +import org.apache.spark.annotation.DeveloperApi + import scala.tools.nsc._ import scala.tools.nsc.interpreter._ @@ -16,27 +18,45 @@ import Completion._ import scala.collection.mutable.ListBuffer import org.apache.spark.Logging -// REPL completor - queries supplied interpreter for valid -// completions based on current contents of buffer. +/** + * Represents an auto-completion tool for the supplied interpreter that + * utilizes supplied queries for valid completions based on the current + * contents of the internal buffer. + * + * @param intp The interpreter to use for information retrieval to do with + * auto completion + */ +@DeveloperApi class SparkJLineCompletion(val intp: SparkIMain) extends Completion with CompletionOutput with Logging { + // NOTE: Exposed in package as used in quite a few classes + // NOTE: Must be public to override the global found in CompletionOutput val global: intp.global.type = intp.global + import global._ import definitions.{ PredefModule, AnyClass, AnyRefClass, ScalaPackage, JavaLangPackage } import rootMirror.{ RootClass, getModuleIfDefined } type ExecResult = Any import intp.{ debugging } - // verbosity goes up with consecutive tabs - private var verbosity: Int = 0 + /** + * Represents the level of verbosity. Increments with consecutive tabs. + */ + @DeveloperApi + var verbosity: Int = 0 + + /** + * Resets the level of verbosity to zero. + */ + @DeveloperApi def resetVerbosity() = verbosity = 0 - def getSymbol(name: String, isModule: Boolean) = ( + private def getSymbol(name: String, isModule: Boolean) = ( if (isModule) getModuleIfDefined(name) else getModuleIfDefined(name) ) - def getType(name: String, isModule: Boolean) = getSymbol(name, isModule).tpe - def typeOf(name: String) = getType(name, false) - def moduleOf(name: String) = getType(name, true) + private def getType(name: String, isModule: Boolean) = getSymbol(name, isModule).tpe + private def typeOf(name: String) = getType(name, false) + private def moduleOf(name: String) = getType(name, true) trait CompilerCompletion { def tp: Type @@ -258,12 +278,12 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet // the list of completion aware objects which should be consulted // for top level unqualified, it's too noisy to let much in. - lazy val topLevelBase: List[CompletionAware] = List(ids, rootClass, predef, scalalang, javalang, literals) - def topLevel = topLevelBase ++ imported - def topLevelThreshold = 50 + private lazy val topLevelBase: List[CompletionAware] = List(ids, rootClass, predef, scalalang, javalang, literals) + private def topLevel = topLevelBase ++ imported + private def topLevelThreshold = 50 // the first tier of top level objects (doesn't include file completion) - def topLevelFor(parsed: Parsed): List[String] = { + private def topLevelFor(parsed: Parsed): List[String] = { val buf = new ListBuffer[String] topLevel foreach { ca => buf ++= (ca completionsFor parsed) @@ -275,9 +295,9 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet } // the most recent result - def lastResult = Forwarder(() => ids follow intp.mostRecentVar) + private def lastResult = Forwarder(() => ids follow intp.mostRecentVar) - def lastResultFor(parsed: Parsed) = { + private def lastResultFor(parsed: Parsed) = { /** The logic is a little tortured right now because normally '.' is * ignored as a delimiter, but on .<tab> it needs to be propagated. */ @@ -286,9 +306,15 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet } // generic interface for querying (e.g. interpreter loop, testing) - def completions(buf: String): List[String] = + private def completions(buf: String): List[String] = topLevelFor(Parsed.dotted(buf + ".", buf.length + 1)) + /** + * Constructs a new ScalaCompleter for auto completion. + * + * @return The new JLineTabCompletion instance + */ + @DeveloperApi def completer(): ScalaCompleter = new JLineTabCompletion /** This gets a little bit hairy. It's no small feat delegating everything diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala index 0db26c3407..016e0f039f 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala @@ -22,7 +22,7 @@ import io.Streamable.slurp /** * Reads from the console using JLine. */ -class SparkJLineReader(_completion: => Completion) extends InteractiveReader { +private[repl] class SparkJLineReader(_completion: => Completion) extends InteractiveReader { val interactive = true val consoleReader = new JLineConsoleReader() @@ -82,7 +82,7 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { } /** Changes the default history file to not collide with the scala repl's. */ -class SparkJLineHistory extends JLineFileHistory { +private[repl] class SparkJLineHistory extends JLineFileHistory { import Properties.userHome def defaultFileName = ".spark_history" diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala index 13cd2b7fa5..4de9714247 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala @@ -16,7 +16,7 @@ import scala.reflect.internal.Chars import scala.reflect.internal.Flags._ import scala.language.implicitConversions -trait SparkMemberHandlers { +private[repl] trait SparkMemberHandlers { val intp: SparkIMain import intp.{ Request, global, naming } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala index 7fd5fbb424..94c801ebec 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala @@ -23,8 +23,7 @@ import scala.tools.nsc.Settings * <i>scala.tools.nsc.Settings</i> implementation adding Spark-specific REPL * command line options. */ -class SparkRunnerSettings(error: String => Unit) extends Settings(error){ - +private[repl] class SparkRunnerSettings(error: String => Unit) extends Settings(error) { val loadfiles = MultiStringSetting( "-i", "file", |