aboutsummaryrefslogtreecommitdiff
path: root/repl
diff options
context:
space:
mode:
Diffstat (limited to 'repl')
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala9
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala2
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala17
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala150
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala2
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala592
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala2
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala56
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala4
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala2
-rw-r--r--repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala3
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",