From 25c0739bad7222d45b4818c7bf6987521a3509d2 Mon Sep 17 00:00:00 2001 From: folone Date: Mon, 14 Jan 2013 09:52:11 +0100 Subject: Moved to scala 2.10.0. Notable changes are: - akka 2.0.3 → 2.1.0 - spray 1.0-M1 → 1.1-M7 For now the repl subproject is commented out, as scala reflection api changed very much since the introduction of macros. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/scala/spark/CacheTracker.scala | 6 +- core/src/main/scala/spark/MapOutputTracker.scala | 6 +- .../scala/spark/api/java/function/Function.java | 8 +-- .../scala/spark/api/java/function/Function2.java | 9 +-- .../api/java/function/PairFlatMapFunction.java | 12 ++-- .../spark/api/java/function/PairFunction.java | 12 ++-- .../src/main/scala/spark/deploy/JsonProtocol.scala | 2 +- .../main/scala/spark/deploy/client/Client.scala | 4 +- .../main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/deploy/master/MasterWebUI.scala | 45 +++++++------- .../main/scala/spark/deploy/worker/Worker.scala | 2 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 43 ++++++------- .../scala/spark/network/ConnectionManager.scala | 6 +- .../spark/network/ConnectionManagerTest.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 4 +- .../main/scala/spark/storage/BlockManager.scala | 6 +- .../scala/spark/storage/BlockManagerMaster.scala | 5 +- .../spark/storage/BlockManagerMasterActor.scala | 4 +- core/src/main/scala/spark/util/AkkaUtils.scala | 37 ++++++------ core/src/test/scala/spark/CacheTrackerSuite.scala | 11 ++-- project/SparkBuild.scala | 70 +++++++++++----------- project/build.properties | 2 +- project/plugins.sbt | 6 +- 23 files changed, 155 insertions(+), 151 deletions(-) diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala index 04c26b2e40..a73438208a 100644 --- a/core/src/main/scala/spark/CacheTracker.scala +++ b/core/src/main/scala/spark/CacheTracker.scala @@ -5,12 +5,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import akka.actor._ -import akka.dispatch._ +import scala.concurrent.Await import akka.pattern.ask import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ import spark.storage.BlockManager import spark.storage.StorageLevel diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 70eb9f702e..08d2956782 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -8,12 +8,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import akka.actor._ -import akka.dispatch._ +import scala.concurrent.Await import akka.pattern.ask import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ import spark.scheduler.MapStatus import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java index dae8295f21..9f6ab9a592 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/spark/api/java/function/Function.java @@ -1,7 +1,7 @@ package spark.api.java.function; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -15,8 +15,8 @@ import java.io.Serializable; public abstract class Function extends WrappedFunction1 implements Serializable { public abstract R call(T t) throws Exception; - public ClassManifest returnType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag returnType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java index 69bf12c8c9..b32c178aa3 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/spark/api/java/function/Function2.java @@ -1,7 +1,7 @@ package spark.api.java.function; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction2; import java.io.Serializable; @@ -14,8 +14,9 @@ public abstract class Function2 extends WrappedFunction2 public abstract R call(T1 t1, T2 t2) throws Exception; - public ClassManifest returnType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag returnType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } + } diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java index b3cc4df6aa..296a3b5044 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java @@ -1,8 +1,8 @@ package spark.api.java.function; import scala.Tuple2; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -19,11 +19,11 @@ public abstract class PairFlatMapFunction public abstract Iterable> call(T t) throws Exception; - public ClassManifest keyType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag keyType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } - public ClassManifest valueType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag valueType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java index 9fc6df4b88..e3f94788e2 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFunction.java @@ -1,8 +1,8 @@ package spark.api.java.function; import scala.Tuple2; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -18,11 +18,11 @@ public abstract class PairFunction public abstract Tuple2 call(T t) throws Exception; - public ClassManifest keyType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag keyType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } - public ClassManifest valueType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag valueType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 732fa08064..12069947fc 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -2,7 +2,7 @@ package spark.deploy import master.{JobInfo, WorkerInfo} import worker.ExecutorRunner -import cc.spray.json._ +import spray.json._ /** * spray-json helper class containing implicit conversion to json for marshalling responses diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 90fe9508cd..0aee759796 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -3,7 +3,7 @@ package spark.deploy.client import spark.deploy._ import akka.actor._ import akka.pattern.ask -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.AskTimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent @@ -11,7 +11,7 @@ import akka.remote.RemoteClientShutdown import spark.deploy.RegisterJob import akka.remote.RemoteClientDisconnected import akka.actor.Terminated -import akka.dispatch.Await +import scala.concurrent.Await /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description, diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 6ecebe626a..e034312c12 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -49,7 +49,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } def startWebUi() { - val webUi = new MasterWebUI(context.system, self) + val webUi = new MasterWebUI(self) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 458ee2d665..a4dadb6ef9 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,39 +1,42 @@ package spark.deploy.master -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await +import akka.actor.{ActorRef, ActorContext, ActorRefFactory} +import scala.concurrent.Await import akka.pattern.ask import akka.util.Timeout -import akka.util.duration._ -import cc.spray.Directives -import cc.spray.directives._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ +import scala.concurrent.duration._ +import spray.routing.Directives +import spray.routing.directives._ +import spray.httpx.TwirlSupport._ +import spray.httpx.SprayJsonSupport._ +import spray.http.MediaTypes._ import spark.deploy._ import spark.deploy.JsonProtocol._ private[spark] -class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { +class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends Directives { + import context.dispatcher + + val actorSystem = context.system val RESOURCE_DIR = "spark/deploy/master/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - + implicit val timeout = Timeout(1 seconds) - + val handler = { get { (path("") & parameters('format ?)) { case Some(js) if js.equalsIgnoreCase("json") => - val future = master ? RequestMasterState - respondWithMediaType(MediaTypes.`application/json`) { ctx => + val future = (master ? RequestMasterState).mapTo[MasterState] + respondWithMediaType(`application/json`) { ctx => ctx.complete(future.mapTo[MasterState]) } case _ => - completeWith { - val future = master ? RequestMasterState + complete { + val future = (master ? RequestMasterState).mapTo[MasterState] future.map { - masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) + masterState => spark.deploy.master.html.index.render(masterState) } } } ~ @@ -50,15 +53,13 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct } } } - respondWithMediaType(MediaTypes.`application/json`) { ctx => + respondWithMediaType(`application/json`) { ctx => ctx.complete(jobInfo.mapTo[JobInfo]) } case (jobId, _) => - completeWith { - val future = master ? RequestMasterState - future.map { state => - val masterState = state.asInstanceOf[MasterState] - + complete { + val future = (master ? RequestMasterState).mapTo[MasterState] + future.map { masterState => masterState.activeJobs.find(_.id == jobId) match { case Some(job) => spark.deploy.master.html.job_details.render(job) case _ => masterState.completedJobs.find(_.id == jobId) match { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 7c9e588ea2..ec25a19e7b 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -100,7 +100,7 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self) + val webUi = new WorkerWebUI(self) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index f9489d99fc..7dd1781900 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -1,46 +1,49 @@ package spark.deploy.worker -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await +import akka.actor.{ActorRef, ActorContext} +import scala.concurrent.Await import akka.pattern.ask import akka.util.Timeout -import akka.util.duration._ -import cc.spray.Directives -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ +import scala.concurrent.duration._ +import spray.routing.Directives +import spray.httpx.TwirlSupport._ +import spray.httpx.SprayJsonSupport._ +import spray.http.MediaTypes._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ private[spark] -class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { - val RESOURCE_DIR = "spark/deploy/worker/webui" +class WorkerWebUI(worker: ActorRef)(implicit val context: ActorContext) extends Directives { + import context.dispatcher + + val actorSystem = context.system + val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - + implicit val timeout = Timeout(1 seconds) - + val handler = { get { - (path("") & parameters('format ?)) { + (path("") & parameters('format ?)) { case Some(js) if js.equalsIgnoreCase("json") => { - val future = worker ? RequestWorkerState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[WorkerState]) + val future = (worker ? RequestWorkerState).mapTo[WorkerState] + respondWithMediaType(`application/json`) { ctx => + ctx.complete(future) } } case _ => - completeWith{ - val future = worker ? RequestWorkerState + complete { + val future = (worker ? RequestWorkerState).mapTo[WorkerState] future.map { workerState => - spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) + spark.deploy.worker.html.index(workerState) } } } ~ path("log") { parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => - respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName("work/" + jobId + "/" + executorId + "/" + logType) + respondWithMediaType(`text/plain`) { + getFromFile("work/" + jobId + "/" + executorId + "/" + logType) } } } ~ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 36c01ad629..04b303afe0 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -14,9 +14,9 @@ import scala.collection.mutable.SynchronizedQueue import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer -import akka.dispatch.{Await, Promise, ExecutionContext, Future} -import akka.util.Duration -import akka.util.duration._ +import scala.concurrent.{Await, Promise, ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.duration._ private[spark] case class ConnectionManagerId(host: String, port: Int) { def toSocketAddress() = new InetSocketAddress(host, port) diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 533e4610f3..1b5b0935c2 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -8,8 +8,8 @@ import scala.io.Source import java.nio.ByteBuffer import java.net.InetAddress -import akka.dispatch.Await -import akka.util.duration._ +import scala.concurrent.Await +import scala.concurrent.duration._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index eeaae23dc8..03dc5f4b9b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -3,11 +3,11 @@ package spark.scheduler.cluster import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.ask import spark.{SparkException, Logging, TaskState} -import akka.dispatch.Await +import scala.concurrent.Await import java.util.concurrent.atomic.AtomicInteger import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a8ac10cdd..7a1344668f 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -8,9 +8,9 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} -import akka.dispatch.{Await, Future} -import akka.util.Duration -import akka.util.duration._ +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index a3d8671834..46e1860d09 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -4,10 +4,9 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import akka.dispatch.Await +import scala.concurrent.Await import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, SparkException, Utils} diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index f4d026da33..9bf9161c31 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -7,8 +7,7 @@ import scala.collection.JavaConversions._ import scala.util.Random import akka.actor.{Actor, ActorRef, Cancellable} -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, Utils} @@ -42,6 +41,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { override def preStart() { if (!BlockManager.getDisableHeartBeatsForTesting) { + import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule( 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e67cb0336d..6fd9aa70fc 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,16 +1,16 @@ package spark.util -import akka.actor.{Props, ActorSystemImpl, ActorSystem} +import akka.actor.{Props, ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.ask import akka.remote.RemoteActorRefProvider -import cc.spray.Route -import cc.spray.io.IoWorker -import cc.spray.{SprayCanRootService, HttpService} -import cc.spray.can.server.HttpServer -import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler -import akka.dispatch.Await +import spray.routing.Route +import spray.io.IOExtension +import spray.routing.HttpServiceActor +import spray.can.server.{HttpServer, ServerSettings} +import spray.io.SingletonHandler +import scala.concurrent.Await import spark.SparkException import java.util.concurrent.TimeoutException @@ -23,11 +23,11 @@ private[spark] object AkkaUtils { * ActorSystem itself and its port (which is hard to get from Akka). */ def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { - val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt + val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt + val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt - val akkaConf = ConfigFactory.parseString(""" + val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] akka.actor.provider = "akka.remote.RemoteActorRefProvider" @@ -44,7 +44,7 @@ private[spark] object AkkaUtils { // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. - val provider = actorSystem.asInstanceOf[ActorSystemImpl].provider + val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get return (actorSystem, boundPort) } @@ -54,14 +54,13 @@ private[spark] object AkkaUtils { * handle requests. Throws a SparkException if this fails. */ def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) { - val ioWorker = new IoWorker(actorSystem).start() - val httpService = actorSystem.actorOf(Props(new HttpService(route))) - val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) - val server = actorSystem.actorOf( - Props(new HttpServer(ioWorker, SingletonHandler(rootService))), name = "HttpServer") - actorSystem.registerOnTermination { ioWorker.stop() } + val ioWorker = IOExtension(actorSystem).ioBridge() + val httpService = actorSystem.actorOf(Props(HttpServiceActor(route))) + val server = actorSystem.actorOf( + Props(new HttpServer(ioWorker, SingletonHandler(httpService), ServerSettings())), name = "HttpServer") + actorSystem.registerOnTermination { actorSystem.stop(ioWorker) } val timeout = 3.seconds - val future = server.ask(HttpServer.Bind(ip, port))(timeout) + val future = server.ask(HttpServer.Bind(ip, port))(timeout) try { Await.result(future, timeout) match { case bound: HttpServer.Bound => diff --git a/core/src/test/scala/spark/CacheTrackerSuite.scala b/core/src/test/scala/spark/CacheTrackerSuite.scala index 467605981b..d0c2bd47fc 100644 --- a/core/src/test/scala/spark/CacheTrackerSuite.scala +++ b/core/src/test/scala/spark/CacheTrackerSuite.scala @@ -5,20 +5,19 @@ import org.scalatest.FunSuite import scala.collection.mutable.HashMap import akka.actor._ -import akka.dispatch._ -import akka.pattern.ask +import scala.concurrent.{Await, Future} import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ class CacheTrackerSuite extends FunSuite { // Send a message to an actor and wait for a reply, in a blocking manner private def ask(actor: ActorRef, message: Any): Any = { try { val timeout = 10.seconds - val future = actor.ask(message)(timeout) - return Await.result(future, timeout) + val future: Future[Any] = akka.pattern.ask(actor, message)(timeout) + Await.result(future, timeout) } catch { case e: Exception => throw new SparkException("Error communicating with actor", e) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 219674028e..d0b3c350f1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,11 +17,11 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, /*repl,*/ examples, bagel) lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) +// lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) @@ -32,10 +32,10 @@ object SparkBuild extends Build { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") def sharedSettings = Defaults.defaultSettings ++ Seq( - organization := "org.spark-project", - version := "0.7.0-SNAPSHOT", - scalaVersion := "2.9.2", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + organization := "org.spark-project", + version := "0.7.0-SNAPSHOT", + scalaVersion := "2.10.0", + scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -87,11 +87,11 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.8" % "test", - "org.scalacheck" %% "scalacheck" % "1.9" % "test", - "com.novocode" % "junit-interface" % "0.8" % "test" - ), + "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", + "com.novocode" % "junit-interface" % "0.8" % "test" + ), parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, @@ -112,31 +112,33 @@ object SparkBuild extends Build { name := "spark-core", resolvers ++= Seq( "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Spray Repository" at "http://repo.spray.cc/", + "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", + "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "11.0.1", - "log4j" % "log4j" % "1.2.16", - "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "com.ning" % "compress-lzf" % "0.8.4", - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, - "asm" % "asm-all" % "3.3.1", - "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.20", - "com.typesafe.akka" % "akka-actor" % "2.0.3", - "com.typesafe.akka" % "akka-remote" % "2.0.3", - "com.typesafe.akka" % "akka-slf4j" % "2.0.3", - "it.unimi.dsi" % "fastutil" % "6.4.4", - "colt" % "colt" % "1.2.0", - "cc.spray" % "spray-can" % "1.0-M2.1", - "cc.spray" % "spray-server" % "1.0-M2.1", - "cc.spray" %% "spray-json" % "1.1.1", - "org.apache.mesos" % "mesos" % "0.9.0-incubating" - ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, + "com.google.guava" % "guava" % "11.0.1", + "log4j" % "log4j" % "1.2.16", + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "com.ning" % "compress-lzf" % "0.8.4", + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, + "asm" % "asm-all" % "3.3.1", + "com.google.protobuf" % "protobuf-java" % "2.4.1", + "de.javakaffee" % "kryo-serializers" % "0.20", + "com.typesafe.akka" %% "akka-remote" % "2.1.0", + "com.typesafe.akka" %% "akka-slf4j" % "2.1.0", + "it.unimi.dsi" % "fastutil" % "6.4.4", + "io.spray" % "spray-can" % "1.1-M7", + "io.spray" % "spray-io" % "1.1-M7", + "io.spray" % "spray-routing" % "1.1-M7", + "io.spray" %% "spray-json" % "1.2.3", + "colt" % "colt" % "1.2.0", + "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "org.scala-lang" % "scala-actors" % "2.10.0" + ) ++ (if (HADOOP_MAJOR_VERSION == "2") + Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings @@ -144,10 +146,10 @@ object SparkBuild extends Build { publish := {} ) - def replSettings = sharedSettings ++ Seq( +/* def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) + )*/ def examplesSettings = sharedSettings ++ Seq( name := "spark-examples" diff --git a/project/build.properties b/project/build.properties index d4287112c6..4474a03e1a 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.11.3 +sbt.version=0.12.1 diff --git a/project/plugins.sbt b/project/plugins.sbt index 4d0e696a11..0e4eb7085c 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,11 +6,11 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1") +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0") +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") -addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2") +addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) -- cgit v1.2.3 From a5403acd4ead9ecabfa9f2ed6d6d58afe912a238 Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 14:42:16 +0100 Subject: Updated maven build for scala 2.10. --- core/pom.xml | 20 ++++++++++++-------- pom.xml | 57 +++++++++++++++++++++++++++++++-------------------------- 2 files changed, 43 insertions(+), 34 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 862d3ec37a..6316b28a7b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -44,16 +44,16 @@ kryo-serializers - com.typesafe.akka - akka-actor + org.scala-lang + scala-actors com.typesafe.akka - akka-remote + akka-remote_${scala.version} com.typesafe.akka - akka-slf4j + akka-slf4j_${scala.version} it.unimi.dsi @@ -64,15 +64,19 @@ colt - cc.spray + io.spray spray-can - cc.spray - spray-server + io.spray + spray-routing + + + io.spray + spray-io - cc.spray + io.spray spray-json_${scala.version} diff --git a/pom.xml b/pom.xml index 751189a9d8..756fe8783b 100644 --- a/pom.xml +++ b/pom.xml @@ -41,8 +41,8 @@ core bagel examples - repl - repl-bin + @@ -50,20 +50,20 @@ UTF-8 1.5 - 2.9.2 + 2.10 0.9.0-incubating - 2.0.3 - 1.0-M2.1 - 1.1.1 + 2.1.0 + 1.1-M7 + 1.2.3 1.6.1 4.1.2 - jboss-repo - JBoss Repository - http://repository.jboss.org/nexus/content/repositories/releases/ + typesafe-repo + Typesafe Repository + http://repo.typesafe.com/typesafe/releases/ true @@ -72,9 +72,9 @@ - cloudera-repo - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos/ + jboss-repo + JBoss Repository + http://repository.jboss.org/nexus/content/repositories/releases/ true @@ -83,9 +83,9 @@ - typesafe-repo - Typesafe Repository - http://repo.typesafe.com/typesafe/releases/ + cloudera-repo + Cloudera Repository + https://repository.cloudera.com/artifactory/cloudera-repos/ true @@ -189,18 +189,18 @@ 0.20 - com.typesafe.akka - akka-actor - ${akka.version} + org.scala-lang + scala-actors + 2.10.0 com.typesafe.akka - akka-remote + akka-remote_${scala.version} ${akka.version} com.typesafe.akka - akka-slf4j + akka-slf4j_${scala.version} ${akka.version} @@ -214,17 +214,22 @@ 1.2.0 - cc.spray + io.spray spray-can ${spray.version} - cc.spray - spray-server + io.spray + spray-routing + ${spray.version} + + + io.spray + spray-io ${spray.version} - cc.spray + io.spray spray-json_${scala.version} ${spray.json.version} @@ -258,13 +263,13 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test -- cgit v1.2.3 From fd6e51deec83f01be3db41e84255329eedbe15da Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 17:02:58 +0100 Subject: Fixed the failing test. --- core/src/test/scala/spark/AccumulatorSuite.scala | 33 +++++++++++------------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index d8be99dde7..9f5335978f 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -13,6 +13,20 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter var sc: SparkContext = null + implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } + } + after { if (sc != null) { sc.stop() @@ -40,7 +54,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } test ("add value to collection accumulators") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -60,22 +73,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - implicit object SetAccum extends AccumulableParam[mutable.Set[Any], Any] { - def addInPlace(t1: mutable.Set[Any], t2: mutable.Set[Any]) : mutable.Set[Any] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[Any], t2: Any) : mutable.Set[Any] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[Any]) : mutable.Set[Any] = { - new mutable.HashSet[Any]() - } - } - test ("value not readable in tasks") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -123,7 +121,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } test ("localValue readable in tasks") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -135,7 +132,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } acc.value should be ( (0 to maxI).toSet) sc.stop() - sc = null + sc = null } } -- cgit v1.2.3 From 15530c2b23d1f3871a51fd24d14ce11b8ffaff4a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 17 Mar 2013 10:47:17 +0530 Subject: porting of repl to scala-2.10 --- project/SparkBuild.scala | 17 +- project/plugins.sbt | 4 +- repl/src/main/scala/spark/repl/Main.scala | 8 +- .../src/main/scala/spark/repl/SparkExprTyper.scala | 109 ++ repl/src/main/scala/spark/repl/SparkILoop.scala | 908 ++++++----- .../src/main/scala/spark/repl/SparkILoopInit.scala | 142 ++ repl/src/main/scala/spark/repl/SparkIMain.scala | 1675 +++++++++++--------- .../src/main/scala/spark/repl/SparkISettings.scala | 63 - repl/src/main/scala/spark/repl/SparkImports.scala | 113 +- .../scala/spark/repl/SparkJLineCompletion.scala | 207 ++- .../main/scala/spark/repl/SparkJLineReader.scala | 65 +- .../scala/spark/repl/SparkMemberHandlers.scala | 112 +- run | 2 +- 13 files changed, 1908 insertions(+), 1517 deletions(-) create mode 100644 repl/src/main/scala/spark/repl/SparkExprTyper.scala create mode 100644 repl/src/main/scala/spark/repl/SparkILoopInit.scala delete mode 100644 repl/src/main/scala/spark/repl/SparkISettings.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d0b3c350f1..9fcdbb5b92 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,11 +17,11 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, /*repl,*/ examples, bagel) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel) lazy val core = Project("core", file("core"), settings = coreSettings) -// lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) @@ -35,7 +35,7 @@ object SparkBuild extends Build { organization := "org.spark-project", version := "0.7.0-SNAPSHOT", scalaVersion := "2.10.0", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + scalacOptions := Seq("-unchecked", "-optimize"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -136,7 +136,9 @@ object SparkBuild extends Build { "io.spray" %% "spray-json" % "1.2.3", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.9.0-incubating", - "org.scala-lang" % "scala-actors" % "2.10.0" + "org.scala-lang" % "scala-actors" % "2.10.0", + "org.scala-lang" % "jline" % "2.10.0", + "org.scala-lang" % "scala-reflect" % "2.10.0" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } @@ -146,10 +148,11 @@ object SparkBuild extends Build { publish := {} ) -/* def replSettings = sharedSettings ++ Seq( + def replSettings = sharedSettings ++ Seq( name := "spark-repl", - libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - )*/ + // libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) + libraryDependencies ++= Seq("org.scala-lang" % "scala-compiler" % "2.10.0") + ) def examplesSettings = sharedSettings ++ Seq( name := "spark-examples" diff --git a/project/plugins.sbt b/project/plugins.sbt index 0e4eb7085c..4bb4a06a50 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,9 +6,9 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") +// addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") +// addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala index 58809ab646..b0a78e0bb8 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/spark/repl/Main.scala @@ -3,12 +3,12 @@ package spark.repl import scala.collection.mutable.Set object Main { - private var _interp: SparkILoop = null - + private var _interp: SparkILoop = _ + def interp = _interp - + def interp_=(i: SparkILoop) { _interp = i } - + def main(args: Array[String]) { _interp = new SparkILoop _interp.process(args) diff --git a/repl/src/main/scala/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/spark/repl/SparkExprTyper.scala new file mode 100644 index 0000000000..c55797eb54 --- /dev/null +++ b/repl/src/main/scala/spark/repl/SparkExprTyper.scala @@ -0,0 +1,109 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package spark.repl + +import scala.tools.nsc._ +import scala.tools.nsc.interpreter._ + +import scala.reflect.internal.util.BatchSourceFile +import scala.tools.nsc.ast.parser.Tokens.EOF + +import spark.Logging + +trait SparkExprTyper extends Logging { + val repl: SparkIMain + + import repl._ + import global.{ reporter => _, Import => _, _ } + import definitions._ + import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name } + import naming.freshInternalVarName + + object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] { + def applyRule[T](code: String, rule: UnitParser => T): T = { + reporter.reset() + val scanner = newUnitParser(code) + val result = rule(scanner) + + if (!reporter.hasErrors) + scanner.accept(EOF) + + result + } + + def defns(code: String) = stmts(code) collect { case x: DefTree => x } + def expr(code: String) = applyRule(code, _.expr()) + def stmts(code: String) = applyRule(code, _.templateStats()) + def stmt(code: String) = stmts(code).last // guaranteed nonempty + } + + /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ + def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") { + var isIncomplete = false + reporter.withIncompleteHandler((_, _) => isIncomplete = true) { + val trees = codeParser.stmts(line) + if (reporter.hasErrors) Some(Nil) + else if (isIncomplete) None + else Some(trees) + } + } + // def parsesAsExpr(line: String) = { + // import codeParser._ + // (opt expr line).isDefined + // } + + def symbolOfLine(code: String): Symbol = { + def asExpr(): Symbol = { + val name = freshInternalVarName() + // Typing it with a lazy val would give us the right type, but runs + // into compiler bugs with things like existentials, so we compile it + // behind a def and strip the NullaryMethodType which wraps the expr. + val line = "def " + name + " = {\n" + code + "\n}" + + interpretSynthetic(line) match { + case IR.Success => + val sym0 = symbolOfTerm(name) + // drop NullaryMethodType + val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) + if (sym.info.typeSymbol eq UnitClass) NoSymbol + else sym + case _ => NoSymbol + } + } + def asDefn(): Symbol = { + val old = repl.definedSymbolList.toSet + + interpretSynthetic(code) match { + case IR.Success => + repl.definedSymbolList filterNot old match { + case Nil => NoSymbol + case sym :: Nil => sym + case syms => NoSymbol.newOverloaded(NoPrefix, syms) + } + case _ => NoSymbol + } + } + beQuietDuring(asExpr()) orElse beQuietDuring(asDefn()) + } + + private var typeOfExpressionDepth = 0 + def typeOfExpression(expr: String, silent: Boolean = true): Type = { + if (typeOfExpressionDepth > 2) { + logDebug("Terminating typeOfExpression recursion for expression: " + expr) + return NoType + } + typeOfExpressionDepth += 1 + // Don't presently have a good way to suppress undesirable success output + // while letting errors through, so it is first trying it silently: if there + // is an error, and errors are desired, then it re-evaluates non-silently + // to induce the error message. + try beSilentDuring(symbolOfLine(expr).tpe) match { + case NoType if !silent => symbolOfLine(expr).tpe // generate error + case tpe => tpe + } + finally typeOfExpressionDepth -= 1 + } +} diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 22bcb4be8a..2f2b5b2372 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -1,26 +1,38 @@ /* NSC -- new Scala compiler - * Copyright 2005-2011 LAMP/EPFL + * Copyright 2005-2013 LAMP/EPFL * @author Alexander Spoon */ package spark.repl + import scala.tools.nsc._ import scala.tools.nsc.interpreter._ +import scala.tools.nsc.interpreter.{ Results => IR } import Predef.{ println => _, _ } -import java.io.{ BufferedReader, FileReader, PrintWriter } +import java.io.{ BufferedReader, FileReader } +import java.util.concurrent.locks.ReentrantLock import scala.sys.process.Process -import session._ -import scala.tools.nsc.interpreter.{ Results => IR } -import scala.tools.util.{ SignalManager, Signallable, Javap } +import scala.tools.nsc.interpreter.session._ +import scala.util.Properties.{ jdkHome, javaVersion } +import scala.tools.util.{ Javap } import scala.annotation.tailrec -import scala.util.control.Exception.{ ignoring } import scala.collection.mutable.ListBuffer import scala.concurrent.ops -import util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } -import interpreter._ -import io.{ File, Sources } +import scala.tools.nsc.util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } +import scala.tools.nsc.interpreter._ +import scala.tools.nsc.io.{ File, Directory } +import scala.reflect.NameTransformer._ +import scala.tools.nsc.util.ScalaClassLoader +import scala.tools.nsc.util.ScalaClassLoader._ +import scala.tools.util._ +import scala.language.{implicitConversions, existentials} +import scala.reflect.{ClassTag, classTag} +import scala.tools.reflect.StdRuntimeTags._ +import scala.reflect.{ClassTag, classTag} +import java.lang.{Class => jClass} +import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import spark.Logging import spark.SparkContext @@ -37,45 +49,85 @@ import spark.SparkContext * @author Lex Spoon * @version 1.2 */ -class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: Option[String]) +class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, + val master: Option[String]) extends AnyRef with LoopCommands + with SparkILoopInit with Logging { - def this(in0: BufferedReader, out: PrintWriter, master: String) = this(Some(in0), out, Some(master)) - def this(in0: BufferedReader, out: PrintWriter) = this(Some(in0), out, None) - def this() = this(None, new PrintWriter(Console.out, true), None) - + 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 = _ - /* - lazy val power = { - val g = intp.global - Power[g.type](this, g) - } - */ - - // TODO - // object opt extends AestheticSettings - // - @deprecated("Use `intp` instead.", "2.9.0") - def interpreter = intp - - @deprecated("Use `intp` instead.", "2.9.0") - def interpreter_= (i: SparkIMain): Unit = intp = i - + @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp + @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i + + /** Having inherited the difficult "var-ness" of the repl instance, + * I'm trying to work around it by moving operations into a class from + * which it will appear a stable prefix. + */ + private def onIntp[T](f: SparkIMain => T): T = f(intp) + + class IMainOps[T <: SparkIMain](val intp: T) { + import intp._ + import global._ + + def printAfterTyper(msg: => String) = + intp.reporter printMessage afterTyper(msg) + + /** Strip NullaryMethodType artifacts. */ + private def replInfo(sym: Symbol) = { + sym.info match { + case NullaryMethodType(restpe) if sym.isAccessor => restpe + case info => info + } + } + def echoTypeStructure(sym: Symbol) = + printAfterTyper("" + deconstruct.show(replInfo(sym))) + + def echoTypeSignature(sym: Symbol, verbose: Boolean) = { + if (verbose) SparkILoop.this.echo("// Type signature") + printAfterTyper("" + replInfo(sym)) + + if (verbose) { + SparkILoop.this.echo("\n// Internal Type structure") + echoTypeStructure(sym) + } + } + } + implicit def stabilizeIMain(intp: SparkIMain) = new IMainOps[intp.type](intp) + + /** TODO - + * -n normalize + * -l label with case class parameter names + * -c complete - leave nothing out + */ + private def typeCommandInternal(expr: String, verbose: Boolean): Result = { + onIntp { intp => + val sym = intp.symbolOfLine(expr) + if (sym.exists) intp.echoTypeSignature(sym, verbose) + else "" + } + } + + var sparkContext: SparkContext = _ + + override def echoCommandMessage(msg: String) { + intp.reporter printMessage msg + } + + def isAsync = !settings.Yreplsync.value + // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) def history = in.history /** The context class loader at the time this object was created */ protected val originalClassLoader = Thread.currentThread.getContextClassLoader - // Install a signal handler so we can be prodded. - private val signallable = - /*if (isReplDebug) Signallable("Dump repl state.")(dumpCommand()) - else*/ null - // classpath entries added via :cp var addedClasspath: String = "" @@ -87,74 +139,41 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: /** Record a command for replay should the user request a :replay */ def addReplay(cmd: String) = replayCommandStack ::= cmd - - /** Try to install sigint handler: ignore failure. Signal handler - * will interrupt current line execution if any is in progress. - * - * Attempting to protect the repl from accidental exit, we only honor - * a single ctrl-C if the current buffer is empty: otherwise we look - * for a second one within a short time. - */ - private def installSigIntHandler() { - def onExit() { - Console.println("") // avoiding "shell prompt in middle of line" syndrome - sys.exit(1) - } - ignoring(classOf[Exception]) { - SignalManager("INT") = { - if (intp == null) - onExit() - else if (intp.lineManager.running) - intp.lineManager.cancel() - else if (in.currentLine != "") { - // non-empty buffer, so make them hit ctrl-C a second time - SignalManager("INT") = onExit() - io.timer(5)(installSigIntHandler()) // and restore original handler if they don't - } - else onExit() - } - } + + def savingReplayStack[T](body: => T): T = { + val saved = replayCommandStack + try body + finally replayCommandStack = saved + } + def savingReader[T](body: => T): T = { + val saved = in + try body + finally in = saved } /** Close the interpreter and set the var to null. */ def closeInterpreter() { if (intp ne null) { - intp.close + intp.close() intp = null - Thread.currentThread.setContextClassLoader(originalClassLoader) } } - + class SparkILoopInterpreter extends SparkIMain(settings, out) { + outer => + override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } - override protected def createLineManager() = new Line.Manager { - override def onRunaway(line: Line[_]): Unit = { - val template = """ - |// She's gone rogue, captain! Have to take her out! - |// Calling Thread.stop on runaway %s with offending code: - |// scala> %s""".stripMargin - - echo(template.format(line.thread, line.code)) - // XXX no way to suppress the deprecation warning - line.thread.stop() - in.redrawLine() - } - } - override protected def parentClassLoader = { - SparkHelper.explicitParentLoader(settings).getOrElse( classOf[SparkILoop].getClassLoader ) - } + override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } /** Create a new interpreter. */ def createInterpreter() { if (addedClasspath != "") settings.classpath append addedClasspath - + intp = new SparkILoopInterpreter - intp.setContextClassLoader() - installSigIntHandler() } /** print a friendly help message */ @@ -168,10 +187,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: private def helpSummary() = { val usageWidth = commands map (_.usageMsg.length) max val formatStr = "%-" + usageWidth + "s %s %s" - + echo("All commands can be abbreviated, e.g. :he instead of :help.") echo("Those marked with a * have more detailed help, e.g. :help imports.\n") - + commands foreach { cmd => val star = if (cmd.hasLongHelp) "*" else " " echo(formatStr.format(cmd.usageMsg, star, cmd.help)) @@ -182,7 +201,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case Nil => echo(cmd + ": no such command. Type :help for help.") case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") } - Result(true, None) + Result(true, None) } private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) private def uniqueCommand(cmd: String): Option[LoopCommand] = { @@ -193,31 +212,16 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case xs => xs find (_.name == cmd) } } - - /** Print a welcome message */ - def printWelcome() { - echo("""Welcome to - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.7.0 - /_/ -""") - import Properties._ - val welcomeMsg = "Using Scala %s (%s, Java %s)".format( - versionString, javaVmName, javaVersion) - echo(welcomeMsg) - } - + /** Show the history */ lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { override def usage = "[num]" def defaultLines = 20 - + def apply(line: String): Result = { if (history eq NoHistory) return "No history available." - + val xs = words(line) val current = history.index val count = try xs.head.toInt catch { case _: Exception => defaultLines } @@ -229,32 +233,38 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } } - private def echo(msg: String) = { + // 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) = { + echo("\n" + msg) + in.redrawLine() + } + protected def echo(msg: String) = { out println msg out.flush() } - private def echoNoNL(msg: String) = { + protected def echoNoNL(msg: String) = { out print msg out.flush() } - + /** Search the history */ def searchHistory(_cmdline: String) { val cmdline = _cmdline.toLowerCase val offset = history.index - history.size + 1 - + for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) echo("%d %s".format(index + offset, line)) } - + private var currentPrompt = Properties.shellPromptString def setPrompt(prompt: String) = currentPrompt = prompt /** Prompt to print when awaiting input */ def prompt = currentPrompt - + import LoopCommand.{ cmd, nullary } - /** Standard commands **/ + /** Standard commands */ lazy val standardCommands = List( cmd("cp", "", "add a jar or directory to the classpath", addClasspath), cmd("help", "[command]", "print this summary or command-specific help", helpCommand), @@ -263,53 +273,30 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand), cmd("implicits", "[-v]", "show the implicits in scope", implicitsCommand), cmd("javap", "", "disassemble a file or class name", javapCommand), - nullary("keybindings", "show how ctrl-[A-Z] and other keys are bound", keybindingsCommand), cmd("load", "", "load and interpret a Scala file", loadCommand), nullary("paste", "enter paste mode: all input up to ctrl-D compiled together", pasteCommand), - //nullary("power", "enable power user mode", powerCmd), - nullary("quit", "exit the interpreter", () => Result(false, None)), +// nullary("power", "enable power user mode", powerCmd), + nullary("quit", "exit the repl", () => Result(false, None)), nullary("replay", "reset execution and replay all previous commands", replay), + nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), shCommand, nullary("silent", "disable/enable automatic printing of results", verbosity), - cmd("type", "", "display the type of an expression without evaluating it", typeCommand) + cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), + nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) ) - + /** Power user commands */ lazy val powerCommands: List[LoopCommand] = List( - //nullary("dump", "displays a view of the interpreter's internal state", dumpCommand), - //cmd("phase", "", "set the implicit phase for power commands", phaseCommand), - cmd("wrap", "", "name of method to wrap around each repl line", wrapCommand) withLongHelp (""" - |:wrap - |:wrap clear - |:wrap - | - |Installs a wrapper around each line entered into the repl. - |Currently it must be the simple name of an existing method - |with the specific signature shown in the following example. - | - |def timed[T](body: => T): T = { - | val start = System.nanoTime - | try body - | finally println((System.nanoTime - start) + " nanos elapsed.") - |} - |:wrap timed - | - |If given no argument, :wrap names the wrapper installed. - |An argument of clear will remove the wrapper if any is active. - |Note that wrappers do not compose (a new one replaces the old - |one) and also that the :phase command uses the same machinery, - |so setting :wrap will clear any :phase setting. - """.stripMargin.trim) + // cmd("phase", "", "set the implicit phase for power commands", phaseCommand) ) - - /* - private def dumpCommand(): Result = { - echo("" + power) - history.asStrings takeRight 30 foreach echo - in.redrawLine() - } - */ - + + // private def dumpCommand(): Result = { + // echo("" + power) + // history.asStrings takeRight 30 foreach echo + // in.redrawLine() + // } + // private def valsCommand(): Result = power.valsDescription + private val typeTransforms = List( "scala.collection.immutable." -> "immutable.", "scala.collection.mutable." -> "mutable.", @@ -317,7 +304,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: "java.lang." -> "jl.", "scala.runtime." -> "runtime." ) - + private def importsCommand(line: String): Result = { val tokens = words(line) val handlers = intp.languageWildcardHandlers ++ intp.importHandlers @@ -333,7 +320,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") - + intp.reporter.printMessage("%2d) %-30s %s%s".format( idx + 1, handler.importString, @@ -342,12 +329,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: )) } } - - private def implicitsCommand(line: String): Result = { - val intp = SparkILoop.this.intp + + private def implicitsCommand(line: String): Result = onIntp { intp => import intp._ - import global.Symbol - + import global._ + def p(x: Any) = intp.reporter.printMessage("" + x) // If an argument is given, only show a source with that @@ -360,17 +346,17 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else (args exists (source.name.toString contains _)) } } - + if (filtered.isEmpty) return "No implicits have been imported other than those in Predef." - + filtered foreach { case (source, syms) => p("/* " + syms.size + " implicit members imported from " + source.fullName + " */") - + // This groups the members by where the symbol is defined val byOwner = syms groupBy (_.owner) - val sortedOwners = byOwner.toList sortBy { case (owner, _) => intp.afterTyper(source.info.baseClasses indexOf owner) } + val sortedOwners = byOwner.toList sortBy { case (owner, _) => afterTyper(source.info.baseClasses indexOf owner) } sortedOwners foreach { case (owner, members) => @@ -388,10 +374,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: xss map (xs => xs sortBy (_.name.toString)) } - - val ownerMessage = if (owner == source) " defined in " else " inherited from " + + val ownerMessage = if (owner == source) " defined in " else " inherited from " p(" /* " + members.size + ownerMessage + owner.fullName + " */") - + memberGroups foreach { group => group foreach (s => p(" " + intp.symbolDefString(s))) p("") @@ -400,158 +386,182 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: p("") } } - - protected def newJavap() = new Javap(intp.classLoader, new SparkIMain.ReplStrippingWriter(intp)) { - override def tryClass(path: String): Array[Byte] = { - // Look for Foo first, then Foo$, but if Foo$ is given explicitly, - // we have to drop the $ to find object Foo, then tack it back onto - // the end of the flattened name. - def className = intp flatName path - def moduleName = (intp flatName path.stripSuffix("$")) + "$" - val bytes = super.tryClass(className) - if (bytes.nonEmpty) bytes - else super.tryClass(moduleName) + private def findToolsJar() = { + val jdkPath = Directory(jdkHome) + val jar = jdkPath / "lib" / "tools.jar" toFile; + + if (jar isFile) + Some(jar) + else if (jdkPath.isDirectory) + jdkPath.deepFiles find (_.name == "tools.jar") + else None + } + private def addToolsJarToLoader() = { + val cl = findToolsJar match { + case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader) + case _ => intp.classLoader + } + if (Javap.isAvailable(cl)) { + logDebug(":javap available.") + cl + } + else { + logDebug(":javap unavailable: no tools.jar at " + jdkHome) + intp.classLoader + } + } + + protected 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 + // key to finding it. + if (rest.nonEmpty) { + intp optFlatName hd match { + case Some(flat) => + val clazz = flat :: rest mkString NAME_JOIN_STRING + val bytes = super.tryClass(clazz) + if (bytes.nonEmpty) bytes + else super.tryClass(clazz + MODULE_SUFFIX_STRING) + case _ => super.tryClass(path) + } + } + else { + // Look for Foo first, then Foo$, but if Foo$ is given explicitly, + // we have to drop the $ to find object Foo, then tack it back onto + // the end of the flattened name. + def className = intp flatName path + def moduleName = (intp flatName path.stripSuffix(MODULE_SUFFIX_STRING)) + MODULE_SUFFIX_STRING + + val bytes = super.tryClass(className) + if (bytes.nonEmpty) bytes + else super.tryClass(moduleName) + } } } + // private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap()) private lazy val javap = try newJavap() catch { case _: Exception => null } - - private def typeCommand(line: String): Result = { - intp.typeOfExpression(line) match { - case Some(tp) => tp.toString - case _ => "Failed to determine type." + + // Still todo: modules. + private def typeCommand(line0: String): Result = { + line0.trim match { + case "" => ":type [-v] " + case s if s startsWith "-v " => typeCommandInternal(s stripPrefix "-v " trim, true) + case s => typeCommandInternal(s, false) } } - + + private def warningsCommand(): Result = { + if (intp.lastWarnings.isEmpty) + "Can't find any cached warnings." + else + intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) } + } + private def javapCommand(line: String): Result = { if (javap == null) - return ":javap unavailable on this platform." - if (line == "") - return ":javap [-lcsvp] [path1 path2 ...]" - - javap(words(line)) foreach { res => - if (res.isError) return "Failed: " + res.value - else res.show() - } - } - private def keybindingsCommand(): Result = { - if (in.keyBindings.isEmpty) "Key bindings unavailable." - else { - echo("Reading jline properties for default key bindings.") - echo("Accuracy not guaranteed: treat this as a guideline only.\n") - in.keyBindings foreach (x => echo ("" + x)) - } + ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome) + else if (javaVersion startsWith "1.7") + ":javap not yet working with java 1.7" + else if (line == "") + ":javap [-lcsvp] [path1 path2 ...]" + else + javap(words(line)) foreach { res => + if (res.isError) return "Failed: " + res.value + else res.show() + } } + private def wrapCommand(line: String): Result = { def failMsg = "Argument to :wrap must be the name of a method with signature [T](=> T): T" - val intp = SparkILoop.this.intp - val g: intp.global.type = intp.global - import g._ - - words(line) match { - case Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => "Current execution wrapper: " + s - } - case "clear" :: Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." - } - case wrapper :: Nil => - intp.typeOfExpression(wrapper) match { - case Some(PolyType(List(targ), MethodType(List(arg), restpe))) => - intp setExecutionWrapper intp.pathToTerm(wrapper) - "Set wrapper to '" + wrapper + "'" - case Some(x) => - failMsg + "\nFound: " + x - case _ => - failMsg + "\nFound: " - } - case _ => failMsg + onIntp { intp => + import intp._ + import global._ + + words(line) match { + case Nil => + intp.executionWrapper match { + case "" => "No execution wrapper is set." + case s => "Current execution wrapper: " + s + } + case "clear" :: Nil => + intp.executionWrapper match { + case "" => "No execution wrapper is set." + case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." + } + case wrapper :: Nil => + intp.typeOfExpression(wrapper) match { + case PolyType(List(targ), MethodType(List(arg), restpe)) => + intp setExecutionWrapper intp.pathToTerm(wrapper) + "Set wrapper to '" + wrapper + "'" + case tp => + failMsg + "\nFound: " + } + case _ => failMsg + } } } private def pathToPhaseWrapper = intp.pathToTerm("$r") + ".phased.atCurrent" - /* - private def phaseCommand(name: String): Result = { - // This line crashes us in TreeGen: - // - // if (intp.power.phased set name) "..." - // - // Exception in thread "main" java.lang.AssertionError: assertion failed: ._7.type - // at scala.Predef$.assert(Predef.scala:99) - // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:69) - // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:44) - // at scala.tools.nsc.ast.TreeGen.mkAttributedRef(TreeGen.scala:101) - // at scala.tools.nsc.ast.TreeGen.mkAttributedStableRef(TreeGen.scala:143) - // - // But it works like so, type annotated. - val phased: Phased = power.phased - import phased.NoPhaseName - - if (name == "clear") { - phased.set(NoPhaseName) - intp.clearExecutionWrapper() - "Cleared active phase." - } - else if (name == "") phased.get match { - case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" - case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) - } - else { - val what = phased.parse(name) - if (what.isEmpty || !phased.set(what)) - "'" + name + "' does not appear to represent a valid phase." - else { - intp.setExecutionWrapper(pathToPhaseWrapper) - val activeMessage = - if (what.toString.length == name.length) "" + what - else "%s (%s)".format(what, name) - - "Active phase is now: " + activeMessage - } - } - } - */ - + // private def phaseCommand(name: String): Result = { + // val phased: Phased = power.phased + // import phased.NoPhaseName + + // if (name == "clear") { + // phased.set(NoPhaseName) + // intp.clearExecutionWrapper() + // "Cleared active phase." + // } + // else if (name == "") phased.get match { + // case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" + // case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) + // } + // else { + // val what = phased.parse(name) + // if (what.isEmpty || !phased.set(what)) + // "'" + name + "' does not appear to represent a valid phase." + // else { + // intp.setExecutionWrapper(pathToPhaseWrapper) + // val activeMessage = + // if (what.toString.length == name.length) "" + what + // else "%s (%s)".format(what, name) + + // "Active phase is now: " + activeMessage + // } + // } + // } + /** Available commands */ - def commands: List[LoopCommand] = standardCommands /* ++ ( + def commands: List[LoopCommand] = standardCommands /*++ ( if (isReplPower) powerCommands else Nil )*/ - + val replayQuestionMessage = - """|The repl compiler has crashed spectacularly. Shall I replay your - |session? I can re-run all lines except the last one. + """|That entry seems to have slain the compiler. Shall I replay + |your session? I can re-run each line except the last one. |[y/n] """.trim.stripMargin - private val crashRecovery: PartialFunction[Throwable, Unit] = { + private val crashRecovery: PartialFunction[Throwable, Boolean] = { case ex: Throwable => - if (settings.YrichExes.value) { - val sources = implicitly[Sources] - echo("\n" + ex.getMessage) - echo( - if (isReplDebug) "[searching " + sources.path + " for exception contexts...]" - else "[searching for exception contexts...]" - ) - echo(Exceptional(ex).force().context()) - } - else { - echo(util.stackTraceString(ex)) - } + echo(intp.global.throwableAsString(ex)) + ex match { case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("Unrecoverable error.") + echo("\nUnrecoverable error.") throw ex case _ => - def fn(): Boolean = in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + def fn(): Boolean = + try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + catch { case _: RuntimeException => false } + if (fn()) replay() else echo("\nAbandoning crashed session.") } + true } /** The main read-eval-print loop for the repl. It calls @@ -564,66 +574,88 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: in readLine prompt } // return false if repl should exit - def processLine(line: String): Boolean = + def processLine(line: String): Boolean = { + if (isAsync) { + if (!awaitInitialized()) return false + runThunks() + } if (line eq null) false // assume null means EOF else command(line) match { case Result(false, _) => false case Result(_, Some(finalLine)) => addReplay(finalLine) ; true case _ => true } - - while (true) { - try if (!processLine(readOneLine)) return - catch crashRecovery } + def innerLoop() { + if ( try processLine(readOneLine()) catch crashRecovery ) + innerLoop() + } + innerLoop() } /** interpret all lines from a specified file */ - def interpretAllFrom(file: File) { - val oldIn = in - val oldReplay = replayCommandStack - - try file applyReader { reader => - in = SimpleReader(reader, out, false) - echo("Loading " + file + "...") - loop() - } - finally { - in = oldIn - replayCommandStack = oldReplay + def interpretAllFrom(file: File) { + savingReader { + savingReplayStack { + file applyReader { reader => + in = SimpleReader(reader, out, false) + echo("Loading " + file + "...") + loop() + } + } } } - /** create a new interpreter and replay all commands so far */ + /** create a new interpreter and replay the given commands */ def replay() { - closeInterpreter() - createInterpreter() - for (cmd <- replayCommands) { + reset() + if (replayCommandStack.isEmpty) + echo("Nothing to replay.") + else for (cmd <- replayCommands) { echo("Replaying: " + cmd) // flush because maybe cmd will have its own output command(cmd) echo("") } } - + def resetCommand() { + echo("Resetting repl state.") + if (replayCommandStack.nonEmpty) { + echo("Forgetting this session history:\n") + replayCommands foreach echo + echo("") + replayCommandStack = Nil + } + if (intp.namedDefinedTerms.nonEmpty) + echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", ")) + if (intp.definedTypes.nonEmpty) + echo("Forgetting defined types: " + intp.definedTypes.mkString(", ")) + + reset() + } + 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])") { override def usage = "" def apply(line: String): Result = line match { case "" => showUsage() - case _ => + case _ => val toRun = classOf[ProcessResult].getName + "(" + string2codeQuoted(line) + ")" intp interpret toRun () } } - + 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) = { var shouldReplay: Option[String] = None withFile(arg)(f => { @@ -643,23 +675,36 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } else echo("The path '" + f + "' doesn't seem to exist.") } - + def powerCmd(): Result = { if (isReplPower) "Already in power mode." - else enablePowerMode() + else enablePowerMode(false) + } + + def enablePowerMode(isDuringInit: Boolean) = { + // replProps.power setValue true + // unleashAndSetPhase() + // asyncEcho(isDuringInit, power.banner) } - def enablePowerMode() = { - //replProps.power setValue true - //power.unleash() - //echo(power.banner) + // private def unleashAndSetPhase() { +// if (isReplPower) { +// // power.unleash() +// // Set the phase to "typer" +// intp beSilentDuring phaseCommand("typer") +// } +// } + + def asyncEcho(async: Boolean, msg: => String) { + if (async) asyncMessage(msg) + else echo(msg) } - + def verbosity() = { - val old = intp.printResults - intp.printResults = !old - echo("Switched " + (if (old) "off" else "on") + " result printing.") + // val old = intp.printResults + // intp.printResults = !old + // echo("Switched " + (if (old) "off" else "on") + " result printing.") } - + /** 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. */ @@ -674,11 +719,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (intp.global == null) Result(false, None) // Notice failure to create compiler else Result(true, interpretStartingWith(line)) } - + private def readWhile(cond: String => Boolean) = { Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) } - + def pasteCommand(): Result = { echo("// Entering paste mode (ctrl-D to finish)\n") val code = readWhile(_ => true) mkString "\n" @@ -686,23 +731,19 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: intp interpret code () } - + private object paste extends Pasted { val ContinueString = " | " val PromptString = "scala> " - + def interpret(line: String): Unit = { echo(line.trim) intp interpret line echo("") } - + def transcript(start: String) = { - // Printing this message doesn't work very well because it's buried in the - // transcript they just pasted. Todo: a short timer goes off when - // lines stop coming which tells them to hit ctrl-D. - // - // echo("// Detected repl transcript paste: ctrl-D to finish.") + echo("\n// Detected repl transcript paste: ctrl-D to finish.\n") apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim)) } } @@ -717,7 +758,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def interpretStartingWith(code: String): Option[String] = { // signal completion non-completion input has been received in.completion.resetVerbosity() - + def reallyInterpret = { val reallyResult = intp.interpret(code) (reallyResult, reallyResult match { @@ -727,7 +768,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (in.interactive && code.endsWith("\n\n")) { echo("You typed two blank lines. Starting a new command.") None - } + } else in.readLine(ContinueString) match { case null => // we know compilation is going to fail since we're at EOF and the @@ -741,10 +782,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } }) } - + /** Here we place ourselves between the user and the interpreter and examine * the input they are ostensibly submitting. We intervene in several cases: - * + * * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation * on the previous result. @@ -759,28 +800,12 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") { interpretStartingWith(intp.mostRecentVar + code) } - else { - def runCompletion = in.completion execute code map (intp bindValue _) - /** Due to my accidentally letting file completion execution sneak ahead - * of actual parsing this now operates in such a way that the scala - * interpretation always wins. However to avoid losing useful file - * completion I let it fail and then check the others. So if you - * type /tmp it will echo a failure and then give you a Directory object. - * It's not pretty: maybe I'll implement the silence bits I need to avoid - * echoing the failure. - */ - if (intp isParseable code) { - val (code, result) = reallyInterpret - //if (power != null && code == IR.Error) - // runCompletion - - result - } - else runCompletion match { - case Some(_) => None // completion hit: avoid the latent error - case _ => reallyInterpret._2 // trigger the latent error - } + else if (code.trim startsWith "//") { + // line comment, do nothing + None } + else + reallyInterpret._2 } // runs :load `file` on any files passed via -i @@ -794,7 +819,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } case _ => } - + /** Tries to create a JLineReader, falling back to SimpleReader: * unless settings or properties are such that it should start * with SimpleReader. @@ -802,7 +827,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def chooseReader(settings: Settings): InteractiveReader = { if (settings.Xnojline.value || Properties.isEmacsShell) SimpleReader() - else try SparkJLineReader( + else try new SparkJLineReader( if (settings.noCompletion.value) NoCompletion else new SparkJLineCompletion(intp) ) @@ -813,96 +838,97 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } } - def initializeSpark() { - intp.beQuietDuring { - command(""" - spark.repl.Main.interp.out.println("Creating SparkContext..."); - spark.repl.Main.interp.out.flush(); - @transient val sc = spark.repl.Main.interp.createSparkContext(); - spark.repl.Main.interp.out.println("Spark context available as sc."); - spark.repl.Main.interp.out.flush(); - """) - command("import spark.SparkContext._"); - } - echo("Type in expressions to have them evaluated.") - echo("Type :help for more information.") - } - - var sparkContext: SparkContext = null - - def createSparkContext(): SparkContext = { - val master = this.master match { - case Some(m) => m - case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local" - } - } - sparkContext = new SparkContext(master, "Spark shell") - sparkContext - } - - def process(settings: Settings): Boolean = { - // Ensure logging is initialized before any Spark threads try to use logs - // (because SLF4J initialization is not thread safe) - initLogging() - - printWelcome() - echo("Initializing interpreter...") + val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + val m = u.runtimeMirror(getClass.getClassLoader) + private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = + u.TypeTag[T]( + m, + new TypeCreator { + def apply[U <: ApiUniverse with Singleton](m: Mirror[U]): U # Type = + m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] + }) + def process(settings: Settings): Boolean = savingContextLoader { this.settings = settings createInterpreter() - + // sets in to some kind of reader depending on environmental cues in = in0 match { case Some(reader) => SimpleReader(reader, out, true) - case None => chooseReader(settings) + case None => + // some post-initialization + chooseReader(settings) match { + case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x + case x => x + } } + lazy val tagOfSparkIMain = tagOfStaticClass[spark.repl.SparkIMain] + // Bind intp somewhere out of the regular namespace where + // we can get at it in generated code. + addThunk(intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfSparkIMain, classTag[SparkIMain]))) + addThunk({ + import scala.tools.nsc.io._ + import Properties.userHome + import scala.compat.Platform.EOL + val autorun = replProps.replAutorunCode.option flatMap (f => io.File(f).safeSlurp()) + if (autorun.isDefined) intp.quietRun(autorun.get) + }) loadFiles(settings) // it is broken on startup; go ahead and exit if (intp.reporter.hasErrors) return false - - try { - // this is about the illusion of snappiness. We call initialize() - // which spins off a separate thread, then print the prompt and try - // our best to look ready. Ideally the user will spend a - // couple seconds saying "wow, it starts so fast!" and by the time - // they type a command the compiler is ready to roll. - intp.initialize() - initializeSpark() - if (isReplPower) { - echo("Starting in power mode, one moment...\n") - enablePowerMode() - } - loop() + + // This is about the illusion of snappiness. We call initialize() + // which spins off a separate thread, then print the prompt and try + // our best to look ready. The interlocking lazy vals tend to + // inter-deadlock, so we break the cycle with a single asynchronous + // message to an actor. + if (isAsync) { + intp initialize initializedCallback() + addThunk(initializeSpark()) + createAsyncListener() // listens for signal to run postInitialization + } + else { + intp.initializeSynchronous() + postInitialization() } + printWelcome() + + try loop() + catch AbstractOrMissingHandler() finally closeInterpreter() + true } + def createSparkContext(): SparkContext = { + val master = this.master match { + case Some(m) => m + case None => { + val prop = System.getenv("MASTER") + if (prop != null) prop else "local" + } + } + sparkContext = new SparkContext(master, "Spark shell") + echo("Created spark context..") + sparkContext + } + /** process command-line arguments and do as they request */ def process(args: Array[String]): Boolean = { - val command = new CommandLine(args.toList, msg => echo("scala: " + msg)) + val command = new CommandLine(args.toList, echo) def neededHelp(): String = (if (command.settings.help.value) command.usageMsg + "\n" else "") + (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") - + // if they asked for no help and command is valid, we call the real main neededHelp() match { case "" => command.ok && process(command.settings) case help => echoNoNL(help) ; true } } - - @deprecated("Use `process` instead", "2.9.0") - def main(args: Array[String]): Unit = { - if (isReplDebug) - System.out.println(new java.util.Date) - - process(args) - } + @deprecated("Use `process` instead", "2.9.0") def main(settings: Settings): Unit = process(settings) } @@ -916,10 +942,10 @@ object SparkILoop { // like if you'd just typed it into the repl. def runForTranscript(code: String, settings: Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { - val output = new PrintWriter(new OutputStreamWriter(ostream), true) { + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) { override def write(str: String) = { // completely skip continuation lines if (str forall (ch => ch.isWhitespace || ch == '|')) () @@ -945,19 +971,19 @@ 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 = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { val input = new BufferedReader(new StringReader(code)) - val output = new PrintWriter(new OutputStreamWriter(ostream), true) - val repl = new SparkILoop(input, output) - + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) + val repl = new ILoop(input, output) + if (sets.classpath.isDefault) sets.classpath.value = sys.props("java.class.path") @@ -966,32 +992,4 @@ object SparkILoop { } } def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) - - // provide the enclosing type T - // in order to set up the interpreter's classpath and parent class loader properly - def breakIf[T: Manifest](assertion: => Boolean, args: NamedParam*): Unit = - if (assertion) break[T](args.toList) - - // start a repl, binding supplied args - def break[T: Manifest](args: List[NamedParam]): Unit = { - val msg = if (args.isEmpty) "" else " Binding " + args.size + " value%s.".format( - if (args.size == 1) "" else "s" - ) - echo("Debug repl starting." + msg) - val repl = new SparkILoop { - override def prompt = "\ndebug> " - } - repl.settings = new Settings(echo) - repl.settings.embeddedDefaults[T] - repl.createInterpreter() - repl.in = SparkJLineReader(repl) - - // rebind exit so people don't accidentally call sys.exit by way of predef - repl.quietRun("""def exit = println("Type :quit to resume program execution.")""") - args foreach (p => repl.bind(p.name, p.tpe, p.value)) - repl.loop() - - echo("\nDebug repl exiting.") - repl.closeInterpreter() - } } diff --git a/repl/src/main/scala/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/spark/repl/SparkILoopInit.scala new file mode 100644 index 0000000000..b52c477474 --- /dev/null +++ b/repl/src/main/scala/spark/repl/SparkILoopInit.scala @@ -0,0 +1,142 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package spark.repl + +import scala.tools.nsc._ +import scala.tools.nsc.interpreter._ + +import scala.reflect.internal.util.Position +import scala.util.control.Exception.ignoring +import scala.tools.nsc.util.stackTraceString + +/** + * Machinery for the asynchronous initialization of the repl. + */ +trait SparkILoopInit { + self: SparkILoop => + + /** Print a welcome message */ + def printWelcome() { + echo("""Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version 0.7.1-SNAPSHOT + /_/ +""") + import Properties._ + val welcomeMsg = "Using Scala %s (%s, Java %s)".format( + versionString, javaVmName, javaVersion) + echo(welcomeMsg) + } + + protected def asyncMessage(msg: String) { + if (isReplInfo || isReplPower) + echoAndRefresh(msg) + } + + private val initLock = new java.util.concurrent.locks.ReentrantLock() + private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized + private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized + private val initStart = System.nanoTime + + private def withLock[T](body: => T): T = { + initLock.lock() + try body + finally initLock.unlock() + } + // a condition used to ensure serial access to the compiler. + @volatile private var initIsComplete = false + @volatile private var initError: String = null + private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L) + + // the method to be called when the interpreter is initialized. + // Very important this method does nothing synchronous (i.e. do + // not try to use the interpreter) because until it returns, the + // repl's lazy val `global` is still locked. + protected def initializedCallback() = withLock(initCompilerCondition.signal()) + + // Spins off a thread which awaits a single message once the interpreter + // has been initialized. + protected def createAsyncListener() = { + io.spawn { + withLock(initCompilerCondition.await()) + asyncMessage("[info] compiler init time: " + elapsed() + " s.") + postInitialization() + } + } + + // called from main repl loop + protected def awaitInitialized(): Boolean = { + if (!initIsComplete) + withLock { while (!initIsComplete) initLoopCondition.await() } + if (initError != null) { + println(""" + |Failed to initialize the REPL due to an unexpected error. + |This is a bug, please, report it along with the error diagnostics printed below. + |%s.""".stripMargin.format(initError) + ) + false + } else true + } + // private def warningsThunks = List( + // () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _), + // ) + + protected def postInitThunks = List[Option[() => Unit]]( + Some(intp.setContextClassLoader _), + if (isReplPower) Some(() => enablePowerMode(true)) else None + ).flatten + // ++ ( + // warningsThunks + // ) + // called once after init condition is signalled + protected def postInitialization() { + try { + postInitThunks foreach (f => addThunk(f())) + runThunks() + } catch { + case ex: Throwable => + initError = stackTraceString(ex) + throw ex + } finally { + initIsComplete = true + + if (isAsync) { + asyncMessage("[info] total init time: " + elapsed() + " s.") + withLock(initLoopCondition.signal()) + } + } + } + + def initializeSpark() { + intp.beQuietDuring { + command(""" + @transient lazy val sc = spark.repl.Main.interp.createSparkContext(); + """) + command("import spark.SparkContext._"); + } + // echo("Type in expressions to have them evaluated.") + // echo("Type :help for more information.") + } + + // code to be executed only after the interpreter is initialized + // and the lazy val `global` can be accessed without risk of deadlock. + private var pendingThunks: List[() => Unit] = Nil + protected def addThunk(body: => Unit) = synchronized { + pendingThunks :+= (() => body) + } + protected def runThunks(): Unit = synchronized { + if (pendingThunks.nonEmpty) + println("Clearing " + pendingThunks.size + " thunks.") + + while (pendingThunks.nonEmpty) { + val thunk = pendingThunks.head + pendingThunks = pendingThunks.tail + thunk() + } + } +} diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/spark/repl/SparkIMain.scala index 43b6a6c950..8453628233 100644 --- a/repl/src/main/scala/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/spark/repl/SparkIMain.scala @@ -1,5 +1,5 @@ /* NSC -- new Scala compiler - * Copyright 2005-2011 LAMP/EPFL + * Copyright 2005-2013 LAMP/EPFL * @author Martin Odersky */ @@ -9,304 +9,334 @@ import scala.tools.nsc._ import scala.tools.nsc.interpreter._ import Predef.{ println => _, _ } -import java.io.{ PrintWriter } -import java.lang.reflect +import util.stringFromWriter +import scala.reflect.internal.util._ import java.net.URL -import util.{ Set => _, _ } -import io.{ AbstractFile, PlainFile, VirtualDirectory } -import reporters.{ ConsoleReporter, Reporter } -import symtab.{ Flags, Names } -import scala.tools.nsc.interpreter.{ Results => IR } +import scala.sys.BooleanProp +import io.{AbstractFile, PlainFile, VirtualDirectory} + +import reporters._ +import symtab.Flags +import scala.reflect.internal.Names import scala.tools.util.PathResolver -import scala.tools.nsc.util.{ ScalaClassLoader, Exceptional } +import scala.tools.nsc.util.ScalaClassLoader import ScalaClassLoader.URLClassLoader -import Exceptional.unwrap +import scala.tools.nsc.util.Exceptional.unwrap import scala.collection.{ mutable, immutable } -import scala.PartialFunction.{ cond, condOpt } import scala.util.control.Exception.{ ultimately } -import scala.reflect.NameTransformer import SparkIMain._ +import java.util.concurrent.Future +import typechecker.Analyzer +import scala.language.implicitConversions +import scala.reflect.runtime.{ universe => ru } +import scala.reflect.{ ClassTag, classTag } +import scala.tools.reflect.StdRuntimeTags._ +import scala.util.control.ControlThrowable +import util.stackTraceString import spark.HttpServer import spark.Utils import spark.SparkEnv +import spark.Logging + +// /** directory to save .class files to */ +// private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { +// private def pp(root: AbstractFile, indentLevel: Int) { +// val spaces = " " * indentLevel +// out.println(spaces + root.name) +// if (root.isDirectory) +// root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) +// } +// // print the contents hierarchically +// def show() = pp(this, 0) +// } + + /** An interpreter for Scala code. + * + * The main public entry points are compile(), interpret(), and bind(). + * The compile() method loads a complete Scala file. The interpret() method + * executes one line of Scala code at the request of the user. The bind() + * method binds an object to a variable that can then be used by later + * interpreted code. + * + * The overall approach is based on compiling the requested code and then + * using a Java classloader and Java reflection to run the code + * and access its results. + * + * In more detail, a single compiler instance is used + * to accumulate all successfully compiled or interpreted Scala code. To + * "interpret" a line of code, the compiler generates a fresh object that + * includes the line of code and which has public member(s) to export + * all variables defined by that code. To extract the result of an + * interpreted line to show the user, a second "result object" is created + * which imports the variables exported by the above object and then + * exports members called "$eval" and "$print". To accomodate user expressions + * that read from variables or methods defined in previous statements, "import" + * statements are used. + * + * This interpreter shares the strengths and weaknesses of using the + * full compiler-to-Java. The main strength is that interpreted code + * behaves exactly as does compiled code, including running at full speed. + * The main weakness is that redefining classes and methods is not handled + * properly, because rebinding at the Java level is technically difficult. + * + * @author Moez A. Abdel-Gawad + * @author Lex Spoon + */ + class SparkIMain(initialSettings: Settings, val out: JPrintWriter) extends SparkImports with Logging { + imain => -/** An interpreter for Scala code. - * - * The main public entry points are compile(), interpret(), and bind(). - * The compile() method loads a complete Scala file. The interpret() method - * executes one line of Scala code at the request of the user. The bind() - * method binds an object to a variable that can then be used by later - * interpreted code. - * - * The overall approach is based on compiling the requested code and then - * using a Java classloader and Java reflection to run the code - * and access its results. - * - * In more detail, a single compiler instance is used - * to accumulate all successfully compiled or interpreted Scala code. To - * "interpret" a line of code, the compiler generates a fresh object that - * includes the line of code and which has public member(s) to export - * all variables defined by that code. To extract the result of an - * interpreted line to show the user, a second "result object" is created - * which imports the variables exported by the above object and then - * exports a single member named "$export". To accomodate user expressions - * that read from variables or methods defined in previous statements, "import" - * statements are used. - * - * This interpreter shares the strengths and weaknesses of using the - * full compiler-to-Java. The main strength is that interpreted code - * behaves exactly as does compiled code, including running at full speed. - * The main weakness is that redefining classes and methods is not handled - * properly, because rebinding at the Java level is technically difficult. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - */ -class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends SparkImports { - imain => - - /** construct an interpreter that reports to Console */ - def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) - def this() = this(new Settings()) - - /** whether to print out result lines */ - var printResults: Boolean = true - - /** whether to print errors */ - var totalSilence: Boolean = false - - private val RESULT_OBJECT_PREFIX = "RequestResult$" - - lazy val formatting: Formatting = new Formatting { - val prompt = Properties.shellPromptString - } - import formatting._ - - val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - - /** Local directory to save .class files too */ - val outputDir = { - val tmp = System.getProperty("java.io.tmpdir") - val rootDir = System.getProperty("spark.repl.classdir", tmp) - Utils.createTempDir(rootDir) - } - if (SPARK_DEBUG_REPL) { - echo("Output directory: " + outputDir) - } - /** Scala compiler virtual directory for outputDir */ - val virtualDirectory = new PlainFile(outputDir) + val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - /** Jetty server that will serve our classes to worker nodes */ - val classServer = new HttpServer(outputDir) + /** Local directory to save .class files too */ + val outputDir = { + val tmp = System.getProperty("java.io.tmpdir") + val rootDir = System.getProperty("spark.repl.classdir", tmp) + Utils.createTempDir(rootDir) + } + if (SPARK_DEBUG_REPL) { + echo("Output directory: " + outputDir) + } - // Start the classServer and store its URI in a spark system property - // (which will be passed to executors so that they can connect to it) - classServer.start() - System.setProperty("spark.repl.class.uri", classServer.uri) - if (SPARK_DEBUG_REPL) { - echo("Class server started, URI = " + classServer.uri) - } + val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles + val classServer = new HttpServer(outputDir) /** Jetty server that will serve our classes to worker nodes */ + private var currentSettings: Settings = initialSettings + var printResults = true // whether to print result lines + 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 + private var _executionWrapper = "" // code to be wrapped around all lines + + + // Start the classServer and store its URI in a spark system property + // (which will be passed to executors so that they can connect to it) + classServer.start() + System.setProperty("spark.repl.class.uri", classServer.uri) + if (SPARK_DEBUG_REPL) { + echo("Class server started, URI = " + 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 + * through the roof if we wait for it. So we initialize it with a future and + * use a lazy val to ensure that any attempt to use the compiler object waits + * on the future. + */ + private var _classLoader: AbstractFileClassLoader = null // active classloader + private val _compiler: Global = newCompiler(settings, reporter) // our private compiler - /* - // directory to save .class files to - val virtualDirectory = new VirtualDirectory("(memory)", None) { - private def pp(root: io.AbstractFile, indentLevel: Int) { - val spaces = " " * indentLevel - out.println(spaces + root.name) - if (root.isDirectory) - root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) + private val nextReqId = { + var counter = 0 + () => { counter += 1 ; counter } } - // print the contents hierarchically - def show() = pp(this, 0) - } - */ - - /** reporter */ - lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) - import reporter.{ printMessage, withoutTruncating } - - // not sure if we have some motivation to print directly to console - private def echo(msg: String) { Console println msg } - - // protected def defaultImports: List[String] = List("_root_.scala.sys.exit") - - /** We're going to go to some trouble to initialize the compiler asynchronously. - * It's critical that nothing call into it until it's been initialized or we will - * run into unrecoverable issues, but the perceived repl startup time goes - * through the roof if we wait for it. So we initialize it with a future and - * use a lazy val to ensure that any attempt to use the compiler object waits - * on the future. - */ - private val _compiler: Global = newCompiler(settings, reporter) - private var _initializeComplete = false - def isInitializeComplete = _initializeComplete - - private def _initialize(): Boolean = { - val source = """ - |class $repl_$init { - | List(1) map (_ + 1) - |} - |""".stripMargin - - val result = try { - new _compiler.Run() compileSources List(new BatchSourceFile("", source)) - if (isReplDebug || settings.debug.value) { - // Can't use printMessage here, it deadlocks - Console.println("Repl compiler initialized.") - } - // addImports(defaultImports: _*) - true - } - catch { - case x: AbstractMethodError => - printMessage(""" - |Failed to initialize compiler: abstract method error. - |This is most often remedied by a full clean and recompile. - |""".stripMargin - ) - x.printStackTrace() - false - case x: MissingRequirementError => printMessage(""" - |Failed to initialize compiler: %s not found. - |** Note that as of 2.8 scala does not assume use of the java classpath. - |** For the old behavior pass -usejavacp to scala, or if using a Settings - |** object programatically, settings.usejavacp.value = true.""".stripMargin.format(x.req) + + def compilerClasspath: Seq[URL] = ( + if (isInitializeComplete) global.classPath.asURLs + else new PathResolver(settings).result.asURLs // the compiler's classpath ) - false + def settings = currentSettings + 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 { + val saved = settings.nowarn.value + if (!saved) + settings.nowarn.value = true + + try body + finally if (!saved) settings.nowarn.value = false } - - try result - finally _initializeComplete = result - } - - // set up initialization future - private var _isInitialized: () => Boolean = null - def initialize() = synchronized { - if (_isInitialized == null) - _isInitialized = scala.concurrent.ops future _initialize() - } - /** the public, go through the future compiler */ - lazy val global: Global = { - initialize() + /** construct an interpreter that reports to Console */ + def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) + def this() = this(new Settings()) - // blocks until it is ; false means catastrophic failure - if (_isInitialized()) _compiler - else null - } - @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - lazy val compiler: global.type = global - - import global._ - - object naming extends { - val global: imain.global.type = imain.global - } with Naming { - // make sure we don't overwrite their unwisely named res3 etc. - override def freshUserVarName(): String = { - val name = super.freshUserVarName() - if (definedNameMap contains name) freshUserVarName() - else name + 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" } - } - import naming._ - - // object dossiers extends { - // val intp: imain.type = imain - // } with Dossiers { } - // import dossiers._ - - lazy val memberHandlers = new { - val intp: imain.type = imain - } with SparkMemberHandlers - import memberHandlers._ - - def atPickler[T](op: => T): T = atPhase(currentRun.picklerPhase)(op) - def afterTyper[T](op: => T): T = atPhase(currentRun.typerPhase.next)(op) - - /** Temporarily be quiet */ - def beQuietDuring[T](operation: => T): T = { - val wasPrinting = printResults - ultimately(printResults = wasPrinting) { - if (isReplDebug) echo(">> beQuietDuring") - else printResults = false - - operation + lazy val formatting: Formatting = new Formatting { + val prompt = Properties.shellPromptString } - } - def beSilentDuring[T](operation: => T): T = { - val saved = totalSilence - totalSilence = true - try operation - finally totalSilence = saved - } - - def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - /** whether to bind the lastException variable */ - private var bindLastException = true - - /** A string representing code to be wrapped around all lines. */ - private var _executionWrapper: String = "" - def executionWrapper = _executionWrapper - def setExecutionWrapper(code: String) = _executionWrapper = code - def clearExecutionWrapper() = _executionWrapper = "" - - /** Temporarily stop binding lastException */ - def withoutBindingLastException[T](operation: => T): T = { - val wasBinding = bindLastException - ultimately(bindLastException = wasBinding) { - bindLastException = false - operation + lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + import formatting._ + import reporter.{ printMessage, withoutTruncating } + + // This exists mostly because using the reporter too early leads to deadlock. + private def echo(msg: String) { Console println msg } + private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }")) + private def _initialize() = { + try { + // todo. if this crashes, REPL will hang + new _compiler.Run() compileSources _initSources + _initializeComplete = true + true + } + catch AbstractOrMissingHandler() + } + private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" + + // argument is a thunk to execute after init is done + def initialize(postInitSignal: => Unit) { + synchronized { + if (_isInitialized == null) { + _isInitialized = io.spawn { + try _initialize() + finally postInitSignal + } + } + } + } + def initializeSynchronous(): Unit = { + if (!isInitializeComplete) { + _initialize() + assert(global != null, global) + } + } + def isInitializeComplete = _initializeComplete + + /** the public, go through the future compiler */ + lazy val global: Global = { + if (isInitializeComplete) _compiler + else { + // If init hasn't been called yet you're on your own. + if (_isInitialized == null) { + logWarning("Warning: compiler accessed before init set up. Assuming no postInit code.") + initialize(()) + } + // // blocks until it is ; false means catastrophic failure + if (_isInitialized.get()) _compiler + else null + } + } + @deprecated("Use `global` for access to the compiler instance.", "2.9.0") + 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) { + 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) + } + + // 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 { + val global: imain.global.type = imain.global + } with Naming { + // make sure we don't overwrite their unwisely named res3 etc. + def freshUserTermName(): TermName = { + val name = newTermName(freshUserVarName()) + if (definedNameMap contains name) freshUserTermName() + else name + } + def isUserTermName(name: Name) = isUserVarName("" + name) + def isInternalTermName(name: Name) = isInternalVarName("" + name) + } + import naming._ + + object deconstruct extends { + val global: imain.global.type = imain.global + } with StructuredTypeStrings + + lazy val memberHandlers = new { + val intp: imain.type = imain + } with SparkMemberHandlers + import memberHandlers._ + + /** Temporarily be quiet */ + def beQuietDuring[T](body: => T): T = { + val saved = printResults + printResults = false + try body + finally printResults = saved + } + def beSilentDuring[T](operation: => T): T = { + val saved = totalSilence + totalSilence = true + try operation + finally totalSilence = saved + } + + def quietRun[T](code: String) = beQuietDuring(interpret(code)) + + + private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { + case t: ControlThrowable => throw t + case t: Throwable => + logDebug(label + ": " + unwrap(t)) + logDebug(stackTraceString(unwrap(t))) + alt + } + /** takes AnyRef because it may be binding a Throwable or an Exceptional */ + + private def withLastExceptionLock[T](body: => T, alt: => T): T = { + assert(bindExceptions, "withLastExceptionLock called incorrectly.") + bindExceptions = false + + try beQuietDuring(body) + catch logAndDiscard("withLastExceptionLock", alt) + finally bindExceptions = true } - } - - protected def createLineManager(): Line.Manager = new Line.Manager - lazy val lineManager = createLineManager() - - /** interpreter settings */ - lazy val isettings = new SparkISettings(this) - - /** Instantiate a compiler. Subclasses can override this to - * change the compiler class used by this interpreter. */ - protected def newCompiler(settings: Settings, reporter: Reporter) = { - settings.outputDirs setSingleOutput virtualDirectory - settings.exposeEmptyPackage.value = true - new Global(settings, reporter) - } - - /** the compiler's classpath, as URL's */ - lazy val compilerClasspath: List[URL] = new PathResolver(settings) asURLs - /* A single class loader is used for all commands interpreted by this Interpreter. + def executionWrapper = _executionWrapper + def setExecutionWrapper(code: String) = _executionWrapper = code + def clearExecutionWrapper() = _executionWrapper = "" + + /** interpreter settings */ + lazy val isettings = new SparkISettings(this) + + /** Instantiate a compiler. Overridable. */ + protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { + settings.outputDirs setSingleOutput virtualDirectory + settings.exposeEmptyPackage.value = true + new Global(settings, reporter) with ReplGlobal { + override def toString: String = "" + } + } + + /** Parent classloader. Overridable. */ + protected def parentClassLoader: ClassLoader = + SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) + + /* A single class loader is used for all commands interpreted by this Interpreter. It would also be possible to create a new class loader for each command to interpret. The advantages of the current approach are: - - Expressions are only evaluated one time. This is especially - significant for I/O, e.g. "val x = Console.readLine" - - The main disadvantage is: - - - Objects, classes, and methods cannot be rebound. Instead, definitions - shadow the old ones, and old code objects refer to the old - definitions. - */ - private var _classLoader: AbstractFileClassLoader = null - def resetClassLoader() = _classLoader = makeClassLoader() - def classLoader: AbstractFileClassLoader = { - if (_classLoader == null) - resetClassLoader() - - _classLoader - } - private def makeClassLoader(): AbstractFileClassLoader = { - val parent = - if (parentClassLoader == null) ScalaClassLoader fromURLs compilerClasspath - else new URLClassLoader(compilerClasspath, parentClassLoader) + - Expressions are only evaluated one time. This is especially + significant for I/O, e.g. "val x = Console.readLine" + + The main disadvantage is: - new AbstractFileClassLoader(virtualDirectory, parent) { + - Objects, classes, and methods cannot be rebound. Instead, definitions + shadow the old ones, and old code objects refer to the old + definitions. + */ + def resetClassLoader() = { + logDebug("Setting new classloader: was " + _classLoader) + _classLoader = null + ensureClassLoader() + } + final def ensureClassLoader() { + if (_classLoader == null) + _classLoader = makeClassLoader() + } + def classLoader: AbstractFileClassLoader = { + ensureClassLoader() + _classLoader + } + private class TranslatingClassLoader(parent: ClassLoader) extends AbstractFileClassLoader(virtualDirectory, parent) { /** Overridden here to try translating a simple name to the generated * class name if the original attempt fails. This method is used by * getResourceAsStream as well as findClass. @@ -314,223 +344,300 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends override protected def findAbstractFile(name: String): AbstractFile = { super.findAbstractFile(name) match { // deadlocks on startup if we try to translate names too early - case null if isInitializeComplete => generatedName(name) map (x => super.findAbstractFile(x)) orNull - case file => file + case null if isInitializeComplete => + generatedName(name) map (x => super.findAbstractFile(x)) orNull + case file => + file } } } - } - private def loadByName(s: String): JClass = - (classLoader tryToInitializeClass s) getOrElse sys.error("Failed to load expected class: '" + s + "'") - - protected def parentClassLoader: ClassLoader = - SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) - - def getInterpreterClassLoader() = classLoader - - // Set the current Java "context" class loader to this interpreter's class loader - def setContextClassLoader() = classLoader.setAsContext() - - /** Given a simple repl-defined name, returns the real name of - * the class representing it, e.g. for "Bippy" it may return - * - * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy - */ - def generatedName(simpleName: String): Option[String] = { - if (simpleName endsWith "$") optFlatName(simpleName.init) map (_ + "$") - else optFlatName(simpleName) - } - def flatName(id: String) = optFlatName(id) getOrElse id - def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) - - def allDefinedNames = definedNameMap.keys.toList sortBy (_.toString) - def pathToType(id: String): String = pathToName(newTypeName(id)) - def pathToTerm(id: String): String = pathToName(newTermName(id)) - def pathToName(name: Name): String = { - if (definedNameMap contains name) - definedNameMap(name) fullPath name - else name.toString - } + private def makeClassLoader(): AbstractFileClassLoader = + new TranslatingClassLoader(parentClassLoader match { + case null => ScalaClassLoader fromURLs compilerClasspath + case p => new URLClassLoader(compilerClasspath, p) + }) + + def getInterpreterClassLoader() = classLoader + + // Set the current Java "context" class loader to this interpreter's class loader + def setContextClassLoader() = classLoader.setAsContext() + + /** Given a simple repl-defined name, returns the real name of + * the class representing it, e.g. for "Bippy" it may return + * {{{ + * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy + * }}} + */ + def generatedName(simpleName: String): Option[String] = { + if (simpleName endsWith 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) + + def allDefinedNames = definedNameMap.keys.toList.sorted + def pathToType(id: String): String = pathToName(newTypeName(id)) + def pathToTerm(id: String): String = pathToName(newTermName(id)) + def pathToName(name: Name): String = { + if (definedNameMap contains name) + definedNameMap(name) fullPath name + else name.toString + } - /** Most recent tree handled which wasn't wholly synthetic. */ - private def mostRecentlyHandledTree: Option[Tree] = { - prevRequests.reverse foreach { req => - req.handlers.reverse foreach { - case x: MemberDefHandler if x.definesValue && !isInternalVarName(x.name.toString) => return Some(x.member) - case _ => () + /** Most recent tree handled which wasn't wholly synthetic. */ + private def mostRecentlyHandledTree: Option[Tree] = { + prevRequests.reverse foreach { req => + req.handlers.reverse foreach { + case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member) + case _ => () + } } + None } - None - } - - /** Stubs for work in progress. */ - def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { - for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { - DBG("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) + + /** Stubs for work in progress. */ + 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) = { - for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { - // Printing the types here has a tendency to cause assertion errors, like - // assertion failed: fatal: has owner value x, but a class owner is required - // so DBG is by-name now to keep it in the family. (It also traps the assertion error, - // but we don't want to unnecessarily risk hosing the compiler's internal state.) - DBG("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) + def handleTermRedefinition(name: TermName, old: Request, req: Request) = { + for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { + // Printing the types here has a tendency to cause assertion errors, like + // assertion failed: fatal: has owner value x, but a class owner is required + // so DBG is by-name now to keep it in the family. (It also traps the assertion error, + // but we don't want to unnecessarily risk hosing the compiler's internal state.) + logDebug("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) + } } - } - def recordRequest(req: Request) { - if (req == null || referencedNameMap == null) - return - - prevRequests += req - req.referencedNames foreach (x => referencedNameMap(x) = req) - - // warning about serially defining companions. It'd be easy - // enough to just redefine them together but that may not always - // be what people want so I'm waiting until I can do it better. - if (!settings.nowarnings.value) { + + def recordRequest(req: Request) { + if (req == null || referencedNameMap == null) + return + + prevRequests += req + req.referencedNames foreach (x => referencedNameMap(x) = req) + + // warning about serially defining companions. It'd be easy + // enough to just redefine them together but that may not always + // be what people want so I'm waiting until I can do it better. for { name <- req.definedNames filterNot (x => req.definedNames contains x.companionName) oldReq <- definedNameMap get name.companionName newSym <- req.definedSymbols get name oldSym <- oldReq.definedSymbols get name.companionName + if Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule } } { - printMessage("warning: previously defined %s is not a companion to %s.".format(oldSym, newSym)) - printMessage("Companions must be defined together; you may wish to use :paste mode for this.") + afterTyper(replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.")) + replwarn("Companions must be defined together; you may wish to use :paste mode for this.") } - } - - // Updating the defined name map - req.definedNames foreach { name => - if (definedNameMap contains name) { - if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) - else handleTermRedefinition(name.toTermName, definedNameMap(name), req) + + // Updating the defined name map + req.definedNames foreach { name => + if (definedNameMap contains name) { + if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) + else handleTermRedefinition(name.toTermName, definedNameMap(name), req) + } + definedNameMap(name) = req } - definedNameMap(name) = req } - } - /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ - def parse(line: String): Option[List[Tree]] = { - var justNeedsMore = false - reporter.withIncompleteHandler((pos,msg) => {justNeedsMore = true}) { - // simple parse: just parse it, nothing else - def simpleParse(code: String): List[Tree] = { - reporter.reset() - val unit = new CompilationUnit(new BatchSourceFile("", code)) - val scanner = new syntaxAnalyzer.UnitParser(unit) - - scanner.templateStatSeq(false)._2 - } - val trees = simpleParse(line) - - if (reporter.hasErrors) Some(Nil) // the result did not parse, so stop - else if (justNeedsMore) None - else Some(trees) + def replwarn(msg: => String) { + if (!settings.nowarnings.value) + printMessage(msg) } - } - - def isParseable(line: String): Boolean = { - beSilentDuring { - parse(line) match { - case Some(xs) => xs.nonEmpty // parses as-is - case None => true // incomplete + + def isParseable(line: String): Boolean = { + beSilentDuring { + try parse(line) match { + case Some(xs) => xs.nonEmpty // parses as-is + case None => true // incomplete + } + catch { case x: Exception => // crashed the compiler + replwarn("Exception in isParseable(\"" + line + "\"): " + x) + false + } } } + + 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. + */ + def compileSources(sources: SourceFile*): Boolean = + compileSourcesKeepingRun(sources: _*)._1 + + /** Compile a string. Returns true if there are no + * compilation errors, or false otherwise. + */ + def compileString(code: String): Boolean = + compileSources(new BatchSourceFile("