aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--.gitignore1
-rw-r--r--README.md5
-rw-r--r--assembly/pom.xml2
-rw-r--r--bagel/pom.xml8
-rw-r--r--bin/compute-classpath.cmd2
-rwxr-xr-xbin/compute-classpath.sh2
-rw-r--r--core/pom.xml31
-rw-r--r--core/src/main/scala/org/apache/spark/MapOutputTracker.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/Partitioner.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala76
-rw-r--r--core/src/main/scala/org/apache/spark/SparkEnv.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/TaskState.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala59
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala33
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala65
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/Function.java8
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/Function2.java8
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java12
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java12
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/Client.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala39
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/executor/Executor.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/network/ConnectionManager.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala31
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDD.scala53
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/AkkaUtils.scala34
-rw-r--r--core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala11
-rw-r--r--core/src/test/scala/org/apache/spark/AccumulatorSuite.scala32
-rw-r--r--core/src/test/scala/org/apache/spark/CheckpointSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/DistributedSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/DriverSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/ui/UISuite.scala3
-rw-r--r--core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala72
-rw-r--r--docs/_config.yml2
-rw-r--r--docs/_plugins/copy_api_dirs.rb2
-rw-r--r--examples/pom.xml12
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala5
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala8
-rw-r--r--mllib/pom.xml8
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala2
-rw-r--r--pom.xml51
-rw-r--r--project/SparkBuild.scala111
-rwxr-xr-xpyspark2
-rw-r--r--pyspark2.cmd2
-rw-r--r--python/pyspark/rdd.py4
-rwxr-xr-xrepl-bin/src/deb/bin/run2
-rw-r--r--repl/lib/scala-jline.jarbin158463 -> 0 bytes
-rw-r--r--repl/pom.xml10
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/Main.scala8
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala110
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala941
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala143
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala1674
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala63
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkImports.scala108
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala206
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala65
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala109
-rw-r--r--repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala152
-rwxr-xr-xrun-example2
-rw-r--r--run-example2.cmd2
-rwxr-xr-xspark-class3
-rw-r--r--streaming/pom.xml16
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStream.scala31
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala47
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala40
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala31
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala57
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala84
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala12
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala14
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala11
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala9
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala5
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala7
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala35
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala13
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala45
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java2
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala18
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala36
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala25
-rw-r--r--tools/pom.xml6
-rw-r--r--yarn/pom.xml4
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala2
131 files changed, 2925 insertions, 2394 deletions
diff --git a/.gitignore b/.gitignore
index e1f64a1133..b3c4363af0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -41,3 +41,4 @@ derby.log
dist/
spark-*-bin.tar.gz
unit-tests.log
+lib/
diff --git a/README.md b/README.md
index 28ad1e4604..7d3f9d4845 100644
--- a/README.md
+++ b/README.md
@@ -12,9 +12,8 @@ This README file only contains basic setup instructions.
## Building
-Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is
-built using Simple Build Tool (SBT), which is packaged with it. To build
-Spark and its example programs, run:
+Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT),
+which is packaged with it. To build Spark and its example programs, run:
sbt/sbt assembly
diff --git a/assembly/pom.xml b/assembly/pom.xml
index d62332137a..f9c8b9708a 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -79,7 +79,7 @@
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
- <outputFile>${project.build.directory}/scala-${scala.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
+ <outputFile>${project.build.directory}/scala-${scala-short.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
diff --git a/bagel/pom.xml b/bagel/pom.xml
index c4ce006085..271ab6ce56 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -43,18 +43,18 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
index cf38188c4b..9e3e10ecaa 100644
--- a/bin/compute-classpath.cmd
+++ b/bin/compute-classpath.cmd
@@ -20,7 +20,7 @@ rem
rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
rem script and the ExecutorRunner in standalone cluster mode.
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
set FWDIR=%~dp0..\
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index c7819d4932..4fe3d0ef3a 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -20,7 +20,7 @@
# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
# script and the ExecutorRunner in standalone cluster mode.
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Figure out where Spark is installed
FWDIR="$(cd `dirname $0`/..; pwd)"
diff --git a/core/pom.xml b/core/pom.xml
index 9c2d6046a9..2ec8fa6d0a 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -83,7 +83,7 @@
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>chill_2.9.3</artifactId>
+ <artifactId>chill_${scala-short.version}</artifactId>
<version>0.3.1</version>
</dependency>
<dependency>
@@ -93,19 +93,11 @@
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
- <artifactId>akka-actor</artifactId>
+ <artifactId>akka-remote_${scala-short.version}</artifactId>
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
- <artifactId>akka-remote</artifactId>
- </dependency>
- <dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-slf4j</artifactId>
- </dependency>
- <dependency>
- <groupId>org.scala-lang</groupId>
- <artifactId>scalap</artifactId>
+ <artifactId>akka-slf4j_${scala-short.version}</artifactId>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
@@ -113,7 +105,7 @@
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
- <artifactId>lift-json_2.9.2</artifactId>
+ <artifactId>lift-json_${scala-short.version}</artifactId>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
@@ -125,7 +117,7 @@
</dependency>
<dependency>
<groupId>com.github.scala-incubator.io</groupId>
- <artifactId>scala-io-file_2.9.2</artifactId>
+ <artifactId>scala-io-file_${scala-short.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.mesos</groupId>
@@ -161,13 +153,18 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -187,8 +184,8 @@
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index ae7cf2a893..1afb1870f1 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -24,11 +24,13 @@ 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 scala.concurrent.duration._
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
@@ -56,7 +58,7 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac
private[spark] class MapOutputTracker extends Logging {
private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
-
+
// Set to the MapOutputTrackerActor living on the driver
var trackerActor: ActorRef = _
@@ -159,7 +161,7 @@ private[spark] class MapOutputTracker extends Logging {
fetching += shuffleId
}
}
-
+
if (fetchedStatuses == null) {
// We won the race to fetch the output locs; do so
logInfo("Doing the fetch; tracker actor = " + trackerActor)
@@ -186,7 +188,7 @@ private[spark] class MapOutputTracker extends Logging {
else{
throw new FetchFailedException(null, shuffleId, -1, reduceId,
new Exception("Missing all output locations for shuffle " + shuffleId))
- }
+ }
} else {
statuses.synchronized {
return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses)
@@ -283,7 +285,7 @@ private[spark] class MapOutputTracker extends Logging {
val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes)))
objIn.readObject().
// // drop all null's from status - not sure why they are occuring though. Causes NPE downstream in slave if present
- // comment this out - nulls could be due to missing location ?
+ // comment this out - nulls could be due to missing location ?
asInstanceOf[Array[MapStatus]] // .filter( _ != null )
}
}
@@ -300,7 +302,7 @@ private[spark] object MapOutputTracker {
statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = {
assert (statuses != null)
statuses.map {
- status =>
+ status =>
if (status == null) {
throw new FetchFailedException(null, shuffleId, -1, reduceId,
new Exception("Missing an output location for shuffle " + shuffleId))
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 0e2c987a59..62b608c088 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -20,6 +20,8 @@ package org.apache.spark
import org.apache.spark.util.Utils
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
+
/**
* An object that defines how the elements in a key-value pair RDD are partitioned by key.
* Maps each key to a partition ID, from 0 to `numPartitions - 1`.
@@ -72,7 +74,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
case null => 0
case _ => Utils.nonNegativeMod(key.hashCode, numPartitions)
}
-
+
override def equals(other: Any): Boolean = other match {
case h: HashPartitioner =>
h.numPartitions == numPartitions
@@ -85,7 +87,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
* A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
* Determines the ranges by sampling the RDD passed in.
*/
-class RangePartitioner[K <% Ordered[K]: ClassManifest, V](
+class RangePartitioner[K <% Ordered[K]: ClassTag, V](
partitions: Int,
@transient rdd: RDD[_ <: Product2[K,V]],
private val ascending: Boolean = true)
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 912ce752fb..e67390cfd1 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -27,6 +27,8 @@ import scala.collection.generic.Growable
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
+import scala.reflect.{ ClassTag, classTag}
+import scala.util.DynamicVariable
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -299,19 +301,19 @@ class SparkContext(
// Methods for creating RDDs
/** Distribute a local Scala collection to form an RDD. */
- def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
+ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())
}
/** Distribute a local Scala collection to form an RDD. */
- def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
+ def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
parallelize(seq, numSlices)
}
/** Distribute a local Scala collection to form an RDD, with one or more
* location preferences (hostnames of Spark nodes) for each object.
* Create a new partition for each collection item. */
- def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = {
+ def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = {
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
}
@@ -354,7 +356,7 @@ class SparkContext(
}
/**
- * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys,
+ * Smarter version of hadoopFile() that uses class tags to figure out the classes of keys,
* values and the InputFormat so that users don't need to pass them directly. Instead, callers
* can just write, for example,
* {{{
@@ -362,17 +364,17 @@ class SparkContext(
* }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int)
- (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F])
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F])
: RDD[(K, V)] = {
hadoopFile(path,
- fm.erasure.asInstanceOf[Class[F]],
- km.erasure.asInstanceOf[Class[K]],
- vm.erasure.asInstanceOf[Class[V]],
+ fm.runtimeClass.asInstanceOf[Class[F]],
+ km.runtimeClass.asInstanceOf[Class[K]],
+ vm.runtimeClass.asInstanceOf[Class[V]],
minSplits)
}
/**
- * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys,
+ * Smarter version of hadoopFile() that uses class tags to figure out the classes of keys,
* values and the InputFormat so that users don't need to pass them directly. Instead, callers
* can just write, for example,
* {{{
@@ -380,17 +382,17 @@ class SparkContext(
* }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String)
- (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] =
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] =
hadoopFile[K, V, F](path, defaultMinSplits)
/** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */
def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String)
- (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] = {
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = {
newAPIHadoopFile(
path,
- fm.erasure.asInstanceOf[Class[F]],
- km.erasure.asInstanceOf[Class[K]],
- vm.erasure.asInstanceOf[Class[V]])
+ fm.runtimeClass.asInstanceOf[Class[F]],
+ km.runtimeClass.asInstanceOf[Class[K]],
+ vm.runtimeClass.asInstanceOf[Class[V]])
}
/**
@@ -448,11 +450,11 @@ class SparkContext(
* IntWritable). The most natural thing would've been to have implicit objects for the
* converters, but then we couldn't have an object for every subclass of Writable (you can't
* have a parameterized singleton object). We use functions instead to create a new converter
- * for the appropriate type. In addition, we pass the converter a ClassManifest of its type to
+ * for the appropriate type. In addition, we pass the converter a ClassTag of its type to
* allow it to figure out the Writable class to use in the subclass case.
*/
def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits)
- (implicit km: ClassManifest[K], vm: ClassManifest[V],
+ (implicit km: ClassTag[K], vm: ClassTag[V],
kcf: () => WritableConverter[K], vcf: () => WritableConverter[V])
: RDD[(K, V)] = {
val kc = kcf()
@@ -471,7 +473,7 @@ class SparkContext(
* slow if you use the default serializer (Java serialization), though the nice thing about it is
* that there's very little effort required to save arbitrary objects.
*/
- def objectFile[T: ClassManifest](
+ def objectFile[T: ClassTag](
path: String,
minSplits: Int = defaultMinSplits
): RDD[T] = {
@@ -480,17 +482,17 @@ class SparkContext(
}
- protected[spark] def checkpointFile[T: ClassManifest](
+ protected[spark] def checkpointFile[T: ClassTag](
path: String
): RDD[T] = {
new CheckpointRDD[T](this, path)
}
/** Build the union of a list of RDDs. */
- def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
+ def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
/** Build the union of a list of RDDs passed as variable-length arguments. */
- def union[T: ClassManifest](first: RDD[T], rest: RDD[T]*): RDD[T] =
+ def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] =
new UnionRDD(this, Seq(first) ++ rest)
// Methods for creating shared variables
@@ -718,7 +720,7 @@ class SparkContext(
* flag specifies whether the scheduler can run the computation on the driver rather than
* shipping it out to the cluster, for short actions like first().
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
@@ -739,7 +741,7 @@ class SparkContext(
* allowLocal flag specifies whether the scheduler can run the computation on the driver rather
* than shipping it out to the cluster, for short actions like first().
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
@@ -754,7 +756,7 @@ class SparkContext(
* Run a job on a given set of partitions of an RDD, but take a function of type
* `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`.
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: Iterator[T] => U,
partitions: Seq[Int],
@@ -766,21 +768,21 @@ class SparkContext(
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
- def runJob[T, U: ClassManifest](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
+ def runJob[T, U: ClassTag](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
runJob(rdd, func, 0 until rdd.partitions.size, false)
}
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
- def runJob[T, U: ClassManifest](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
+ def runJob[T, U: ClassTag](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
runJob(rdd, func, 0 until rdd.partitions.size, false)
}
/**
* Run a job on all partitions in an RDD and pass the results to a handler function.
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
processPartition: (TaskContext, Iterator[T]) => U,
resultHandler: (Int, U) => Unit)
@@ -791,7 +793,7 @@ class SparkContext(
/**
* Run a job on all partitions in an RDD and pass the results to a handler function.
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
processPartition: Iterator[T] => U,
resultHandler: (Int, U) => Unit)
@@ -897,14 +899,14 @@ object SparkContext {
// TODO: Add AccumulatorParams for other types, e.g. lists and strings
- implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
+ implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
new PairRDDFunctions(rdd)
- implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest](
+ implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
rdd: RDD[(K, V)]) =
new SequenceFileRDDFunctions(rdd)
- implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
+ implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag](
rdd: RDD[(K, V)]) =
new OrderedRDDFunctions[K, V, (K, V)](rdd)
@@ -929,16 +931,16 @@ object SparkContext {
implicit def stringToText(s: String) = new Text(s)
- private implicit def arrayToArrayWritable[T <% Writable: ClassManifest](arr: Traversable[T]): ArrayWritable = {
+ private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = {
def anyToWritable[U <% Writable](u: U): Writable = u
- new ArrayWritable(classManifest[T].erasure.asInstanceOf[Class[Writable]],
+ new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]],
arr.map(x => anyToWritable(x)).toArray)
}
// Helper objects for converting common types to Writable
- private def simpleWritableConverter[T, W <: Writable: ClassManifest](convert: W => T) = {
- val wClass = classManifest[W].erasure.asInstanceOf[Class[W]]
+ private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = {
+ val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]]
new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W]))
}
@@ -957,7 +959,7 @@ object SparkContext {
implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString)
implicit def writableWritableConverter[T <: Writable]() =
- new WritableConverter[T](_.erasure.asInstanceOf[Class[T]], _.asInstanceOf[T])
+ new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T])
/**
* Find the JAR from which a given class was loaded, to make it easy for users to pass
@@ -994,12 +996,12 @@ object SparkContext {
/**
* A class encapsulating how to convert some type T to Writable. It stores both the Writable class
* corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion.
- * The getter for the writable class takes a ClassManifest[T] in case this is a generic object
+ * The getter for the writable class takes a ClassTag[T] in case this is a generic object
* that doesn't know the type of T when it is created. This sounds strange but is necessary to
* support converting subclasses of Writable to themselves (writableWritableConverter).
*/
private[spark] class WritableConverter[T](
- val writableClass: ClassManifest[T] => Class[_ <: Writable],
+ val writableClass: ClassTag[T] => Class[_ <: Writable],
val convert: Writable => T)
extends Serializable
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 29968c273c..a267407c67 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -83,7 +83,8 @@ class SparkEnv (
actorSystem.shutdown()
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut
// down, but let's call it anyway in case it gets fixed in a later release
- actorSystem.awaitTermination()
+ // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it.
+ //actorSystem.awaitTermination()
}
def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = {
@@ -168,7 +169,7 @@ object SparkEnv extends Logging {
val driverHost: String = System.getProperty("spark.driver.host", "localhost")
val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt
Utils.checkHost(driverHost, "Expected hostname")
- val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name)
+ val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name)
logInfo("Connecting to " + name + ": " + url)
actorSystem.actorFor(url)
}
diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala
index 19ce8369d9..0bf1e4a5e2 100644
--- a/core/src/main/scala/org/apache/spark/TaskState.scala
+++ b/core/src/main/scala/org/apache/spark/TaskState.scala
@@ -19,8 +19,7 @@ package org.apache.spark
import org.apache.mesos.Protos.{TaskState => MesosTaskState}
-private[spark] object TaskState
- extends Enumeration("LAUNCHING", "RUNNING", "FINISHED", "FAILED", "KILLED", "LOST") {
+private[spark] object TaskState extends Enumeration {
val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
index 5fd1fab580..f0a1960a1b 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
@@ -17,18 +17,21 @@
package org.apache.spark.api.java
+import scala.reflect.ClassTag
+
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions
import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.util.StatCounter
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.storage.StorageLevel
+
import java.lang.Double
import org.apache.spark.Partitioner
class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
- override val classManifest: ClassManifest[Double] = implicitly[ClassManifest[Double]]
+ override val classTag: ClassTag[Double] = implicitly[ClassTag[Double]]
override val rdd: RDD[Double] = srdd.map(x => Double.valueOf(x))
@@ -42,7 +45,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
/** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
def cache(): JavaDoubleRDD = fromRDD(srdd.cache())
- /**
+ /**
* Set this RDD's storage level to persist its values across operations after the first time
* it is computed. Can only be called once on each RDD.
*/
@@ -82,7 +85,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
/**
* Return an RDD with the elements from `this` that are not in `other`.
- *
+ *
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index a6518abf45..899e17d4fa 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -22,6 +22,7 @@ import java.util.Comparator
import scala.Tuple2
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import com.google.common.base.Optional
import org.apache.hadoop.io.compress.CompressionCodec
@@ -43,13 +44,13 @@ import org.apache.spark.rdd.OrderedRDDFunctions
import org.apache.spark.storage.StorageLevel
-class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K],
- implicit val vManifest: ClassManifest[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
+class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K],
+ implicit val vClassTag: ClassTag[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
- override val classManifest: ClassManifest[(K, V)] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+ override val classTag: ClassTag[(K, V)] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]]
import JavaPairRDD._
@@ -58,7 +59,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
def cache(): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.cache())
- /**
+ /**
* Set this RDD's storage level to persist its values across operations after the first time
* it is computed. Can only be called once on each RDD.
*/
@@ -114,14 +115,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
override def first(): (K, V) = rdd.first()
// Pair RDD functions
-
+
/**
- * Generic function to combine the elements for each key using a custom set of aggregation
- * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a
- * "combined type" C * Note that V and C can be different -- for example, one might group an
- * RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three
+ * Generic function to combine the elements for each key using a custom set of aggregation
+ * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a
+ * "combined type" C * Note that V and C can be different -- for example, one might group an
+ * RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three
* functions:
- *
+ *
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
@@ -133,8 +134,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C],
partitioner: Partitioner): JavaPairRDD[K, C] = {
- implicit val cm: ClassManifest[C] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ implicit val cm: ClassTag[C] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
fromRDD(rdd.combineByKey(
createCombiner,
mergeValue,
@@ -171,14 +172,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/** Count the number of elements for each key, and return the result to the master as a Map. */
def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey())
- /**
+ /**
* (Experimental) Approximate version of countByKey that can return a partial result if it does
* not finish within a timeout.
*/
def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] =
rdd.countByKeyApprox(timeout).map(mapAsJavaMap)
- /**
+ /**
* (Experimental) Approximate version of countByKey that can return a partial result if it does
* not finish within a timeout.
*/
@@ -234,7 +235,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/**
* Return an RDD with the elements from `this` that are not in `other`.
- *
+ *
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
@@ -291,15 +292,15 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)})
}
- /**
+ /**
* Simplified version of combineByKey that hash-partitions the resulting RDD using the existing
* partitioner/parallelism level.
*/
def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = {
- implicit val cm: ClassManifest[C] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ implicit val cm: ClassTag[C] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd)))
}
@@ -390,8 +391,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* this also retains the original RDD's partitioning.
*/
def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
fromRDD(rdd.mapValues(f))
}
@@ -402,8 +403,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = {
import scala.collection.JavaConverters._
def fn = (x: V) => f.apply(x).asScala
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
fromRDD(rdd.flatMapValues(fn))
}
@@ -579,22 +580,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
}
object JavaPairRDD {
- def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassManifest[K],
- vcm: ClassManifest[T]): RDD[(K, JList[T])] =
+ def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassTag[K],
+ vcm: ClassTag[T]): RDD[(K, JList[T])] =
rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _)
- def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassManifest[K],
- vcm: ClassManifest[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V],
+ def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K],
+ vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V],
Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2)))
def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1],
- Seq[W2]))])(implicit kcm: ClassManifest[K]) : RDD[(K, (JList[V], JList[W1],
+ Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1],
JList[W2]))] = rddToPairRDDFunctions(rdd).mapValues(
(x: (Seq[V], Seq[W1], Seq[W2])) => (seqAsJavaList(x._1),
seqAsJavaList(x._2),
seqAsJavaList(x._3)))
- def fromRDD[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]): JavaPairRDD[K, V] =
+ def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd)
implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index eec58abdd6..9968bc8e5f 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -17,12 +17,14 @@
package org.apache.spark.api.java
+import scala.reflect.ClassTag
+
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.storage.StorageLevel
-class JavaRDD[T](val rdd: RDD[T])(implicit val classManifest: ClassManifest[T]) extends
+class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends
JavaRDDLike[T, JavaRDD[T]] {
override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd)
@@ -108,8 +110,7 @@ JavaRDDLike[T, JavaRDD[T]] {
object JavaRDD {
- implicit def fromRDD[T: ClassManifest](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd)
+ implicit def fromRDD[T: ClassTag](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd)
implicit def toRDD[T](rdd: JavaRDD[T]): RDD[T] = rdd.rdd
}
-
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 7a3568c5ef..d7b45d4caa 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -20,6 +20,7 @@ package org.apache.spark.api.java
import java.util.{List => JList, Comparator}
import scala.Tuple2
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import com.google.common.base.Optional
import org.apache.hadoop.io.compress.CompressionCodec
@@ -35,7 +36,7 @@ import org.apache.spark.storage.StorageLevel
trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def wrapRDD(rdd: RDD[T]): This
- implicit val classManifest: ClassManifest[T]
+ implicit val classTag: ClassTag[T]
def rdd: RDD[T]
@@ -87,7 +88,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* Return a new RDD by applying a function to all elements of this RDD.
*/
def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = {
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
new JavaPairRDD(rdd.map(f)(cm))(f.keyType(), f.valueType())
}
@@ -118,7 +119,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType())
}
@@ -158,18 +159,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* elements (a, b) where a is in `this` and b is in `other`.
*/
def cartesian[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] =
- JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest,
- other.classManifest)
+ JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag,
+ other.classTag)
/**
* Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
* mapping to that key.
*/
def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = {
- implicit val kcm: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val vcm: ClassManifest[JList[T]] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]]
+ implicit val kcm: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val vcm: ClassTag[JList[T]] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]]
JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm)
}
@@ -178,10 +179,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* mapping to that key.
*/
def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = {
- implicit val kcm: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val vcm: ClassManifest[JList[T]] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]]
+ implicit val kcm: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val vcm: ClassTag[JList[T]] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]]
JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm)
}
@@ -209,7 +210,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* a map on the other).
*/
def zip[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = {
- JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest)
+ JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classTag))(classTag, other.classTag)
}
/**
@@ -224,7 +225,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator(
f.apply(asJavaIterator(x), asJavaIterator(y)).iterator())
JavaRDD.fromRDD(
- rdd.zipPartitions(other.rdd)(fn)(other.classManifest, f.elementType()))(f.elementType())
+ rdd.zipPartitions(other.rdd)(fn)(other.classTag, f.elementType()))(f.elementType())
}
// Actions (launch a job to return a value to the user program)
@@ -356,7 +357,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* Creates tuples of the elements in this RDD by applying `f`.
*/
def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = {
- implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+ implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
JavaPairRDD.fromRDD(rdd.keyBy(f))
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 8869e072bf..d5b77357a2 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -21,6 +21,7 @@ import java.util.{Map => JMap}
import scala.collection.JavaConversions
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.InputFormat
@@ -82,8 +83,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/** Distribute a local Scala collection to form an RDD. */
def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)
}
@@ -94,10 +95,10 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/** Distribute a local Scala collection to form an RDD. */
def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int)
: JavaPairRDD[K, V] = {
- implicit val kcm: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val vcm: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val kcm: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val vcm: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices))
}
@@ -132,16 +133,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
valueClass: Class[V],
minSplits: Int
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits))
}
/**Get an RDD for a Hadoop SequenceFile. */
def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]):
JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass))
}
@@ -153,8 +154,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
* that there's very little effort required to save arbitrary objects.
*/
def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
sc.objectFile(path, minSplits)(cm)
}
@@ -166,8 +167,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
* that there's very little effort required to save arbitrary objects.
*/
def objectFile[T](path: String): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
sc.objectFile(path)(cm)
}
@@ -183,8 +184,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
valueClass: Class[V],
minSplits: Int
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits))
}
@@ -199,8 +200,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
keyClass: Class[K],
valueClass: Class[V]
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass))
}
@@ -212,8 +213,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
valueClass: Class[V],
minSplits: Int
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits))
}
@@ -224,8 +225,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
keyClass: Class[K],
valueClass: Class[V]
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopFile(path,
inputFormatClass, keyClass, valueClass))
}
@@ -240,8 +241,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
kClass: Class[K],
vClass: Class[V],
conf: Configuration): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(kClass)
- implicit val vcm = ClassManifest.fromClass(vClass)
+ implicit val kcm: ClassTag[K] = ClassTag(kClass)
+ implicit val vcm: ClassTag[V] = ClassTag(vClass)
new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf))
}
@@ -254,15 +255,15 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
fClass: Class[F],
kClass: Class[K],
vClass: Class[V]): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(kClass)
- implicit val vcm = ClassManifest.fromClass(vClass)
+ implicit val kcm: ClassTag[K] = ClassTag(kClass)
+ implicit val vcm: ClassTag[V] = ClassTag(vClass)
new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass))
}
/** Build the union of two or more RDDs. */
override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = {
val rdds: Seq[RDD[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd)
- implicit val cm: ClassManifest[T] = first.classManifest
+ implicit val cm: ClassTag[T] = first.classTag
sc.union(rdds)(cm)
}
@@ -270,9 +271,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]])
: JavaPairRDD[K, V] = {
val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd)
- implicit val cm: ClassManifest[(K, V)] = first.classManifest
- implicit val kcm: ClassManifest[K] = first.kManifest
- implicit val vcm: ClassManifest[V] = first.vManifest
+ implicit val cm: ClassTag[(K, V)] = first.classTag
+ implicit val kcm: ClassTag[K] = first.kClassTag
+ implicit val vcm: ClassTag[V] = first.vClassTag
new JavaPairRDD(sc.union(rdds)(cm))(kcm, vcm)
}
@@ -405,8 +406,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
}
protected def checkpointFile[T](path: String): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
new JavaRDD(sc.checkpointFile(path))
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
index 158539a846..b7c0d78e33 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
@@ -17,6 +17,8 @@
package org.apache.spark.api.java.function
+import scala.reflect.ClassTag
+
/**
* A function that returns zero or more output records from each input record.
*/
@@ -24,5 +26,5 @@ abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]]
@throws(classOf[Exception])
def call(x: T) : java.lang.Iterable[R]
- def elementType() : ClassManifest[R] = ClassManifest.Any.asInstanceOf[ClassManifest[R]]
+ def elementType() : ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]]
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
index 5ef6a814f5..7a505df4be 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
@@ -17,6 +17,8 @@
package org.apache.spark.api.java.function
+import scala.reflect.ClassTag
+
/**
* A function that takes two inputs and returns zero or more output records.
*/
@@ -24,5 +26,5 @@ abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Itera
@throws(classOf[Exception])
def call(a: A, b:B) : java.lang.Iterable[C]
- def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]]
+ def elementType() : ClassTag[C] = ClassTag.Any.asInstanceOf[ClassTag[C]]
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
index b9070cfd83..f9dae6ed34 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
@@ -17,8 +17,8 @@
package org.apache.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;
@@ -32,8 +32,8 @@ import java.io.Serializable;
public abstract class Function<T, R> extends WrappedFunction1<T, R> implements Serializable {
public abstract R call(T t) throws Exception;
- public ClassManifest<R> returnType() {
- return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<R> returnType() {
+ return (ClassTag<R>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
index d4c9154869..1659bfc552 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
@@ -17,8 +17,8 @@
package org.apache.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;
@@ -31,8 +31,8 @@ public abstract class Function2<T1, T2, R> extends WrappedFunction2<T1, T2, R>
public abstract R call(T1 t1, T2 t2) throws Exception;
- public ClassManifest<R> returnType() {
- return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<R> returnType() {
+ return (ClassTag<R>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
index c0e5544b7d..5a5c9b6296 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
@@ -18,8 +18,8 @@
package org.apache.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;
@@ -36,11 +36,11 @@ public abstract class PairFlatMapFunction<T, K, V>
public abstract Iterable<Tuple2<K, V>> call(T t) throws Exception;
- public ClassManifest<K> keyType() {
- return (ClassManifest<K>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<K> keyType() {
+ return (ClassTag<K>) ClassTag$.MODULE$.apply(Object.class);
}
- public ClassManifest<V> valueType() {
- return (ClassManifest<V>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<V> valueType() {
+ return (ClassTag<V>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
index 40480fe8e8..4c39f483e5 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
@@ -18,8 +18,8 @@
package org.apache.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;
@@ -35,11 +35,11 @@ public abstract class PairFunction<T, K, V>
public abstract Tuple2<K, V> call(T t) throws Exception;
- public ClassManifest<K> keyType() {
- return (ClassManifest<K>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<K> keyType() {
+ return (ClassTag<K>) ClassTag$.MODULE$.apply(Object.class);
}
- public ClassManifest<V> valueType() {
- return (ClassManifest<V>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<V> valueType() {
+ return (ClassTag<V>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index ccd3833964..cb2db77f39 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -22,6 +22,7 @@ import java.net._
import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
import org.apache.spark.broadcast.Broadcast
@@ -30,8 +31,7 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.PipedRDD
import org.apache.spark.util.Utils
-
-private[spark] class PythonRDD[T: ClassManifest](
+private[spark] class PythonRDD[T: ClassTag](
parent: RDD[T],
command: Seq[String],
envVars: JMap[String, String],
@@ -166,7 +166,7 @@ private[spark] class PythonRDD[T: ClassManifest](
case eof: EOFException => {
throw new SparkException("Python worker exited unexpectedly (crashed)", eof)
}
- case e => throw e
+ case e : Throwable => throw e
}
}
@@ -226,7 +226,7 @@ private[spark] object PythonRDD {
val arr = elem.asInstanceOf[Array[Byte]]
dOut.writeInt(arr.length)
dOut.write(arr)
- } else if (elem.isInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]) {
+ } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) {
val t = elem.asInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]
val length = t._1.length + t._2.length - 3 - 3 + 4 // stripPickle() removes 3 bytes
dOut.writeInt(length)
@@ -265,7 +265,7 @@ private[spark] object PythonRDD {
}
} catch {
case eof: EOFException => {}
- case e => throw e
+ case e : Throwable => throw e
}
JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
}
@@ -284,7 +284,7 @@ private[spark] object PythonRDD {
}
def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = {
- implicit val cm : ClassManifest[T] = rdd.elementClassManifest
+ implicit val cm : ClassTag[T] = rdd.elementClassTag
rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 10161c8204..6a7d5a85ba 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -34,11 +34,11 @@ import scala.collection.mutable.ArrayBuffer
*/
private[spark]
class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging {
-
+
private val localHostname = Utils.localHostName()
private val masterActorSystems = ArrayBuffer[ActorSystem]()
private val workerActorSystems = ArrayBuffer[ActorSystem]()
-
+
def start(): String = {
logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
@@ -60,10 +60,13 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
def stop() {
logInfo("Shutting down local Spark cluster.")
// Stop the workers before the master so they don't get upset that it disconnected
+ // TODO: In Akka 2.1.x, ActorSystem.awaitTermination hangs when you have remote actors!
+ // This is unfortunate, but for now we just comment it out.
workerActorSystems.foreach(_.shutdown())
- workerActorSystems.foreach(_.awaitTermination())
-
+ //workerActorSystems.foreach(_.awaitTermination())
masterActorSystems.foreach(_.shutdown())
- masterActorSystems.foreach(_.awaitTermination())
+ //masterActorSystems.foreach(_.awaitTermination())
+ masterActorSystems.clear()
+ workerActorSystems.clear()
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
index a342dd724a..164386782c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
@@ -19,14 +19,14 @@ package org.apache.spark.deploy.client
import java.util.concurrent.TimeoutException
+import scala.concurrent.duration._
+import scala.concurrent.Await
+
import akka.actor._
import akka.actor.Terminated
+import akka.pattern.AskTimeoutException
import akka.pattern.ask
-import akka.util.Duration
-import akka.remote.RemoteClientDisconnected
-import akka.remote.RemoteClientLifeCycleEvent
-import akka.remote.RemoteClientShutdown
-import akka.dispatch.Await
+import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent}
import org.apache.spark.Logging
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
@@ -59,7 +59,7 @@ private[spark] class Client(
master = context.actorFor(Master.toAkkaUrl(masterUrl))
masterAddress = master.path.address
master ! RegisterApplication(appDescription)
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
context.watch(master) // Doesn't work with remote actors, but useful for testing
} catch {
case e: Exception =>
@@ -97,12 +97,12 @@ private[spark] class Client(
markDisconnected()
context.stop(self)
- case RemoteClientDisconnected(transport, address) if address == masterAddress =>
+ case DisassociatedEvent(_, address, _) if address == masterAddress =>
logError("Connection to master failed; stopping client")
markDisconnected()
context.stop(self)
- case RemoteClientShutdown(transport, address) if address == masterAddress =>
+ case AssociationErrorEvent(_, _, address, _) if address == masterAddress =>
logError("Connection to master failed; stopping client")
markDisconnected()
context.stop(self)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index bde59905bc..cb0fe6a850 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -21,14 +21,12 @@ import java.util.Date
import java.text.SimpleDateFormat
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.concurrent.Await
+import scala.concurrent.duration._
import akka.actor._
-import akka.actor.Terminated
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
-import akka.util.duration._
-import akka.util.Timeout
+import akka.remote._
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
@@ -36,7 +34,23 @@ import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.ui.MasterWebUI
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{Utils, AkkaUtils}
-import akka.util.{Duration, Timeout}
+import akka.util.Timeout
+import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed
+import org.apache.spark.deploy.DeployMessages.KillExecutor
+import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
+import scala.Some
+import org.apache.spark.deploy.DeployMessages.WebUIPortResponse
+import org.apache.spark.deploy.DeployMessages.LaunchExecutor
+import org.apache.spark.deploy.DeployMessages.RegisteredApplication
+import org.apache.spark.deploy.DeployMessages.RegisterWorker
+import org.apache.spark.deploy.DeployMessages.ExecutorUpdated
+import org.apache.spark.deploy.DeployMessages.MasterStateResponse
+import org.apache.spark.deploy.DeployMessages.ExecutorAdded
+import org.apache.spark.deploy.DeployMessages.RegisterApplication
+import org.apache.spark.deploy.DeployMessages.ApplicationRemoved
+import org.apache.spark.deploy.DeployMessages.Heartbeat
+import org.apache.spark.deploy.DeployMessages.RegisteredWorker
+import akka.actor.Terminated
private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
@@ -82,8 +96,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
override def preStart() {
logInfo("Starting Spark master at spark://" + host + ":" + port)
// Listen for remote client disconnection events, since they don't go through Akka's watch()
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi.start()
+ import context.dispatcher
context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut)
masterMetricsSystem.registerSource(masterSource)
@@ -165,13 +180,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
actorToApp.get(actor).foreach(finishApplication)
}
- case RemoteClientDisconnected(transport, address) => {
+ case DisassociatedEvent(_, address, _) => {
// The disconnected client could've been either a worker or an app; remove whichever it was
addressToWorker.get(address).foreach(removeWorker)
addressToApp.get(address).foreach(finishApplication)
}
- case RemoteClientShutdown(transport, address) => {
+ case AssociationErrorEvent(_, _, address, _) => {
// The disconnected client could've been either a worker or an app; remove whichever it was
addressToWorker.get(address).foreach(removeWorker)
addressToApp.get(address).foreach(finishApplication)
@@ -376,11 +391,11 @@ private[spark] object Master {
actorSystem.awaitTermination()
}
- /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
+ /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
def toAkkaUrl(sparkUrl: String): String = {
sparkUrl match {
case sparkUrlRegex(host, port) =>
- "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
+ "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
case _ =>
throw new SparkException("Invalid master URL: " + sparkUrl)
}
@@ -388,7 +403,7 @@ private[spark] object Master {
def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = {
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
- val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName)
+ val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName)
val timeoutDuration = Duration.create(
System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
implicit val timeout = Timeout(timeoutDuration)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index f4e574d15d..3b983c19eb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -19,9 +19,10 @@ package org.apache.spark.deploy.master.ui
import scala.xml.Node
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.util.duration._
+
+import scala.concurrent.Await
+import scala.concurrent.duration._
import javax.servlet.http.HttpServletRequest
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index d7a57229b0..65e7a14e7a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -21,9 +21,9 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import akka.dispatch.Await
+import scala.concurrent.Await
import akka.pattern.ask
-import akka.util.duration._
+import scala.concurrent.duration._
import net.liftweb.json.JsonAST.JValue
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index f4df729e87..a211ce2b42 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -17,7 +17,7 @@
package org.apache.spark.deploy.master.ui
-import akka.util.Duration
+import scala.concurrent.duration._
import javax.servlet.http.HttpServletRequest
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 09530beb3b..3904b701b2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -22,12 +22,12 @@ import java.util.Date
import java.io.File
import scala.collection.mutable.HashMap
+import scala.concurrent.duration._
import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
-import akka.util.duration._
+import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent}
-import org.apache.spark.{Logging}
+import org.apache.spark.Logging
import org.apache.spark.deploy.ExecutorState
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
@@ -113,15 +113,17 @@ private[spark] class Worker(
logInfo("Connecting to master " + masterUrl)
master = context.actorFor(Master.toAkkaUrl(masterUrl))
master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress)
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
context.watch(master) // Doesn't work with remote actors, but useful for testing
}
+ import context.dispatcher
+
override def receive = {
case RegisteredWorker(url) =>
masterWebUiUrl = url
logInfo("Successfully registered with master")
- context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) {
+ context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) {
master ! Heartbeat(workerId)
}
@@ -163,7 +165,7 @@ private[spark] class Worker(
logInfo("Asked to kill unknown executor " + fullId)
}
- case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
+ case DisassociatedEvent(_, _, _) =>
masterDisconnected()
case RequestWorkerState => {
@@ -205,8 +207,8 @@ private[spark] object Worker {
// The LocalSparkCluster runs multiple local sparkWorkerX actor systems
val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
- val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory,
- masterUrl, workDir)), name = "Worker")
+ actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
+ masterUrl, workDir), name = "Worker")
(actorSystem, boundPort)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index d2d3617498..1a768d501f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -21,9 +21,10 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import akka.dispatch.Await
+import scala.concurrent.duration._
+import scala.concurrent.Await
+
import akka.pattern.ask
-import akka.util.duration._
import net.liftweb.json.JsonAST.JValue
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 95d6007f3b..07bc479c83 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -17,7 +17,10 @@
package org.apache.spark.deploy.worker.ui
-import akka.util.{Duration, Timeout}
+import akka.actor.ActorRef
+import akka.util.Timeout
+
+import scala.concurrent.duration._
import java.io.{FileInputStream, File}
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index ceae3b8289..99a4a95e82 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -105,7 +105,7 @@ private[spark] class Executor(
SparkEnv.set(env)
env.metricsSystem.registerSource(executorSource)
- private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size")
+ private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size")
// Start worker thread pool
val threadPool = new ThreadPoolExecutor(
diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
index 7839023868..f705a5631a 100644
--- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
@@ -19,13 +19,25 @@ package org.apache.spark.executor
import java.nio.ByteBuffer
-import akka.actor.{ActorRef, Actor, Props, Terminated}
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
+import akka.actor._
+import akka.remote._
import org.apache.spark.{Logging, SparkEnv}
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._
import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask
+import akka.remote.DisassociatedEvent
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask
+import akka.remote.AssociationErrorEvent
+import akka.remote.DisassociatedEvent
+import akka.actor.Terminated
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed
private[spark] class StandaloneExecutorBackend(
@@ -40,14 +52,14 @@ private[spark] class StandaloneExecutorBackend(
Utils.checkHostPort(hostPort, "Expected hostport")
var executor: Executor = null
- var driver: ActorRef = null
+ var driver: ActorSelection = null
override def preStart() {
logInfo("Connecting to driver: " + driverUrl)
- driver = context.actorFor(driverUrl)
+ driver = context.actorSelection(driverUrl)
driver ! RegisterExecutor(executorId, hostPort, cores)
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
- context.watch(driver) // Doesn't work with remote actors, but useful for testing
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+ // context.watch(driver) // Doesn't work with remote actors, but useful for testing
}
override def receive = {
@@ -69,7 +81,7 @@ private[spark] class StandaloneExecutorBackend(
executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
}
- case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
+ case DisassociatedEvent(_, _, _) =>
logError("Driver terminated or disconnected! Shutting down.")
System.exit(1)
}
@@ -90,8 +102,8 @@ private[spark] object StandaloneExecutorBackend {
// set it
val sparkHostPort = hostname + ":" + boundPort
System.setProperty("spark.hostPort", sparkHostPort)
- val actor = actorSystem.actorOf(
- Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)),
+ actorSystem.actorOf(
+ Props(classOf[StandaloneExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
name = "Executor")
actorSystem.awaitTermination()
}
diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
index e15a839c4e..c24fd48c04 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -31,11 +31,11 @@ import scala.collection.mutable.SynchronizedMap
import scala.collection.mutable.SynchronizedQueue
import scala.collection.mutable.ArrayBuffer
-import akka.dispatch.{Await, Promise, ExecutionContext, Future}
-import akka.util.Duration
-import akka.util.duration._
-import org.apache.spark.util.Utils
+import scala.concurrent.{Await, Promise, ExecutionContext, Future}
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration._
+import org.apache.spark.util.Utils
private[spark] class ConnectionManager(port: Int) extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
index 8d9ad9604d..4f5742d29b 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
@@ -25,8 +25,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/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
index 9b0c882481..4fb7f3aace 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
@@ -18,6 +18,7 @@
package org.apache.spark.rdd
import java.io.{ObjectOutputStream, IOException}
+import scala.reflect.ClassTag
import org.apache.spark._
@@ -43,7 +44,7 @@ class CartesianPartition(
}
private[spark]
-class CartesianRDD[T: ClassManifest, U:ClassManifest](
+class CartesianRDD[T: ClassTag, U:ClassTag](
sc: SparkContext,
var rdd1 : RDD[T],
var rdd2 : RDD[U])
diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index 3311757189..3f4d4ad46a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -17,6 +17,7 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
import org.apache.spark._
import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
import org.apache.hadoop.conf.Configuration
@@ -32,7 +33,7 @@ private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
* This RDD represents a RDD checkpoint file (similar to HadoopRDD).
*/
private[spark]
-class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: String)
+class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String)
extends RDD[T](sc, Nil) {
@transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration)
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index aca0146884..e72f86fb13 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -19,6 +19,7 @@ package org.apache.spark.rdd
import java.sql.{Connection, ResultSet}
+import scala.reflect.ClassTag
import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
import org.apache.spark.util.NextIterator
@@ -45,7 +46,7 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e
* This should only call getInt, getString, etc; the RDD takes care of calling next.
* The default maps a ResultSet to an array of Object.
*/
-class JdbcRDD[T: ClassManifest](
+class JdbcRDD[T: ClassTag](
sc: SparkContext,
getConnection: () => Connection,
sql: String,
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index a47c512275..aed585e6a1 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -25,6 +25,7 @@ import java.util.{HashMap => JHashMap}
import scala.collection.{mutable, Map}
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConversions._
+import scala.reflect.{ ClassTag, classTag}
import org.apache.hadoop.mapred._
import org.apache.hadoop.io.compress.CompressionCodec
@@ -409,7 +410,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](Seq(self, other), partitioner)
- val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
+ val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag)
prfs.mapValues { case Seq(vs, ws) =>
(vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
}
@@ -425,7 +426,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner)
- val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
+ val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag)
prfs.mapValues { case Seq(vs, w1s, w2s) =>
(vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]])
}
@@ -482,15 +483,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
- def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] =
subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size)))
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
- def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =
subtractByKey(other, new HashPartitioner(numPartitions))
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
- def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] =
new SubtractedRDD[K, V, W](self, other, p)
/**
@@ -519,8 +520,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class
* supporting the key and value types K and V in this RDD.
*/
- def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) {
- saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) {
+ saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -529,16 +530,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* supplied codec.
*/
def saveAsHadoopFile[F <: OutputFormat[K, V]](
- path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) {
- saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec)
+ path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) {
+ saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]], codec)
}
/**
* Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
* (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
*/
- def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) {
- saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) {
+ saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -692,11 +693,11 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
*/
def values: RDD[V] = self.map(_._2)
- private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure
+ private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass
- private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure
+ private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass
}
-private[spark] object Manifests {
- val seqSeqManifest = classManifest[Seq[Seq[_]]]
+private[spark] object ClassTags {
+ val seqSeqClassTag = classTag[Seq[Seq[_]]]
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
index 6dbd4309aa..78fe0cdcdb 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -20,13 +20,15 @@ package org.apache.spark.rdd
import scala.collection.immutable.NumericRange
import scala.collection.mutable.ArrayBuffer
import scala.collection.Map
+import scala.reflect.ClassTag
+
import org.apache.spark._
import java.io._
import scala.Serializable
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.util.Utils
-private[spark] class ParallelCollectionPartition[T: ClassManifest](
+private[spark] class ParallelCollectionPartition[T: ClassTag](
var rddId: Long,
var slice: Int,
var values: Seq[T])
@@ -78,7 +80,7 @@ private[spark] class ParallelCollectionPartition[T: ClassManifest](
}
}
-private[spark] class ParallelCollectionRDD[T: ClassManifest](
+private[spark] class ParallelCollectionRDD[T: ClassTag](
@transient sc: SparkContext,
@transient data: Seq[T],
numSlices: Int,
@@ -108,7 +110,7 @@ private object ParallelCollectionRDD {
* collections specially, encoding the slices as other Ranges to minimize memory cost. This makes
* it efficient to run Spark over RDDs representing large sets of numbers.
*/
- def slice[T: ClassManifest](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = {
+ def slice[T: ClassTag](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = {
if (numSlices < 1) {
throw new IllegalArgumentException("Positive number of slices required")
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
index 165cd412fc..bb9b309a70 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
@@ -17,6 +17,8 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.spark.{NarrowDependency, SparkEnv, Partition, TaskContext}
@@ -47,7 +49,7 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo
* and the execution DAG has a filter on the key, we can avoid launching tasks
* on partitions that don't have the range covering the key.
*/
-class PartitionPruningRDD[T: ClassManifest](
+class PartitionPruningRDD[T: ClassTag](
@transient prev: RDD[T],
@transient partitionFilterFunc: Int => Boolean)
extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) {
@@ -67,6 +69,6 @@ object PartitionPruningRDD {
* when its type T is not known at compile time.
*/
def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = {
- new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassManifest)
+ new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassTag)
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
index d5304ab0ae..1dbbe39898 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -24,6 +24,7 @@ import scala.collection.Map
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
+import scala.reflect.ClassTag
import org.apache.spark.{SparkEnv, Partition, TaskContext}
import org.apache.spark.broadcast.Broadcast
@@ -33,7 +34,7 @@ import org.apache.spark.broadcast.Broadcast
* An RDD that pipes the contents of each parent partition through an external command
* (printing them one per line) and returns the output as a collection of strings.
*/
-class PipedRDD[T: ClassManifest](
+class PipedRDD[T: ClassTag](
prev: RDD[T],
command: Seq[String],
envVars: Map[String, String],
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 1893627ee2..731ef90c90 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -23,6 +23,9 @@ import scala.collection.Map
import scala.collection.JavaConversions.mapAsScalaMap
import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashMap
+import scala.reflect.{classTag, ClassTag}
+
import org.apache.hadoop.io.BytesWritable
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.io.NullWritable
@@ -69,7 +72,7 @@ import org.apache.spark._
* [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details
* on RDD internals.
*/
-abstract class RDD[T: ClassManifest](
+abstract class RDD[T: ClassTag](
@transient private var sc: SparkContext,
@transient private var deps: Seq[Dependency[_]]
) extends Serializable with Logging {
@@ -243,13 +246,13 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD by applying a function to all elements of this RDD.
*/
- def map[U: ClassManifest](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f))
+ def map[U: ClassTag](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f))
/**
* Return a new RDD by first applying a function to all elements of this
* RDD, and then flattening the results.
*/
- def flatMap[U: ClassManifest](f: T => TraversableOnce[U]): RDD[U] =
+ def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] =
new FlatMappedRDD(this, sc.clean(f))
/**
@@ -361,25 +364,25 @@ abstract class RDD[T: ClassManifest](
* Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of
* elements (a, b) where a is in `this` and b is in `other`.
*/
- def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other)
+ def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other)
/**
* Return an RDD of grouped items.
*/
- def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] =
+ def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] =
groupBy[K](f, defaultPartitioner(this))
/**
* Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
* mapping to that key.
*/
- def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] =
+ def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] =
groupBy(f, new HashPartitioner(numPartitions))
/**
* Return an RDD of grouped items.
*/
- def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = {
+ def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = {
val cleanF = sc.clean(f)
this.map(t => (cleanF(t), t)).groupByKey(p)
}
@@ -426,7 +429,7 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD by applying a function to each partition of this RDD.
*/
- def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U],
+ def mapPartitions[U: ClassTag](f: Iterator[T] => Iterator[U],
preservesPartitioning: Boolean = false): RDD[U] =
new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
@@ -434,7 +437,7 @@ abstract class RDD[T: ClassManifest](
* Return a new RDD by applying a function to each partition of this RDD, while tracking the index
* of the original partition.
*/
- def mapPartitionsWithIndex[U: ClassManifest](
+ def mapPartitionsWithIndex[U: ClassTag](
f: (Int, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean = false): RDD[U] =
new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
@@ -444,7 +447,7 @@ abstract class RDD[T: ClassManifest](
* of the original partition.
*/
@deprecated("use mapPartitionsWithIndex", "0.7.0")
- def mapPartitionsWithSplit[U: ClassManifest](
+ def mapPartitionsWithSplit[U: ClassTag](
f: (Int, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean = false): RDD[U] =
new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
@@ -454,7 +457,7 @@ abstract class RDD[T: ClassManifest](
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def mapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false)
+ def mapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false)
(f:(T, A) => U): RDD[U] = {
def iterF(index: Int, iter: Iterator[T]): Iterator[U] = {
val a = constructA(index)
@@ -468,7 +471,7 @@ abstract class RDD[T: ClassManifest](
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def flatMapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false)
+ def flatMapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false)
(f:(T, A) => Seq[U]): RDD[U] = {
def iterF(index: Int, iter: Iterator[T]): Iterator[U] = {
val a = constructA(index)
@@ -482,7 +485,7 @@ abstract class RDD[T: ClassManifest](
* This additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def foreachWith[A: ClassManifest](constructA: Int => A)
+ def foreachWith[A: ClassTag](constructA: Int => A)
(f:(T, A) => Unit) {
def iterF(index: Int, iter: Iterator[T]): Iterator[T] = {
val a = constructA(index)
@@ -496,7 +499,7 @@ abstract class RDD[T: ClassManifest](
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def filterWith[A: ClassManifest](constructA: Int => A)
+ def filterWith[A: ClassTag](constructA: Int => A)
(p:(T, A) => Boolean): RDD[T] = {
def iterF(index: Int, iter: Iterator[T]): Iterator[T] = {
val a = constructA(index)
@@ -511,7 +514,7 @@ abstract class RDD[T: ClassManifest](
* partitions* and the *same number of elements in each partition* (e.g. one was made through
* a map on the other).
*/
- def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other)
+ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other)
/**
* Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by
@@ -519,17 +522,17 @@ abstract class RDD[T: ClassManifest](
* *same number of partitions*, but does *not* require them to have the same number
* of elements in each partition.
*/
- def zipPartitions[B: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, V: ClassTag]
(rdd2: RDD[B])
(f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2)
- def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C])
(f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3)
- def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D])
(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4)
@@ -569,7 +572,7 @@ abstract class RDD[T: ClassManifest](
/**
* Return an RDD that contains all matching values by applying `f`.
*/
- def collect[U: ClassManifest](f: PartialFunction[T, U]): RDD[U] = {
+ def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = {
filter(f.isDefinedAt).map(f)
}
@@ -659,7 +662,7 @@ abstract class RDD[T: ClassManifest](
* allowed to modify and return their first argument instead of creating a new U to avoid memory
* allocation.
*/
- def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = {
+ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = {
// Clone the zero value since we will also be serializing it as part of tasks
var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance())
val cleanSeqOp = sc.clean(seqOp)
@@ -706,7 +709,7 @@ abstract class RDD[T: ClassManifest](
* combine step happens locally on the master, equivalent to running a single reduce task.
*/
def countByValue(): Map[T, Long] = {
- if (elementClassManifest.erasure.isArray) {
+ if (elementClassTag.runtimeClass.isArray) {
throw new SparkException("countByValue() does not support arrays")
}
// TODO: This should perhaps be distributed by default.
@@ -737,7 +740,7 @@ abstract class RDD[T: ClassManifest](
timeout: Long,
confidence: Double = 0.95
): PartialResult[Map[T, BoundedDouble]] = {
- if (elementClassManifest.erasure.isArray) {
+ if (elementClassTag.runtimeClass.isArray) {
throw new SparkException("countByValueApprox() does not support arrays")
}
val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) =>
@@ -904,12 +907,12 @@ abstract class RDD[T: ClassManifest](
/** Record user function generating this RDD. */
private[spark] val origin = Utils.formatSparkCallSite
- private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
+ private[spark] def elementClassTag: ClassTag[T] = classTag[T]
private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None
/** Returns the first parent RDD */
- protected[spark] def firstParent[U: ClassManifest] = {
+ protected[spark] def firstParent[U: ClassTag] = {
dependencies.head.rdd.asInstanceOf[RDD[U]]
}
@@ -971,7 +974,7 @@ abstract class RDD[T: ClassManifest](
origin)
def toJavaRDD() : JavaRDD[T] = {
- new JavaRDD(this)(elementClassManifest)
+ new JavaRDD(this)(elementClassTag)
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
index 6009a41570..3b56e45aa9 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
@@ -17,6 +17,8 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.hadoop.fs.Path
import org.apache.hadoop.conf.Configuration
@@ -38,7 +40,7 @@ private[spark] object CheckpointState extends Enumeration {
* manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations
* of the checkpointed RDD.
*/
-private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
+private[spark] class RDDCheckpointData[T: ClassTag](rdd: RDD[T])
extends Logging with Serializable {
import CheckpointState._
diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
index 2c5253ae30..d433670cc2 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
@@ -17,6 +17,7 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
import java.util.Random
import cern.jet.random.Poisson
@@ -29,9 +30,9 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition
override val index: Int = prev.index
}
-class SampledRDD[T: ClassManifest](
+class SampledRDD[T: ClassTag](
prev: RDD[T],
- withReplacement: Boolean,
+ withReplacement: Boolean,
frac: Double,
seed: Int)
extends RDD[T](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
index 5fe4676029..2d1bd5b481 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.spark.rdd
+import scala.reflect.{ ClassTag, classTag}
+
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapred.SequenceFileOutputFormat
import org.apache.hadoop.io.compress.CompressionCodec
@@ -32,15 +33,15 @@ import org.apache.spark.Logging
*
* Import `org.apache.spark.SparkContext._` at the top of their program to use these functions.
*/
-class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest](
+class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag](
self: RDD[(K, V)])
extends Logging
with Serializable {
- private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = {
+ private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = {
val c = {
- if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) {
- classManifest[T].erasure
+ if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) {
+ classTag[T].runtimeClass
} else {
// We get the type of the Writable class by looking at the apply method which converts
// from T to Writable. Since we have two apply methods we filter out the one which
diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index 8c1a29dfff..85c512f3de 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -18,8 +18,11 @@
package org.apache.spark.rdd
import java.util.{HashMap => JHashMap}
+
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
import org.apache.spark.Partitioner
import org.apache.spark.Dependency
import org.apache.spark.TaskContext
@@ -45,7 +48,7 @@ import org.apache.spark.OneToOneDependency
* you can use `rdd1`'s partitioner/partition size and not worry about running
* out of memory because of the size of `rdd2`.
*/
-private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest](
+private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag](
@transient var rdd1: RDD[_ <: Product2[K, V]],
@transient var rdd2: RDD[_ <: Product2[K, W]],
part: Partitioner)
diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
index ae8a9f36a6..08a41ac558 100644
--- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
@@ -18,10 +18,13 @@
package org.apache.spark.rdd
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
import org.apache.spark.{Dependency, RangeDependency, SparkContext, Partition, TaskContext}
+
import java.io.{ObjectOutputStream, IOException}
-private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
+private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int)
extends Partition {
var split: Partition = rdd.partitions(splitIndex)
@@ -40,7 +43,7 @@ private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], spl
}
}
-class UnionRDD[T: ClassManifest](
+class UnionRDD[T: ClassTag](
sc: SparkContext,
@transient var rdds: Seq[RDD[T]])
extends RDD[T](sc, Nil) { // Nil since we implement getDependencies
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
index 31e6fd519d..e02c17bf45 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
@@ -19,6 +19,7 @@ package org.apache.spark.rdd
import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext}
import java.io.{ObjectOutputStream, IOException}
+import scala.reflect.ClassTag
private[spark] class ZippedPartitionsPartition(
idx: Int,
@@ -37,7 +38,7 @@ private[spark] class ZippedPartitionsPartition(
}
}
-abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
+abstract class ZippedPartitionsBaseRDD[V: ClassTag](
sc: SparkContext,
var rdds: Seq[RDD[_]])
extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) {
@@ -72,7 +73,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
}
}
-class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest](
+class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag](
sc: SparkContext,
f: (Iterator[A], Iterator[B]) => Iterator[V],
var rdd1: RDD[A],
@@ -92,7 +93,7 @@ class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest]
}
class ZippedPartitionsRDD3
- [A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest](
+ [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag](
sc: SparkContext,
f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V],
var rdd1: RDD[A],
@@ -116,7 +117,7 @@ class ZippedPartitionsRDD3
}
class ZippedPartitionsRDD4
- [A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest](
+ [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag](
sc: SparkContext,
f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V],
var rdd1: RDD[A],
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
index 567b67dfee..fb5b070c18 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
@@ -18,10 +18,12 @@
package org.apache.spark.rdd
import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext}
+
import java.io.{ObjectOutputStream, IOException}
+import scala.reflect.ClassTag
-private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
+private[spark] class ZippedPartition[T: ClassTag, U: ClassTag](
idx: Int,
@transient rdd1: RDD[T],
@transient rdd2: RDD[U]
@@ -42,7 +44,7 @@ private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
}
}
-class ZippedRDD[T: ClassManifest, U: ClassManifest](
+class ZippedRDD[T: ClassTag, U: ClassTag](
sc: SparkContext,
var rdd1: RDD[T],
var rdd2: RDD[U])
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 8a55df4af0..693d8a7c5d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -23,6 +23,7 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
+import scala.reflect.ClassTag
import org.apache.spark._
import org.apache.spark.rdd.RDD
@@ -267,7 +268,7 @@ class DAGScheduler(
* The job is assumed to have at least one partition; zero partition jobs should be handled
* without a JobSubmitted event.
*/
- private[scheduler] def prepareJob[T, U: ClassManifest](
+ private[scheduler] def prepareJob[T, U: ClassTag](
finalRdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
@@ -285,7 +286,7 @@ class DAGScheduler(
(toSubmit, waiter)
}
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
finalRdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 9c49768c0c..fa83ae19d6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -42,7 +42,7 @@ private[spark] class SparkDeploySchedulerBackend(
super.start()
// The endpoint for executors to talk to us
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
StandaloneSchedulerBackend.ACTOR_NAME)
val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index f3aeea43d5..b6f0ec961a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -20,13 +20,12 @@ package org.apache.spark.scheduler.cluster
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.concurrent.Await
+import scala.concurrent.duration._
import akka.actor._
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
-import akka.util.Duration
-import akka.util.duration._
+import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
import org.apache.spark.{SparkException, Logging, TaskState}
import org.apache.spark.scheduler.TaskDescription
@@ -55,10 +54,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
override def preStart() {
// Listen for remote client disconnection events, since they don't go through Akka's watch()
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
// Periodically revive offers to allow delay scheduling to work
val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong
+ import context.dispatcher
context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
}
@@ -102,11 +102,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
case Terminated(actor) =>
actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
- case RemoteClientDisconnected(transport, address) =>
- addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected"))
+ case DisassociatedEvent(_, remoteAddress, _) =>
+ addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected"))
- case RemoteClientShutdown(transport, address) =>
- addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown"))
+ case AssociationErrorEvent(_, _, remoteAddress, _) =>
+ addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown"))
}
// Make fake resource offers on all executors
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 8f2eef9a53..bf4040fafc 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -119,7 +119,7 @@ private[spark] class CoarseMesosSchedulerBackend(
}
val command = CommandInfo.newBuilder()
.setEnvironment(environment)
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
System.getProperty("spark.driver.host"),
System.getProperty("spark.driver.port"),
StandaloneSchedulerBackend.ACTOR_NAME)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 60fdc5f2ee..13b98a51a1 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -23,9 +23,9 @@ import java.nio.{ByteBuffer, MappedByteBuffer}
import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet}
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 it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
@@ -984,6 +984,7 @@ private[spark] object BlockManager extends Logging {
}
def getHeartBeatFrequencyFromSystemProperties: Long =
+
System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
def getDisableHeartBeatsForTesting: Boolean =
@@ -1043,4 +1044,3 @@ private[spark] object BlockManager extends Logging {
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host))
}
}
-
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index cf463d6ffc..0c977f05d1 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -17,15 +17,24 @@
package org.apache.spark.storage
-import akka.actor.ActorRef
-import akka.dispatch.{Await, Future}
+import java.io._
+import java.util.{HashMap => JHashMap}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.util.Random
+
+import akka.actor.{Actor, ActorRef, ActorSystem, Props}
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.ExecutionContext.Implicits.global
+
import akka.pattern.ask
-import akka.util.Duration
+import scala.concurrent.duration._
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.storage.BlockManagerMessages._
-
private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index c7b23ab094..3776951782 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -23,10 +23,10 @@ import scala.collection.mutable
import scala.collection.JavaConversions._
import akka.actor.{Actor, ActorRef, Cancellable}
-import akka.dispatch.Future
import akka.pattern.ask
-import akka.util.Duration
-import akka.util.duration._
+
+import scala.concurrent.duration._
+import scala.concurrent.Future
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.storage.BlockManagerMessages._
@@ -65,6 +65,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/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
index e7eab374ad..c1ee2f3d00 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
@@ -17,7 +17,7 @@
package org.apache.spark.ui.jobs
-import akka.util.Duration
+import scala.concurrent.duration._
import java.text.SimpleDateFormat
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
index 1d633d374a..a5446b3fc3 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
@@ -17,7 +17,7 @@
package org.apache.spark.ui.storage
-import akka.util.Duration
+import scala.concurrent.duration._
import javax.servlet.http.HttpServletRequest
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index d4c5065c3f..af1c36b34d 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -19,10 +19,10 @@ package org.apache.spark.util
import akka.actor.{ActorSystem, ExtendedActorSystem}
import com.typesafe.config.ConfigFactory
-import akka.util.duration._
+import scala.concurrent.duration._
+import scala.concurrent.Await
import akka.remote.RemoteActorRefProvider
-
/**
* Various utility classes for working with Akka.
*/
@@ -36,37 +36,39 @@ private[spark] object AkkaUtils {
* host + port, if the system name is incorrect, Akka will drop the message.
*/
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", "60").toInt
+
val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
// 10 seconds is the default akka timeout, but in a cluster, we need higher by default.
val akkaWriteTimeout = System.getProperty("spark.akka.writeTimeout", "30").toInt
-
+
val akkaConf = ConfigFactory.parseString("""
akka.daemonic = on
- akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
+ akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
akka.stdout-loglevel = "ERROR"
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
- akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
- akka.remote.netty.hostname = "%s"
- akka.remote.netty.port = %d
- akka.remote.netty.connection-timeout = %ds
- akka.remote.netty.message-frame-size = %d MiB
- akka.remote.netty.execution-pool-size = %d
+ akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
+ akka.remote.netty.tcp.hostname = "%s"
+ akka.remote.netty.tcp.port = %d
+ akka.remote.netty.tcp.connection-timeout = %ds
+ akka.remote.netty.tcp.message-frame-size = %d MiB
+ akka.remote.netty.tcp.execution-pool-size = %d
akka.actor.default-dispatcher.throughput = %d
akka.remote.log-remote-lifecycle-events = %s
- akka.remote.netty.write-timeout = %ds
- """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
- lifecycleEvents, akkaWriteTimeout))
+ """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
+ lifecycleEvents))
val actorSystem = ActorSystem(name, akkaConf)
// 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[ExtendedActorSystem].provider
- val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get
- return (actorSystem, boundPort)
+ val boundPort = provider.getDefaultAddress.port.get
+ (actorSystem, boundPort)
}
+
}
diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
index 277de2f8a6..dbff571de9 100644
--- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
@@ -85,7 +85,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging {
}
override def filter(p: ((A, B)) => Boolean): Map[A, B] = {
- JavaConversions.asScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
+ JavaConversions.mapAsScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
}
override def empty: Map[A, B] = new TimeStampedHashMap[A, B]()
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 886f071503..be215fc127 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -20,13 +20,20 @@ package org.apache.spark.util
import java.io._
import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket}
import java.util.{Locale, Random, UUID}
+
import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor}
import java.util.regex.Pattern
-import scala.collection.Map
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
+
import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.collection.JavaConversions._
+import scala.collection.Map
import scala.io.Source
+import scala.reflect.ClassTag
+import scala.Some
+
import com.google.common.io.Files
import com.google.common.util.concurrent.ThreadFactoryBuilder
@@ -308,7 +315,7 @@ private[spark] object Utils extends Logging {
* result in a new collection. Unlike scala.util.Random.shuffle, this method
* uses a local random number generator, avoiding inter-thread contention.
*/
- def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = {
+ def randomize[T: ClassTag](seq: TraversableOnce[T]): Seq[T] = {
randomizeInPlace(seq.toArray)
}
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index 4434f3b87c..c443c5266e 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -27,6 +27,21 @@ import org.apache.spark.SparkContext._
class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
+
+ 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]()
+ }
+ }
+
test ("basic accumulation"){
sc = new SparkContext("local", "test")
val acc : Accumulator[Int] = sc.accumulator(0)
@@ -51,7 +66,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte
}
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")
@@ -68,22 +82,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte
}
}
- 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")
@@ -125,7 +124,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte
}
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")
diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
index d9103aebb7..78e35732d2 100644
--- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark
+import scala.reflect.ClassTag
import org.scalatest.FunSuite
import java.io.File
import org.apache.spark.rdd._
@@ -207,7 +208,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
* not, but this is not done by default as usually the partitions do not refer to any RDD and
* therefore never store the lineage.
*/
- def testCheckpointing[U: ClassManifest](
+ def testCheckpointing[U: ClassTag](
op: (RDD[Int]) => RDD[U],
testRDDSize: Boolean = true,
testRDDPartitionSize: Boolean = false
@@ -276,7 +277,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
* RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed,
* this RDD will remember the partitions and therefore potentially the whole lineage.
*/
- def testParentCheckpointing[U: ClassManifest](
+ def testParentCheckpointing[U: ClassTag](
op: (RDD[Int]) => RDD[U],
testRDDSize: Boolean,
testRDDPartitionSize: Boolean
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 7a856d4081..c719a54a61 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -325,7 +325,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
// when running under LocalScheduler:
sc = new SparkContext("local-cluster[1,1,512]", "test")
val akkaFrameSize =
- sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt
+ sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt
val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)}
val exception = intercept[SparkException] {
rdd.reduce((x, y) => x)
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index 01a72d8401..6d1695eae7 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts {
// Regression test for SPARK-530: "Spark driver process doesn't exit after finishing"
val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
forAll(masters) { (master: String) =>
- failAfter(30 seconds) {
+ failAfter(60 seconds) {
Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
new File(System.getenv("SPARK_HOME")))
}
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index 6013320eaa..18fb1bf590 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -109,7 +109,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
val slaveTracker = new MapOutputTracker()
slaveTracker.trackerActor = slaveSystem.actorFor(
- "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker")
+ "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")
masterTracker.registerShuffle(10, 1)
masterTracker.incrementEpoch()
diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
index 07c9f2382b..f4e1d4e802 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
@@ -25,11 +25,8 @@ import org.eclipse.jetty.server.Server
class UISuite extends FunSuite {
test("jetty port increases under contention") {
val startPort = 4040
- val server = new Server(startPort)
- server.start()
val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq())
val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq())
-
// Allow some wiggle room in case ports on the machine are under contention
assert(boundPort1 > startPort && boundPort1 < startPort + 10)
assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10)
diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
index 4e40dcbdee..5aff26f9fc 100644
--- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
@@ -63,54 +63,53 @@ class SizeEstimatorSuite
}
test("simple classes") {
- assert(SizeEstimator.estimate(new DummyClass1) === 16)
- assert(SizeEstimator.estimate(new DummyClass2) === 16)
- assert(SizeEstimator.estimate(new DummyClass3) === 24)
- assert(SizeEstimator.estimate(new DummyClass4(null)) === 24)
- assert(SizeEstimator.estimate(new DummyClass4(new DummyClass3)) === 48)
+ expectResult(16)(SizeEstimator.estimate(new DummyClass1))
+ expectResult(16)(SizeEstimator.estimate(new DummyClass2))
+ expectResult(24)(SizeEstimator.estimate(new DummyClass3))
+ expectResult(24)(SizeEstimator.estimate(new DummyClass4(null)))
+ expectResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
}
// NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors
// (Sun vs IBM). Use a DummyString class to make tests deterministic.
test("strings") {
- assert(SizeEstimator.estimate(DummyString("")) === 40)
- assert(SizeEstimator.estimate(DummyString("a")) === 48)
- assert(SizeEstimator.estimate(DummyString("ab")) === 48)
- assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56)
+ expectResult(40)(SizeEstimator.estimate(DummyString("")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("a")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("ab")))
+ expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
}
test("primitive arrays") {
- assert(SizeEstimator.estimate(new Array[Byte](10)) === 32)
- assert(SizeEstimator.estimate(new Array[Char](10)) === 40)
- assert(SizeEstimator.estimate(new Array[Short](10)) === 40)
- assert(SizeEstimator.estimate(new Array[Int](10)) === 56)
- assert(SizeEstimator.estimate(new Array[Long](10)) === 96)
- assert(SizeEstimator.estimate(new Array[Float](10)) === 56)
- assert(SizeEstimator.estimate(new Array[Double](10)) === 96)
- assert(SizeEstimator.estimate(new Array[Int](1000)) === 4016)
- assert(SizeEstimator.estimate(new Array[Long](1000)) === 8016)
+ expectResult(32)(SizeEstimator.estimate(new Array[Byte](10)))
+ expectResult(40)(SizeEstimator.estimate(new Array[Char](10)))
+ expectResult(40)(SizeEstimator.estimate(new Array[Short](10)))
+ expectResult(56)(SizeEstimator.estimate(new Array[Int](10)))
+ expectResult(96)(SizeEstimator.estimate(new Array[Long](10)))
+ expectResult(56)(SizeEstimator.estimate(new Array[Float](10)))
+ expectResult(96)(SizeEstimator.estimate(new Array[Double](10)))
+ expectResult(4016)(SizeEstimator.estimate(new Array[Int](1000)))
+ expectResult(8016)(SizeEstimator.estimate(new Array[Long](1000)))
}
test("object arrays") {
// Arrays containing nulls should just have one pointer per element
- assert(SizeEstimator.estimate(new Array[String](10)) === 56)
- assert(SizeEstimator.estimate(new Array[AnyRef](10)) === 56)
-
+ expectResult(56)(SizeEstimator.estimate(new Array[String](10)))
+ expectResult(56)(SizeEstimator.estimate(new Array[AnyRef](10)))
// For object arrays with non-null elements, each object should take one pointer plus
// however many bytes that class takes. (Note that Array.fill calls the code in its
// second parameter separately for each object, so we get distinct objects.)
- assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)) === 216)
- assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)) === 216)
- assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)) === 296)
- assert(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)) === 56)
+ expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)))
+ expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)))
+ expectResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)))
+ expectResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)))
// Past size 100, our samples 100 elements, but we should still get the right size.
- assert(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)) === 28016)
+ expectResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)))
// If an array contains the *same* element many times, we should only count it once.
val d1 = new DummyClass1
- assert(SizeEstimator.estimate(Array.fill(10)(d1)) === 72) // 10 pointers plus 8-byte object
- assert(SizeEstimator.estimate(Array.fill(100)(d1)) === 432) // 100 pointers plus 8-byte object
+ expectResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object
+ expectResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object
// Same thing with huge array containing the same element many times. Note that this won't
// return exactly 4032 because it can't tell that *all* the elements will equal the first
@@ -128,11 +127,10 @@ class SizeEstimatorSuite
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
- assert(SizeEstimator.estimate(DummyString("")) === 40)
- assert(SizeEstimator.estimate(DummyString("a")) === 48)
- assert(SizeEstimator.estimate(DummyString("ab")) === 48)
- assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56)
-
+ expectResult(40)(SizeEstimator.estimate(DummyString("")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("a")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("ab")))
+ expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
resetOrClear("os.arch", arch)
}
@@ -145,10 +143,10 @@ class SizeEstimatorSuite
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
- assert(SizeEstimator.estimate(DummyString("")) === 56)
- assert(SizeEstimator.estimate(DummyString("a")) === 64)
- assert(SizeEstimator.estimate(DummyString("ab")) === 64)
- assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 72)
+ expectResult(56)(SizeEstimator.estimate(DummyString("")))
+ expectResult(64)(SizeEstimator.estimate(DummyString("a")))
+ expectResult(64)(SizeEstimator.estimate(DummyString("ab")))
+ expectResult(72)(SizeEstimator.estimate(DummyString("abcdefgh")))
resetOrClear("os.arch", arch)
resetOrClear("spark.test.useCompressedOops", oops)
diff --git a/docs/_config.yml b/docs/_config.yml
index 48ecb8d0c9..02067f9750 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -5,6 +5,6 @@ markdown: kramdown
# of Spark, Scala, and Mesos.
SPARK_VERSION: 0.9.0-incubating-SNAPSHOT
SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT
-SCALA_VERSION: 2.9.3
+SCALA_VERSION: 2.10
MESOS_VERSION: 0.13.0
SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index c574ea7f5c..431de909cb 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -35,7 +35,7 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1')
# Copy over the scaladoc from each project into the docs directory.
# This directory will be copied over to _site when `jekyll` command is run.
projects.each do |project_name|
- source = "../" + project_name + "/target/scala-2.9.3/api"
+ source = "../" + project_name + "/target/scala-2.10/api"
dest = "api/" + project_name
puts "echo making directory " + dest
diff --git a/examples/pom.xml b/examples/pom.xml
index b9cc6f5e0a..3c0a8d06ed 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -77,23 +77,23 @@
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>algebird-core_2.9.2</artifactId>
+ <artifactId>algebird-core_${scala-short.version}</artifactId>
<version>0.1.11</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.cassandra</groupId>
<artifactId>cassandra-all</artifactId>
- <version>1.2.5</version>
+ <version>1.2.6</version>
<exclusions>
<exclusion>
<groupId>com.google.guava</groupId>
@@ -128,8 +128,8 @@
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index cd3423a07b..08e399f9ee 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -19,6 +19,7 @@ package org.apache.spark.streaming.examples
import scala.collection.mutable.LinkedList
import scala.util.Random
+import scala.reflect.ClassTag
import akka.actor.Actor
import akka.actor.ActorRef
@@ -82,7 +83,7 @@ class FeederActor extends Actor {
*
* @see [[org.apache.spark.streaming.examples.FeederActor]]
*/
-class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String)
+class SampleActorReceiver[T: ClassTag](urlOfPublisher: String)
extends Actor with Receiver {
lazy private val remotePublisher = context.actorFor(urlOfPublisher)
@@ -164,7 +165,7 @@ object ActorWordCount {
*/
val lines = ssc.actorStream[String](
- Props(new SampleActorReceiver[String]("akka://test@%s:%s/user/FeederActor".format(
+ Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format(
host, port.toInt))), "SampleReceiver")
//compute wordcount
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index c8743b9e25..e83ce78aa5 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -23,6 +23,7 @@ import akka.zeromq._
import org.apache.spark.streaming.{ Seconds, StreamingContext }
import org.apache.spark.streaming.StreamingContext._
import akka.zeromq.Subscribe
+import akka.util.ByteString
/**
* A simple publisher for demonstration purposes, repeatedly publishes random Messages
@@ -40,10 +41,11 @@ object SimpleZeroMQPublisher {
val acs: ActorSystem = ActorSystem()
val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url))
- val messages: Array[String] = Array("words ", "may ", "count ")
+ implicit def stringToByteString(x: String) = ByteString(x)
+ val messages: List[ByteString] = List("words ", "may ", "count ")
while (true) {
Thread.sleep(1000)
- pubSocket ! ZMQMessage(Frame(topic) :: messages.map(x => Frame(x.getBytes)).toList)
+ pubSocket ! ZMQMessage(ByteString(topic) :: messages)
}
acs.awaitTermination()
}
@@ -78,7 +80,7 @@ object ZeroMQWordCount {
val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2),
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
- def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator
+ def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator
//For this stream, a zeroMQ publisher should be running.
val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator)
diff --git a/mllib/pom.xml b/mllib/pom.xml
index 4ef4f0ae4e..4cff5e3700 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -48,12 +48,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -63,8 +63,8 @@
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index 5aec867257..d5f3f6b8db 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -83,7 +83,7 @@ object MFDataGenerator{
scala.math.round(.99 * m * n)).toInt
val rand = new Random()
val mn = m * n
- val shuffled = rand.shuffle(1 to mn toIterable)
+ val shuffled = rand.shuffle(1 to mn toList)
val omega = shuffled.slice(0, sampSize)
val ordered = omega.sortWith(_ < _).toArray
diff --git a/pom.xml b/pom.xml
index ad5051d38a..844ba74252 100644
--- a/pom.xml
+++ b/pom.xml
@@ -76,9 +76,11 @@
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<java.version>1.5</java.version>
- <scala.version>2.9.3</scala.version>
+
+ <scala-short.version>2.10</scala-short.version>
+ <scala.version>2.10.2</scala.version>
<mesos.version>0.13.0</mesos.version>
- <akka.version>2.0.5</akka.version>
+ <akka.version>2.2.1</akka.version>
<slf4j.version>1.7.2</slf4j.version>
<log4j.version>1.2.17</log4j.version>
<hadoop.version>1.0.4</hadoop.version>
@@ -91,9 +93,9 @@
<repositories>
<repository>
- <id>jboss-repo</id>
- <name>JBoss Repository</name>
- <url>http://repository.jboss.org/nexus/content/repositories/releases/</url>
+ <id>typesafe-repo</id>
+ <name>Typesafe Repository</name>
+ <url>http://repo.typesafe.com/typesafe/releases/</url>
<releases>
<enabled>true</enabled>
</releases>
@@ -102,9 +104,9 @@
</snapshots>
</repository>
<repository>
- <id>cloudera-repo</id>
- <name>Cloudera Repository</name>
- <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+ <id>jboss-repo</id>
+ <name>JBoss Repository</name>
+ <url>http://repository.jboss.org/nexus/content/repositories/releases/</url>
<releases>
<enabled>true</enabled>
</releases>
@@ -214,7 +216,7 @@
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>chill_2.9.3</artifactId>
+ <artifactId>chill_${scala-short.version}</artifactId>
<version>0.3.1</version>
</dependency>
<dependency>
@@ -224,17 +226,12 @@
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
- <artifactId>akka-actor</artifactId>
+ <artifactId>akka-remote_${scala-short.version}</artifactId>
<version>${akka.version}</version>
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
- <artifactId>akka-remote</artifactId>
- <version>${akka.version}</version>
- </dependency>
- <dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-slf4j</artifactId>
+ <artifactId>akka-slf4j_${scala-short.version}</artifactId>
<version>${akka.version}</version>
</dependency>
<dependency>
@@ -249,7 +246,7 @@
</dependency>
<dependency>
<groupId>com.github.scala-incubator.io</groupId>
- <artifactId>scala-io-file_2.9.2</artifactId>
+ <artifactId>scala-io-file_${scala-short.version}</artifactId>
<version>0.4.1</version>
</dependency>
<dependency>
@@ -270,8 +267,8 @@
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
- <artifactId>lift-json_2.9.2</artifactId>
- <version>2.5</version>
+ <artifactId>lift-json_${scala-short.version}</artifactId>
+ <version>2.5.1</version>
</dependency>
<dependency>
<groupId>com.codahale.metrics</groupId>
@@ -309,24 +306,22 @@
<version>${scala.version}</version>
</dependency>
<dependency>
- <groupId>org.scala-lang</groupId>
- <artifactId>scalap</artifactId>
- <version>${scala.version}</version>
- </dependency>
-
- <dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<version>${log4j.version}</version>
</dependency>
-
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<version>1.9.1</version>
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>2.4</version>
+ </dependency>
+ <dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<version>3.1</version>
@@ -334,7 +329,7 @@
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_${scala-short.version}</artifactId>
<version>1.10.0</version>
<scope>test</scope>
</dependency>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index aef246d8a9..67d03f987f 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -78,9 +78,9 @@ object SparkBuild extends Build {
core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef
def sharedSettings = Defaults.defaultSettings ++ Seq(
- organization := "org.apache.spark",
- version := "0.9.0-incubating-SNAPSHOT",
- scalaVersion := "2.9.3",
+ organization := "org.apache.spark",
+ version := "0.9.0-incubating-SNAPSHOT",
+ scalaVersion := "2.10.2",
scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation",
"-target:" + SCALAC_JVM_VERSION),
javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION),
@@ -97,9 +97,6 @@ object SparkBuild extends Build {
// Only allow one test at a time, even across projects, since they run in the same JVM
concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
- // Shared between both core and streaming.
- resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"),
-
// For Sonatype publishing
resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
@@ -154,12 +151,16 @@ object SparkBuild extends Build {
*/
libraryDependencies ++= Seq(
- "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
- "org.scalatest" %% "scalatest" % "1.9.1" % "test",
- "org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
- "com.novocode" % "junit-interface" % "0.9" % "test",
- "org.easymock" % "easymock" % "3.1" % "test"
+ "io.netty" % "netty-all" % "4.0.0.CR1",
+ "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
+ "org.scalatest" %% "scalatest" % "1.9.1" % "test",
+ "org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
+ "com.novocode" % "junit-interface" % "0.9" % "test",
+ "org.easymock" % "easymock" % "3.1" % "test",
+ "commons-io" % "commons-io" % "2.4" % "test"
),
+
+ parallelExecution := false,
/* Workaround for issue #206 (fixed after SBT 0.11.0) */
watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task,
const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) },
@@ -183,60 +184,61 @@ object SparkBuild extends Build {
def coreSettings = sharedSettings ++ Seq(
name := "spark-core",
resolvers ++= Seq(
- "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
- "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
+ "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
+ "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
),
libraryDependencies ++= Seq(
- "com.google.guava" % "guava" % "14.0.1",
- "com.google.code.findbugs" % "jsr305" % "1.3.9",
- "log4j" % "log4j" % "1.2.17",
- "org.slf4j" % "slf4j-api" % slf4jVersion,
- "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
- "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407
- "com.ning" % "compress-lzf" % "0.8.4",
- "org.xerial.snappy" % "snappy-java" % "1.0.5",
- "org.ow2.asm" % "asm" % "4.0",
- "com.google.protobuf" % "protobuf-java" % "2.4.1",
- "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty),
- "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty),
- "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty),
- "it.unimi.dsi" % "fastutil" % "6.4.4",
- "colt" % "colt" % "1.2.0",
- "net.liftweb" % "lift-json_2.9.2" % "2.5",
- "org.apache.mesos" % "mesos" % "0.13.0",
- "io.netty" % "netty-all" % "4.0.0.Beta2",
- "org.apache.derby" % "derby" % "10.4.2.0" % "test",
- "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
- "net.java.dev.jets3t" % "jets3t" % "0.7.1",
- "org.apache.avro" % "avro" % "1.7.4",
- "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty),
- "com.codahale.metrics" % "metrics-core" % "3.0.0",
- "com.codahale.metrics" % "metrics-jvm" % "3.0.0",
- "com.codahale.metrics" % "metrics-json" % "3.0.0",
- "com.codahale.metrics" % "metrics-ganglia" % "3.0.0",
- "com.twitter" % "chill_2.9.3" % "0.3.1",
- "com.twitter" % "chill-java" % "0.3.1"
- )
+ "com.google.guava" % "guava" % "14.0.1",
+ "com.google.code.findbugs" % "jsr305" % "1.3.9",
+ "log4j" % "log4j" % "1.2.17",
+ "org.slf4j" % "slf4j-api" % slf4jVersion,
+ "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
+ "com.ning" % "compress-lzf" % "0.8.4",
+ "org.xerial.snappy" % "snappy-java" % "1.0.5",
+ "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407
+ "org.ow2.asm" % "asm" % "4.0",
+ "com.google.protobuf" % "protobuf-java" % "2.4.1",
+ "de.javakaffee" % "kryo-serializers" % "0.22",
+ "com.typesafe.akka" %% "akka-remote" % "2.2.1" excludeAll(excludeNetty),
+ "com.typesafe.akka" %% "akka-slf4j" % "2.2.1" excludeAll(excludeNetty),
+ "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty),
+ "it.unimi.dsi" % "fastutil" % "6.4.4",
+ "colt" % "colt" % "1.2.0",
+ "org.apache.mesos" % "mesos" % "0.13.0",
+ "net.java.dev.jets3t" % "jets3t" % "0.7.1",
+ "org.apache.derby" % "derby" % "10.4.2.0" % "test",
+ "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
+ "org.apache.avro" % "avro" % "1.7.4",
+ "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty),
+ "com.codahale.metrics" % "metrics-core" % "3.0.0",
+ "com.codahale.metrics" % "metrics-jvm" % "3.0.0",
+ "com.codahale.metrics" % "metrics-json" % "3.0.0",
+ "com.codahale.metrics" % "metrics-ganglia" % "3.0.0",
+ "com.twitter" %% "chill" % "0.3.1",
+ "com.twitter" % "chill-java" % "0.3.1"
+ )
)
def rootSettings = sharedSettings ++ Seq(
publish := {}
)
- def replSettings = sharedSettings ++ Seq(
+ def replSettings = sharedSettings ++ Seq(
name := "spark-repl",
- libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _)
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ),
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ),
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v )
)
+
def examplesSettings = sharedSettings ++ Seq(
name := "spark-examples",
libraryDependencies ++= Seq(
- "com.twitter" % "algebird-core_2.9.2" % "0.1.11",
-
+ "com.twitter" %% "algebird-core" % "0.1.11",
+ "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty, excludeAsm),
"org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm),
-
- "org.apache.cassandra" % "cassandra-all" % "1.2.5"
+ "org.apache.cassandra" % "cassandra-all" % "1.2.6"
exclude("com.google.guava", "guava")
exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru")
exclude("com.ning","compress-lzf")
@@ -265,14 +267,11 @@ object SparkBuild extends Build {
def streamingSettings = sharedSettings ++ Seq(
name := "spark-streaming",
- resolvers ++= Seq(
- "Akka Repository" at "http://repo.akka.io/releases/"
- ),
libraryDependencies ++= Seq(
- "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
- "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty),
- "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
- "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty)
+ "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
+ "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty),
+ "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
+ "com.typesafe.akka" %% "akka-zeromq" % "2.2.1" excludeAll(excludeNetty)
)
)
diff --git a/pyspark b/pyspark
index 4941a36d0d..69d49eb87c 100755
--- a/pyspark
+++ b/pyspark
@@ -23,7 +23,7 @@ FWDIR="$(cd `dirname $0`; pwd)"
# Export this as SPARK_HOME
export SPARK_HOME="$FWDIR"
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Exit if the user hasn't compiled Spark
if [ ! -f "$FWDIR/RELEASE" ]; then
diff --git a/pyspark2.cmd b/pyspark2.cmd
index f58e349643..21f9a34388 100644
--- a/pyspark2.cmd
+++ b/pyspark2.cmd
@@ -17,7 +17,7 @@ rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
set FWDIR=%~dp0
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 58e1849cad..7611b13e82 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -917,14 +917,14 @@ class PipelinedRDD(RDD):
[x._jbroadcast for x in self.ctx._pickled_broadcast_vars],
self.ctx._gateway._gateway_client)
self.ctx._pickled_broadcast_vars.clear()
- class_manifest = self._prev_jrdd.classManifest()
+ class_tag = self._prev_jrdd.classTag()
env = MapConverter().convert(self.ctx.environment,
self.ctx._gateway._gateway_client)
includes = ListConverter().convert(self.ctx._python_includes,
self.ctx._gateway._gateway_client)
python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(),
pipe_command, env, includes, self.preservesPartitioning, self.ctx.pythonExec,
- broadcast_vars, self.ctx._javaAccumulator, class_manifest)
+ broadcast_vars, self.ctx._javaAccumulator, class_tag)
self._jrdd_val = python_rdd.asJavaRDD()
return self._jrdd_val
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
index 8b5d8300f2..47bb654baf 100755
--- a/repl-bin/src/deb/bin/run
+++ b/repl-bin/src/deb/bin/run
@@ -17,7 +17,7 @@
# limitations under the License.
#
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Figure out where the Scala framework is installed
FWDIR="$(cd `dirname $0`; pwd)"
diff --git a/repl/lib/scala-jline.jar b/repl/lib/scala-jline.jar
deleted file mode 100644
index 2f18c95cdd..0000000000
--- a/repl/lib/scala-jline.jar
+++ /dev/null
Binary files differ
diff --git a/repl/pom.xml b/repl/pom.xml
index 2826c0743c..d4b1ea10be 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -61,10 +61,12 @@
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId>
+ <version>${scala.version}</version>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>jline</artifactId>
+ <version>${scala.version}</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
@@ -76,18 +78,18 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/repl/src/main/scala/org/apache/spark/repl/Main.scala b/repl/src/main/scala/org/apache/spark/repl/Main.scala
index 17e149f8ab..14b448d076 100644
--- a/repl/src/main/scala/org/apache/spark/repl/Main.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/Main.scala
@@ -20,12 +20,12 @@ package org.apache.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/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
new file mode 100644
index 0000000000..85b0978c81
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
@@ -0,0 +1,110 @@
+/* NSC -- new Scala compiler
+ * Copyright 2005-2013 LAMP/EPFL
+ * @author Paul Phillips
+ */
+
+package org.apache.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 org.apache.spark.Logging
+import scala.Some
+
+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/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 36f54a22cf..988b624feb 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/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 org.apache.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 java.lang.{Class => jClass}
+import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse}
import org.apache.spark.Logging
import org.apache.spark.SparkContext
@@ -37,45 +49,86 @@ import org.apache.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
+ def isAsync = false
+ // 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 +140,49 @@ 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
}
+
+ def sparkCleanUp(){
+ echo("Stopping spark context.")
+ intp.beQuietDuring {
+ command("sc.stop()")
+ }
+ }
/** Close the interpreter and set the var to null. */
def closeInterpreter() {
if (intp ne null) {
- intp.close
+ sparkCleanUp()
+ 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 +196,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 +210,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 +221,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.9.0-SNAPSHOT
- /_/
-""")
- 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 +242,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", "<path>", "add a jar or directory to the classpath", addClasspath),
cmd("help", "[command]", "print this summary or command-specific help", helpCommand),
@@ -263,53 +282,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", "<path|class>", "disassemble a file or class name", javapCommand),
- nullary("keybindings", "show how ctrl-[A-Z] and other keys are bound", keybindingsCommand),
cmd("load", "<path>", "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", "<expr>", "display the type of an expression without evaluating it", typeCommand)
+ cmd("type", "[-v] <expr>", "display the type of an expression without evaluating it", typeCommand),
+ nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand)
)
-
+
/** Power user commands */
lazy val powerCommands: List[LoopCommand] = List(
- //nullary("dump", "displays a view of the interpreter's internal state", dumpCommand),
- //cmd("phase", "<phase>", "set the implicit phase for power commands", phaseCommand),
- cmd("wrap", "<method>", "name of method to wrap around each repl line", wrapCommand) withLongHelp ("""
- |:wrap
- |:wrap clear
- |:wrap <method>
- |
- |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", "<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 +313,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 +329,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 +338,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 +355,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 +383,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 +395,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] <expression>"
+ 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: <unknown>"
- }
- 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: <unknown>"
+ }
+ 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 <expr> (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 <expr> (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 +583,89 @@ 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 = "<command line>"
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 +685,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 +729,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 +741,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 +768,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 +778,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 +792,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 +810,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 +829,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 +837,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,107 +848,104 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
}
- def initializeSpark() {
- intp.beQuietDuring {
- command("""
- org.apache.spark.repl.Main.interp.out.println("Creating SparkContext...");
- org.apache.spark.repl.Main.interp.out.flush();
- @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext();
- org.apache.spark.repl.Main.interp.out.println("Spark context available as sc.");
- org.apache.spark.repl.Main.interp.out.flush();
- """)
- command("import org.apache.spark.SparkContext._")
- }
- echo("Type in expressions to have them evaluated.")
- echo("Type :help for more information.")
- }
-
- var sparkContext: SparkContext = null
-
- def createSparkContext(): SparkContext = {
- val uri = System.getenv("SPARK_EXECUTOR_URI")
- if (uri != null) {
- System.setProperty("spark.executor.uri", uri)
- }
- val master = this.master match {
- case Some(m) => m
- case None => {
- val prop = System.getenv("MASTER")
- if (prop != null) prop else "local"
- }
- }
- val jars = Option(System.getenv("ADD_JARS")).map(_.split(','))
- .getOrElse(new Array[String](0))
- .map(new java.io.File(_).getAbsolutePath)
- sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
- 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...")
-
- // Add JARS specified in Spark's ADD_JARS variable to classpath
- val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0))
- jars.foreach(settings.classpath.append(_))
+ 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[org.apache.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)
+ })
+
+ addThunk(printWelcome())
+ addThunk(initializeSpark())
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()
+ createAsyncListener() // listens for signal to run postInitialization
+ }
+ else {
+ intp.initializeSynchronous()
+ postInitialization()
}
+ // printWelcome()
+
+ try loop()
+ catch AbstractOrMissingHandler()
finally closeInterpreter()
+
true
}
+ def createSparkContext(): SparkContext = {
+ val uri = System.getenv("SPARK_EXECUTOR_URI")
+ if (uri != null) {
+ System.setProperty("spark.executor.uri", uri)
+ }
+ val master = this.master match {
+ case Some(m) => m
+ case None => {
+ val prop = System.getenv("MASTER")
+ if (prop != null) prop else "local"
+ }
+ }
+ val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath)
+ sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
+ 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)
}
@@ -922,15 +954,17 @@ object SparkILoop {
implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp
private def echo(msg: String) = Console println msg
+ def getAddedJars: Array[String] = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0))
+
// Designed primarily for use by test code: take a String with a
// bunch of code, and prints out a transcript of what it would look
// like if you'd just typed it into the repl.
def runForTranscript(code: String, settings: Settings): String = {
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 == '|')) ()
@@ -949,26 +983,29 @@ object SparkILoop {
}
}
val repl = new SparkILoop(input, output)
+
if (settings.classpath.isDefault)
settings.classpath.value = sys.props("java.class.path")
+ getAddedJars.foreach(settings.classpath.append(_))
+
repl process settings
}
}
}
-
+
/** 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")
@@ -977,32 +1014,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/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
new file mode 100644
index 0000000000..21b1ba305d
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
@@ -0,0 +1,143 @@
+/* NSC -- new Scala compiler
+ * Copyright 2005-2013 LAMP/EPFL
+ * @author Paul Phillips
+ */
+
+package org.apache.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.9.0-SNAPSHOT
+ /_/
+""")
+ import Properties._
+ val welcomeMsg = "Using Scala %s (%s, Java %s)".format(
+ versionString, javaVmName, javaVersion)
+ echo(welcomeMsg)
+ echo("Type in expressions to have them evaluated.")
+ echo("Type :help for more information.")
+ }
+
+ 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 val sc = org.apache.spark.repl.Main.interp.createSparkContext();
+ """)
+ command("import org.apache.spark.SparkContext._")
+ }
+ echo("Spark context available as sc.")
+ }
+
+ // 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)
+ logDebug("Clearing " + pendingThunks.size + " thunks.")
+
+ while (pendingThunks.nonEmpty) {
+ val thunk = pendingThunks.head
+ pendingThunks = pendingThunks.tail
+ thunk()
+ }
+ }
+}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index e6e35c9b5d..e1455ef8a1 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/src/main/scala/org/apache/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,333 @@ 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 org.apache.spark.HttpServer
import org.apache.spark.util.Utils
import org.apache.spark.SparkEnv
+import org.apache.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)
- }
+ val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
- /** Scala compiler virtual directory for outputDir */
- val virtualDirectory = new PlainFile(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)
+ }
- /** Jetty server that will serve our classes to worker nodes */
- val classServer = new HttpServer(outputDir)
+ 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)
+ }
- // 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("<init>", 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("<init>", "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 = "<global>"
+ }
+ }
+
+ /** 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 +343,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: <refinement> 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: <refinement> 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("<console>", 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("<script>", code))
+
+ /** Build a request from the user. `trees` is `line` after being parsed.
+ */
+ private def buildRequest(line: String, trees: List[Tree]): Request = {
+ executingRequest = new Request(line, trees)
+ executingRequest
+ }
+
+ // rewriting "5 // foo" to "val x = { 5 // foo }" creates broken code because
+ // the close brace is commented out. Strip single-line comments.
+ // ... but for error message output reasons this is not used, and rather than
+ // enclosing in braces it is constructed like "val x =\n5 // foo".
+ private def removeComments(line: String): String = {
+ showCodeIfDebugging(line) // as we're about to lose our // show
+ line.lines map (s => s indexOf "//" match {
+ case -1 => s
+ case idx => s take idx
+ }) mkString "\n"
}
- /** Compile an nsc SourceFile. Returns true if there are
- * no compilation errors, or false otherwise.
- */
- def compileSources(sources: SourceFile*): Boolean = {
- reporter.reset()
- new Run() compileSources sources.toList
- !reporter.hasErrors
+ private def safePos(t: Tree, alt: Int): Int =
+ try t.pos.startOrPoint
+ catch { case _: UnsupportedOperationException => alt }
+
+ // Given an expression like 10 * 10 * 10 we receive the parent tree positioned
+ // at a '*'. So look at each subtree and find the earliest of all positions.
+ private def earliestPosition(tree: Tree): Int = {
+ var pos = Int.MaxValue
+ tree foreach { t =>
+ pos = math.min(pos, safePos(t, Int.MaxValue))
+ }
+ pos
}
- /** Compile a string. Returns true if there are no
- * compilation errors, or false otherwise.
- */
- def compileString(code: String): Boolean =
- compileSources(new BatchSourceFile("<script>", code))
- /** Build a request from the user. `trees` is `line` after being parsed.
- */
- private def buildRequest(line: String, trees: List[Tree]): Request = new Request(line, trees)
-
private def requestFromLine(line: String, synthetic: Boolean): Either[IR.Result, Request] = {
- val trees = parse(indentCode(line)) match {
+ val content = indentCode(line)
+ val trees = parse(content) match {
case None => return Left(IR.Incomplete)
case Some(Nil) => return Left(IR.Error) // parse error or empty input
case Some(trees) => trees
}
-
- // use synthetic vars to avoid filling up the resXX slots
- def varName = if (synthetic) freshInternalVarName() else freshUserVarName()
-
- // Treat a single bare expression specially. This is necessary due to it being hard to
- // modify code at a textual level, and it being hard to submit an AST to the compiler.
- if (trees.size == 1) trees.head match {
- case _:Assign => // we don't want to include assignments
- case _:TermTree | _:Ident | _:Select => // ... but do want these as valdefs.
- requestFromLine("val %s =\n%s".format(varName, line), synthetic) match {
+ logDebug(
+ trees map (t => {
+ // [Eugene to Paul] previously it just said `t map ...`
+ // because there was an implicit conversion from Tree to a list of Trees
+ // however Martin and I have removed the conversion
+ // (it was conflicting with the new reflection API),
+ // so I had to rewrite this a bit
+ val subs = t collect { case sub => sub }
+ subs map (t0 =>
+ " " + safePos(t0, -1) + ": " + t0.shortClass + "\n"
+ ) mkString ""
+ }) mkString "\n"
+ )
+ // If the last tree is a bare expression, pinpoint where it begins using the
+ // AST node position and snap the line off there. Rewrite the code embodied
+ // by the last tree as a ValDef instead, so we can access the value.
+ trees.last match {
+ case _:Assign => // we don't want to include assignments
+ case _:TermTree | _:Ident | _:Select => // ... but do want other unnamed terms.
+ val varName = if (synthetic) freshInternalVarName() else freshUserVarName()
+ val rewrittenLine = (
+ // In theory this would come out the same without the 1-specific test, but
+ // it's a cushion against any more sneaky parse-tree position vs. code mismatches:
+ // this way such issues will only arise on multiple-statement repl input lines,
+ // which most people don't use.
+ if (trees.size == 1) "val " + varName + " =\n" + content
+ else {
+ // The position of the last tree
+ val lastpos0 = earliestPosition(trees.last)
+ // Oh boy, the parser throws away parens so "(2+2)" is mispositioned,
+ // with increasingly hard to decipher positions as we move on to "() => 5",
+ // (x: Int) => x + 1, and more. So I abandon attempts to finesse and just
+ // look for semicolons and newlines, which I'm sure is also buggy.
+ val (raw1, raw2) = content splitAt lastpos0
+ logDebug("[raw] " + raw1 + " <---> " + raw2)
+
+ val adjustment = (raw1.reverse takeWhile (ch => (ch != ';') && (ch != '\n'))).size
+ val lastpos = lastpos0 - adjustment
+
+ // the source code split at the laboriously determined position.
+ val (l1, l2) = content splitAt lastpos
+ logDebug("[adj] " + l1 + " <---> " + l2)
+
+ val prefix = if (l1.trim == "") "" else l1 + ";\n"
+ // Note to self: val source needs to have this precise structure so that
+ // error messages print the user-submitted part without the "val res0 = " part.
+ val combined = prefix + "val " + varName + " =\n" + l2
+
+ logDebug(List(
+ " line" -> line,
+ " content" -> content,
+ " was" -> l2,
+ "combined" -> combined) map {
+ case (label, s) => label + ": '" + s + "'"
+ } mkString "\n"
+ )
+ combined
+ }
+ )
+ // Rewriting "foo ; bar ; 123"
+ // to "foo ; bar ; val resXX = 123"
+ requestFromLine(rewrittenLine, synthetic) match {
case Right(req) => return Right(req withOriginalLine line)
case x => return x
}
- case _ =>
+ case _ =>
}
-
- // figure out what kind of request
Right(buildRequest(line, trees))
}
- /**
- * Interpret one line of input. All feedback, including parse errors
- * and evaluation results, are printed via the supplied compiler's
- * reporter. Values defined are available for future interpreted
- * strings.
- *
- *
- * The return value is whether the line was interpreter successfully,
- * e.g. that there were no parse errors.
- *
+ // normalize non-public types so we don't see protected aliases like Self
+ def normalizeNonPublic(tp: Type) = tp match {
+ case TypeRef(_, sym, _) if sym.isAliasType && !sym.isPublic => tp.dealias
+ case _ => tp
+ }
+
+ /**
+ * Interpret one line of input. All feedback, including parse errors
+ * and evaluation results, are printed via the supplied compiler's
+ * reporter. Values defined are available for future interpreted strings.
*
- * @param line ...
- * @return ...
+ * The return value is whether the line was interpreter successfully,
+ * e.g. that there were no parse errors.
*/
def interpret(line: String): IR.Result = interpret(line, false)
+ def interpretSynthetic(line: String): IR.Result = interpret(line, true)
def interpret(line: String, synthetic: Boolean): IR.Result = {
def loadAndRunReq(req: Request) = {
+ classLoader.setAsContext()
val (result, succeeded) = req.loadAndRun
+
/** To our displeasure, ConsoleReporter offers only printMessage,
* which tacks a newline on the end. Since that breaks all the
* output checking, we have to take one off to balance.
*/
- def show() = {
- if (result == "") ()
- else printMessage(result stripSuffix "\n")
- }
-
if (succeeded) {
- if (printResults)
- show()
+ if (printResults && result != "")
+ printMessage(result stripSuffix "\n")
+ else if (isReplDebug) // show quiet-mode activity
+ printMessage(result.trim.lines map ("[quiet] " + _) mkString "\n")
+
// Book-keeping. Have to record synthetic requests too,
// as they may have been issued for information, e.g. :type
recordRequest(req)
IR.Success
}
- else {
- // don't truncate stack traces
- withoutTruncating(show())
- IR.Error
- }
+ else {
+ // don't truncate stack traces
+ withoutTruncating(printMessage(result))
+ IR.Error
+ }
}
-
+
if (global == null) IR.Error
else requestFromLine(line, synthetic) match {
case Left(result) => result
- case Right(req) =>
+ case Right(req) =>
// null indicates a disallowed statement type; otherwise compile and
// fail if false (implying e.g. a type error)
if (req == null || !req.compile) IR.Error
@@ -546,23 +652,39 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
* @param value the object value to bind to it
* @return an indication of whether the binding succeeded
*/
- def bind(name: String, boundType: String, value: Any): IR.Result = {
+ def bind(name: String, boundType: String, value: Any, modifiers: List[String] = Nil): IR.Result = {
val bindRep = new ReadEvalPrint()
val run = bindRep.compile("""
- |object %s {
- | var value: %s = _
- | def set(x: Any) = value = x.asInstanceOf[%s]
- |}
- """.stripMargin.format(bindRep.evalName, boundType, boundType)
- )
- bindRep.callOpt("set", value) match {
- case Some(_) => interpret("val %s = %s.value".format(name, bindRep.evalPath))
- case _ => DBG("Set failed in bind(%s, %s, %s)".format(name, boundType, value)) ; IR.Error
+ |object %s {
+ | var value: %s = _
+ | def set(x: Any) = value = x.asInstanceOf[%s]
+ |}
+ """.stripMargin.format(bindRep.evalName, boundType, boundType)
+ )
+ bindRep.callEither("set", value) match {
+ case Left(ex) =>
+ logDebug("Set failed in bind(%s, %s, %s)".format(name, boundType, value))
+ logDebug(util.stackTraceString(ex))
+ IR.Error
+
+ case Right(_) =>
+ val line = "%sval %s = %s.value".format(modifiers map (_ + " ") mkString, name, bindRep.evalPath)
+ logDebug("Interpreting: " + line)
+ interpret(line)
}
}
+ def directBind(name: String, boundType: String, value: Any): IR.Result = {
+ val result = bind(name, boundType, value)
+ if (result == IR.Success)
+ directlyBoundNames += newTermName(name)
+ result
+ }
+ def directBind(p: NamedParam): IR.Result = directBind(p.name, p.tpe, p.value)
+ def directBind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = directBind((name, value))
+
def rebind(p: NamedParam): IR.Result = {
val name = p.name
- val oldType = typeOfTerm(name) getOrElse { return IR.Error }
+ val oldType = typeOfTerm(name) orElse { return IR.Error }
val newType = p.tpe
val tempName = freshInternalVarName()
@@ -570,23 +692,27 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
quietRun("val %s = %s.asInstanceOf[%s]".format(name, tempName, newType))
}
def quietImport(ids: String*): IR.Result = beQuietDuring(addImports(ids: _*))
- def addImports(ids: String*): IR.Result =
+ def addImports(ids: String*): IR.Result =
if (ids.isEmpty) IR.Success
else interpret("import " + ids.mkString(", "))
- def quietBind(p: NamedParam): IR.Result = beQuietDuring(bind(p))
- def bind(p: NamedParam): IR.Result = bind(p.name, p.tpe, p.value)
- def bind[T: Manifest](name: String, value: T): IR.Result = bind((name, value))
- def bindValue(x: Any): IR.Result = bind(freshUserVarName(), TypeStrings.fromValue(x), x)
+ def quietBind(p: NamedParam): IR.Result = beQuietDuring(bind(p))
+ def bind(p: NamedParam): IR.Result = bind(p.name, p.tpe, p.value)
+ def bind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = bind((name, value))
+ def bindSyntheticValue(x: Any): IR.Result = bindValue(freshInternalVarName(), x)
+ def bindValue(x: Any): IR.Result = bindValue(freshUserVarName(), x)
+ def bindValue(name: String, x: Any): IR.Result = bind(name, TypeStrings.fromValue(x), x)
/** Reset this interpreter, forgetting all user-specified requests. */
def reset() {
- //virtualDirectory.clear()
- virtualDirectory.delete()
- virtualDirectory.create()
+ clearExecutionWrapper()
resetClassLoader()
resetAllCreators()
prevRequests.clear()
+ referencedNameMap.clear()
+ definedNameMap.clear()
+ virtualDirectory.delete()
+ virtualDirectory.create()
}
/** This instance is no longer needed, so release any resources
@@ -596,9 +722,9 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
reporter.flush()
classServer.stop()
}
-
+
/** Here is where we:
- *
+ *
* 1) Read some source code, and put it in the "read" object.
* 2) Evaluate the read object, and put the result in the "eval" object.
* 3) Create a String for human consumption, and put it in the "print" object.
@@ -608,115 +734,172 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
class ReadEvalPrint(lineId: Int) {
def this() = this(freshLineId())
- val packageName = "$line" + lineId
- val readName = "$read"
- val evalName = "$eval"
- val printName = "$print"
- val valueMethod = "$result" // no-args method giving result
-
+ private var lastRun: Run = _
+ private var evalCaught: Option[Throwable] = None
+ private var conditionalWarnings: List[ConditionalWarning] = Nil
+
+ val packageName = sessionNames.line + lineId
+ val readName = sessionNames.read
+ val evalName = sessionNames.eval
+ val printName = sessionNames.print
+ val resultName = sessionNames.result
+
+ def bindError(t: Throwable) = {
+ if (!bindExceptions) // avoid looping if already binding
+ throw t
+
+ val unwrapped = unwrap(t)
+ withLastExceptionLock[String]({
+ directBind[Throwable]("lastException", unwrapped)(tagOfThrowable, classTag[Throwable])
+ util.stackTraceString(unwrapped)
+ }, util.stackTraceString(unwrapped))
+ }
+
// TODO: split it out into a package object and a regular
// object and we can do that much less wrapping.
def packageDecl = "package " + packageName
-
+
def pathTo(name: String) = packageName + "." + name
def packaged(code: String) = packageDecl + "\n\n" + code
def readPath = pathTo(readName)
def evalPath = pathTo(evalName)
def printPath = pathTo(printName)
-
- def call(name: String, args: Any*): AnyRef =
- evalMethod(name).invoke(evalClass, args.map(_.asInstanceOf[AnyRef]): _*)
-
+
+ def call(name: String, args: Any*): AnyRef = {
+ val m = evalMethod(name)
+ logDebug("Invoking: " + m)
+ if (args.nonEmpty)
+ logDebug(" with args: " + args.mkString(", "))
+
+ m.invoke(evalClass, args.map(_.asInstanceOf[AnyRef]): _*)
+ }
+
+ def callEither(name: String, args: Any*): Either[Throwable, AnyRef] =
+ try Right(call(name, args: _*))
+ catch { case ex: Throwable => Left(ex) }
+
def callOpt(name: String, args: Any*): Option[AnyRef] =
try Some(call(name, args: _*))
- catch { case ex: Exception =>
- quietBind("lastException", ex)
- None
- }
-
- lazy val evalClass = loadByName(evalPath)
- lazy val evalValue = callOpt(valueMethod)
+ catch { case ex: Throwable => bindError(ex) ; None }
- def compile(source: String): Boolean = compileAndSaveRun("<console>", source)
- def lineAfterTyper[T](op: => T): T = {
- assert(lastRun != null, "Internal error: trying to use atPhase, but Run is null." + this)
- atPhase(lastRun.typerPhase.next)(op)
+ class EvalException(msg: String, cause: Throwable) extends RuntimeException(msg, cause) { }
+
+ private def evalError(path: String, ex: Throwable) =
+ throw new EvalException("Failed to load '" + path + "': " + ex.getMessage, ex)
+
+ private def load(path: String): Class[_] = {
+ try Class.forName(path, true, classLoader)
+ catch { case ex: Throwable => evalError(path, unwrap(ex)) }
+ }
+
+ lazy val evalClass = load(evalPath)
+ lazy val evalValue = callEither(resultName) match {
+ case Left(ex) => evalCaught = Some(ex) ; None
+ case Right(result) => Some(result)
}
-
+
+ def compile(source: String): Boolean = compileAndSaveRun("<console>", source)
+
/** The innermost object inside the wrapper, found by
- * following accessPath into the outer one.
- */
+ * following accessPath into the outer one.
+ */
def resolvePathToSymbol(accessPath: String): Symbol = {
- //val readRoot = definitions.getModule(readPath) // the outermost wrapper
- // MATEI: changed this to getClass because the root object is no longer a module (Scala singleton object)
- val readRoot = definitions.getClass(readPath) // the outermost wrapper
- (accessPath split '.').foldLeft(readRoot) { (sym, name) =>
- if (name == "") sym else
- lineAfterTyper(sym.info member newTermName(name))
+ // val readRoot = getRequiredModule(readPath) // the outermost wrapper
+ // MATEI: Changed this to getClass because the root object is no longer a module (Scala singleton object)
+
+ val readRoot = rootMirror.getClassByName(newTypeName(readPath)) // the outermost wrapper
+ (accessPath split '.').foldLeft(readRoot: Symbol) {
+ case (sym, "") => sym
+ case (sym, name) => afterTyper(termMember(sym, name))
}
}
-
- // def compileAndTypeExpr(expr: String): Option[Typer] = {
- // class TyperRun extends Run {
- // override def stopPhase(name: String) = name == "superaccessors"
- // }
- // }
- private var lastRun: Run = _
- private def evalMethod(name: String) = {
- val methods = evalClass.getMethods filter (_.getName == name)
- assert(methods.size == 1, "Internal error - eval object method " + name + " is overloaded: " + methods)
- methods.head
+ /** We get a bunch of repeated warnings for reasons I haven't
+ * entirely figured out yet. For now, squash.
+ */
+ private def updateRecentWarnings(run: Run) {
+ def loop(xs: List[(Position, String)]): List[(Position, String)] = xs match {
+ case Nil => Nil
+ case ((pos, msg)) :: rest =>
+ val filtered = rest filter { case (pos0, msg0) =>
+ (msg != msg0) || (pos.lineContent.trim != pos0.lineContent.trim) || {
+ // same messages and same line content after whitespace removal
+ // but we want to let through multiple warnings on the same line
+ // from the same run. The untrimmed line will be the same since
+ // there's no whitespace indenting blowing it.
+ (pos.lineContent == pos0.lineContent)
+ }
+ }
+ ((pos, msg)) :: loop(filtered)
+ }
+ //PRASHANT: This leads to a NoSuchMethodError for _.warnings. Yet to figure out its purpose.
+ // val warnings = loop(run.allConditionalWarnings flatMap (_.warnings))
+ // if (warnings.nonEmpty)
+ // mostRecentWarnings = warnings
+ }
+ private def evalMethod(name: String) = evalClass.getMethods filter (_.getName == name) match {
+ case Array(method) => method
+ case xs => sys.error("Internal error: eval object " + evalClass + ", " + xs.mkString("\n", "\n", ""))
}
private def compileAndSaveRun(label: String, code: String) = {
showCodeIfDebugging(code)
- reporter.reset()
- lastRun = new Run()
- lastRun.compileSources(List(new BatchSourceFile(label, packaged(code))))
- !reporter.hasErrors
+ val (success, run) = compileSourcesKeepingRun(new BatchSourceFile(label, packaged(code)))
+ updateRecentWarnings(run)
+ lastRun = run
+ success
}
}
/** One line of code submitted by the user for interpretation */
- // private
+ // private
class Request(val line: String, val trees: List[Tree]) {
- val lineRep = new ReadEvalPrint()
- import lineRep.lineAfterTyper
-
+ val reqId = nextReqId()
+ val lineRep = new ReadEvalPrint()
+
private var _originalLine: String = null
def withOriginalLine(s: String): this.type = { _originalLine = s ; this }
def originalLine = if (_originalLine == null) line else _originalLine
/** handlers for each tree in this request */
val handlers: List[MemberHandler] = trees map (memberHandlers chooseHandler _)
+ def defHandlers = handlers collect { case x: MemberDefHandler => x }
/** all (public) names defined by these statements */
val definedNames = handlers flatMap (_.definedNames)
/** list of names used by this expression */
val referencedNames: List[Name] = handlers flatMap (_.referencedNames)
-
+
/** def and val names */
def termNames = handlers flatMap (_.definesTerm)
def typeNames = handlers flatMap (_.definesType)
+ def definedOrImported = handlers flatMap (_.definedOrImported)
+ def definedSymbolList = defHandlers flatMap (_.definedSymbols)
+
+ def definedTypeSymbol(name: String) = definedSymbols(newTypeName(name))
+ def definedTermSymbol(name: String) = definedSymbols(newTermName(name))
/** Code to import bound names from previous lines - accessPath is code to
- * append to objectName to access anything bound by request.
- */
- val ComputedImports(importsPreamble, importsTrailer, accessPath) =
+ * append to objectName to access anything bound by request.
+ */
+ val SparkComputedImports(importsPreamble, importsTrailer, accessPath) =
importsCode(referencedNames.toSet)
/** Code to access a variable with the specified name */
- def fullPath(vname: String) = (
- //lineRep.readPath + accessPath + ".`%s`".format(vname)
+ def fullPath(vname: String) = {
+ // lineRep.readPath + accessPath + ".`%s`".format(vname)
lineRep.readPath + ".INSTANCE" + accessPath + ".`%s`".format(vname)
- )
- /** Same as fullpath, but after it has been flattened, so:
- * $line5.$iw.$iw.$iw.Bippy // fullPath
- * $line5.$iw$$iw$$iw$Bippy // fullFlatName
- */
- def fullFlatName(name: String) =
- lineRep.readPath + accessPath.replace('.', '$') + "$" + name
+ }
+ /** Same as fullpath, but after it has been flattened, so:
+ * $line5.$iw.$iw.$iw.Bippy // fullPath
+ * $line5.$iw$$iw$$iw$Bippy // fullFlatName
+ */
+ def fullFlatName(name: String) =
+ // lineRep.readPath + accessPath.replace('.', '$') + nme.NAME_JOIN_STRING + name
+ lineRep.readPath + ".INSTANCE" + accessPath.replace('.', '$') + nme.NAME_JOIN_STRING + name
+
+ /** The unmangled symbol name, but supplemented with line info. */
+ def disambiguated(name: Name): String = name + " (in " + lineRep + ")"
/** Code to access a variable with the specified name */
def fullPath(vname: Name): String = fullPath(vname.toString)
@@ -726,52 +909,66 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
/** generate the source code for the object that computes this request */
private object ObjectSourceCode extends CodeAssembler[MemberHandler] {
+ def path = pathToTerm("$intp")
+ def envLines = {
+ if (!isReplPower) Nil // power mode only for now
+ // $intp is not bound; punt, but include the line.
+ else if (path == "$intp") List(
+ "def $line = " + tquoted(originalLine),
+ "def $trees = Nil"
+ )
+ else List(
+ "def $line = " + tquoted(originalLine),
+ "def $req = %s.requestForReqId(%s).orNull".format(path, reqId),
+ "def $trees = if ($req eq null) Nil else $req.trees".format(lineRep.readName, path, reqId)
+ )
+ }
+
val preamble = """
|class %s extends Serializable {
- | %s%s
- """.stripMargin.format(lineRep.readName, importsPreamble, indentCode(toCompute))
+ | %s%s%s
+ """.stripMargin.format(lineRep.readName, envLines.map(" " + _ + ";\n").mkString, importsPreamble, indentCode(toCompute))
val postamble = importsTrailer + "\n}" + "\n" +
"object " + lineRep.readName + " {\n" +
" val INSTANCE = new " + lineRep.readName + "();\n" +
"}\n"
val generate = (m: MemberHandler) => m extraCodeToEvaluate Request.this
+
/*
val preamble = """
- |object %s {
- | %s%s
- """.stripMargin.format(lineRep.readName, importsPreamble, indentCode(toCompute))
+ |object %s extends Serializable {
+ |%s%s%s
+ """.stripMargin.format(lineRep.readName, envLines.map(" " + _ + ";\n").mkString, importsPreamble, indentCode(toCompute))
val postamble = importsTrailer + "\n}"
val generate = (m: MemberHandler) => m extraCodeToEvaluate Request.this
*/
+
}
-
+
private object ResultObjectSourceCode extends CodeAssembler[MemberHandler] {
/** We only want to generate this code when the result
* is a value which can be referred to as-is.
- */
+ */
val evalResult =
if (!handlers.last.definesValue) ""
else handlers.last.definesTerm match {
case Some(vname) if typeOf contains vname =>
- """
- |lazy val $result = {
- | $export
- | %s
- |}""".stripMargin.format(fullPath(vname))
+ "lazy val %s = %s".format(lineRep.resultName, fullPath(vname))
case _ => ""
}
+
// first line evaluates object to make sure constructor is run
// initial "" so later code can uniformly be: + etc
val preamble = """
|object %s {
| %s
- | val $export: String = %s {
+ | val %s: String = %s {
| %s
| (""
""".stripMargin.format(
- lineRep.evalName, evalResult, executionWrapper, lineRep.readName + ".INSTANCE" + accessPath
+ lineRep.evalName, evalResult, lineRep.printName,
+ executionWrapper, lineRep.readName + ".INSTANCE" + accessPath
)
-
val postamble = """
| )
| }
@@ -785,7 +982,7 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
def getEval: Option[AnyRef] = {
// ensure it has been compiled
compile
- // try to load it and call the value method
+ // try to load it and call the value method
lineRep.evalValue filterNot (_ == null)
}
@@ -797,114 +994,54 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
// compile the object containing the user's code
lineRep.compile(ObjectSourceCode(handlers)) && {
- // extract and remember types
+ // extract and remember types
typeOf
typesOfDefinedTerms
+ // Assign symbols to the original trees
+ // TODO - just use the new trees.
+ defHandlers foreach { dh =>
+ val name = dh.member.name
+ definedSymbols get name foreach { sym =>
+ dh.member setSymbol sym
+ logDebug("Set symbol of " + name + " to " + sym.defString)
+ }
+ }
+
// compile the result-extraction object
- lineRep compile ResultObjectSourceCode(handlers)
+ withoutWarnings(lineRep compile ResultObjectSourceCode(handlers))
}
}
lazy val resultSymbol = lineRep.resolvePathToSymbol(accessPath)
- def applyToResultMember[T](name: Name, f: Symbol => T) = lineAfterTyper(f(resultSymbol.info.nonPrivateDecl(name)))
+ def applyToResultMember[T](name: Name, f: Symbol => T) = afterTyper(f(resultSymbol.info.nonPrivateDecl(name)))
/* typeOf lookup with encoding */
- def lookupTypeOf(name: Name) = {
- typeOf.getOrElse(name, typeOf(global.encode(name.toString)))
- }
- def simpleNameOfType(name: TypeName) = {
- (compilerTypeOf get name) map (_.typeSymbol.simpleName)
- }
-
- private def typeMap[T](f: Type => T): Map[Name, T] = {
- def toType(name: Name): T = {
- // the types are all =>T; remove the =>
- val tp1 = lineAfterTyper(resultSymbol.info.nonPrivateDecl(name).tpe match {
- case NullaryMethodType(tp) => tp
- case tp => tp
- })
- // normalize non-public types so we don't see protected aliases like Self
- lineAfterTyper(tp1 match {
- case TypeRef(_, sym, _) if !sym.isPublic => f(tp1.normalize)
- case tp => f(tp)
- })
- }
- termNames ++ typeNames map (x => x -> toType(x)) toMap
- }
+ def lookupTypeOf(name: Name) = typeOf.getOrElse(name, typeOf(global.encode(name.toString)))
+ def simpleNameOfType(name: TypeName) = (compilerTypeOf get name) map (_.typeSymbol.simpleName)
+
+ private def typeMap[T](f: Type => T) =
+ mapFrom[Name, Name, T](termNames ++ typeNames)(x => f(cleanMemberDecl(resultSymbol, x)))
+
/** Types of variables defined by this request. */
- lazy val compilerTypeOf = typeMap[Type](x => x)
+ lazy val compilerTypeOf = typeMap[Type](x => x) withDefaultValue NoType
/** String representations of same. */
- lazy val typeOf = typeMap[String](_.toString)
-
+ lazy val typeOf = typeMap[String](tp => afterTyper(tp.toString))
+
// lazy val definedTypes: Map[Name, Type] = {
// typeNames map (x => x -> afterTyper(resultSymbol.info.nonPrivateDecl(x).tpe)) toMap
// }
- lazy val definedSymbols: Map[Name, Symbol] = (
+ lazy val definedSymbols = (
termNames.map(x => x -> applyToResultMember(x, x => x)) ++
- typeNames.map(x => x -> compilerTypeOf.get(x).map(_.typeSymbol).getOrElse(NoSymbol))
- ).toMap
-
- lazy val typesOfDefinedTerms: Map[Name, Type] =
- termNames map (x => x -> applyToResultMember(x, _.tpe)) toMap
-
- private def bindExceptionally(t: Throwable) = {
- val ex: Exceptional =
- if (isettings.showInternalStackTraces) Exceptional(t)
- else new Exceptional(t) {
- override def spanFn(frame: JavaStackFrame) = !(frame.className startsWith lineRep.evalPath)
- override def contextPrelude = super.contextPrelude + "/* The repl internal portion of the stack trace is elided. */\n"
- }
-
- quietBind("lastException", ex)
- ex.contextHead + "\n(access lastException for the full trace)"
- }
- private def bindUnexceptionally(t: Throwable) = {
- quietBind("lastException", t)
- stackTraceString(t)
- }
+ typeNames.map(x => x -> compilerTypeOf(x).typeSymbolDirect)
+ ).toMap[Name, Symbol] withDefaultValue NoSymbol
+
+ lazy val typesOfDefinedTerms = mapFrom[Name, Name, Type](termNames)(x => applyToResultMember(x, _.tpe))
/** load and run the code using reflection */
def loadAndRun: (String, Boolean) = {
- import interpreter.Line._
-
- def handleException(t: Throwable) = {
- /** We turn off the binding to accomodate ticket #2817 */
- withoutBindingLastException {
- val message =
- if (opt.richExes) bindExceptionally(unwrap(t))
- else bindUnexceptionally(unwrap(t))
-
- (message, false)
- }
- }
-
- try {
- val execution = lineManager.set(originalLine) {
- // MATEI: set the right SparkEnv for our SparkContext, because
- // this execution will happen in a separate thread
- val sc = org.apache.spark.repl.Main.interp.sparkContext
- if (sc != null && sc.env != null)
- SparkEnv.set(sc.env)
- // Execute the line
- lineRep call "$export"
- }
- execution.await()
-
- execution.state match {
- case Done => ("" + execution.get(), true)
- case Threw =>
- val ex = execution.caught()
- if (isReplDebug)
- ex.printStackTrace()
-
- if (bindLastException) handleException(ex)
- else throw ex
- case Cancelled => ("Execution interrupted by signal.\n", false)
- case Running => ("Execution still running! Seems impossible.", false)
- }
- }
- finally lineManager.clear()
+ try { ("" + (lineRep call sessionNames.print), true) }
+ catch { case ex: Throwable => (lineRep.bindError(ex), false) }
}
override def toString = "Request(line=%s, %s trees)".format(line, trees.size)
@@ -922,136 +1059,157 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
case ModuleDef(_, name, _) => name
case _ => naming.mostRecentVar
})
-
- private def requestForName(name: Name): Option[Request] = {
+
+ private var mostRecentWarnings: List[(global.Position, String)] = Nil
+ def lastWarnings = mostRecentWarnings
+
+ def treesForRequestId(id: Int): List[Tree] =
+ requestForReqId(id).toList flatMap (_.trees)
+
+ def requestForReqId(id: Int): Option[Request] =
+ if (executingRequest != null && executingRequest.reqId == id) Some(executingRequest)
+ else prevRequests find (_.reqId == id)
+
+ def requestForName(name: Name): Option[Request] = {
assert(definedNameMap != null, "definedNameMap is null")
definedNameMap get name
}
- private def requestForIdent(line: String): Option[Request] =
+ def requestForIdent(line: String): Option[Request] =
requestForName(newTermName(line)) orElse requestForName(newTypeName(line))
-
- def safeClass(name: String): Option[Symbol] = {
- try Some(definitions.getClass(newTypeName(name)))
- catch { case _: MissingRequirementError => None }
- }
- def safeModule(name: String): Option[Symbol] = {
- try Some(definitions.getModule(newTermName(name)))
- catch { case _: MissingRequirementError => None }
- }
+
+ def requestHistoryForName(name: Name): List[Request] =
+ prevRequests.toList.reverse filter (_.definedNames contains name)
+
def definitionForName(name: Name): Option[MemberHandler] =
requestForName(name) flatMap { req =>
req.handlers find (_.definedNames contains name)
}
-
+
def valueOfTerm(id: String): Option[AnyRef] =
- requestForIdent(id) flatMap (_.getEval)
+ requestForName(newTermName(id)) flatMap (_.getEval)
def classOfTerm(id: String): Option[JClass] =
- valueOfTerm(id) map (_.getClass)
+ valueOfTerm(id) map (_.getClass)
- def typeOfTerm(id: String): Option[Type] = newTermName(id) match {
- case nme.ROOTPKG => Some(definitions.RootClass.tpe)
- case name => requestForName(name) flatMap (_.compilerTypeOf get name)
+ def typeOfTerm(id: String): Type = newTermName(id) match {
+ case nme.ROOTPKG => RootClass.tpe
+ case name => requestForName(name).fold(NoType: Type)(_ compilerTypeOf name)
}
+
+ def symbolOfType(id: String): Symbol =
+ requestForName(newTypeName(id)).fold(NoSymbol: Symbol)(_ definedTypeSymbol id)
+
def symbolOfTerm(id: String): Symbol =
- requestForIdent(id) flatMap (_.definedSymbols get newTermName(id)) getOrElse NoSymbol
+ requestForIdent(newTermName(id)).fold(NoSymbol: Symbol)(_ definedTermSymbol id)
def runtimeClassAndTypeOfTerm(id: String): Option[(JClass, Type)] = {
- for {
- clazz <- classOfTerm(id)
- tpe <- runtimeTypeOfTerm(id)
- nonAnon <- new RichClass(clazz).supers.find(c => !(new RichClass(c).isScalaAnonymous))
- } yield {
- (nonAnon, tpe)
- }
- }
-
- def runtimeTypeOfTerm(id: String): Option[Type] = {
- for {
- tpe <- typeOfTerm(id)
- clazz <- classOfTerm(id)
- val staticSym = tpe.typeSymbol
- runtimeSym <- safeClass(clazz.getName)
- if runtimeSym != staticSym
- if runtimeSym isSubClass staticSym
- } yield {
- runtimeSym.info
+ classOfTerm(id) flatMap { clazz =>
+ new RichClass(clazz).supers find(c => !(new RichClass(c).isScalaAnonymous)) map { nonAnon =>
+ (nonAnon, runtimeTypeOfTerm(id))
+ }
}
}
-
- // XXX literals.
- // 1) Identifiers defined in the repl.
- // 2) A path loadable via getModule.
- // 3) Try interpreting it as an expression.
- private var typeOfExpressionDepth = 0
- def typeOfExpression(expr: String): Option[Type] = {
- DBG("typeOfExpression(" + expr + ")")
- if (typeOfExpressionDepth > 2) {
- DBG("Terminating typeOfExpression recursion for expression: " + expr)
- return None
- }
- def asQualifiedImport = {
- val name = expr.takeWhile(_ != '.')
- importedTermNamed(name) flatMap { sym =>
- typeOfExpression(sym.fullName + expr.drop(name.length))
- }
+ def runtimeTypeOfTerm(id: String): Type = {
+ typeOfTerm(id) andAlso { tpe =>
+ val clazz = classOfTerm(id) getOrElse { return NoType }
+ val staticSym = tpe.typeSymbol
+ val runtimeSym = getClassIfDefined(clazz.getName)
+
+ if ((runtimeSym != NoSymbol) && (runtimeSym != staticSym) && (runtimeSym isSubClass staticSym))
+ runtimeSym.info
+ else NoType
}
- def asModule = safeModule(expr) map (_.tpe)
- def asExpr = beSilentDuring {
- val lhs = freshInternalVarName()
- val line = "lazy val " + lhs + " = { " + expr + " } "
-
- interpret(line, true) match {
- case IR.Success => typeOfExpression(lhs)
- case _ => None
+ }
+ def cleanMemberDecl(owner: Symbol, member: Name): Type = afterTyper {
+ normalizeNonPublic {
+ owner.info.nonPrivateDecl(member).tpe match {
+ case NullaryMethodType(tp) => tp
+ case tp => tp
}
}
-
- typeOfExpressionDepth += 1
- try typeOfTerm(expr) orElse asModule orElse asExpr orElse asQualifiedImport
- finally typeOfExpressionDepth -= 1
}
- // def compileAndTypeExpr(expr: String): Option[Typer] = {
- // class TyperRun extends Run {
- // override def stopPhase(name: String) = name == "superaccessors"
- // }
- // }
-
+
+ object exprTyper extends {
+ val repl: SparkIMain.this.type = imain
+ } with SparkExprTyper { }
+
+ def parse(line: String): Option[List[Tree]] = exprTyper.parse(line)
+
+ def symbolOfLine(code: String): Symbol =
+ exprTyper.symbolOfLine(code)
+
+ def typeOfExpression(expr: String, silent: Boolean = true): Type =
+ exprTyper.typeOfExpression(expr, silent)
+
protected def onlyTerms(xs: List[Name]) = xs collect { case x: TermName => x }
protected def onlyTypes(xs: List[Name]) = xs collect { case x: TypeName => x }
-
- def definedTerms = onlyTerms(allDefinedNames) filterNot (x => isInternalVarName(x.toString))
- def definedTypes = onlyTypes(allDefinedNames)
- def definedSymbols = prevRequests.toSet flatMap ((x: Request) => x.definedSymbols.values)
-
+
+ def definedTerms = onlyTerms(allDefinedNames) filterNot isInternalTermName
+ def definedTypes = onlyTypes(allDefinedNames)
+ def definedSymbols = prevRequestList.flatMap(_.definedSymbols.values).toSet[Symbol]
+ def definedSymbolList = prevRequestList flatMap (_.definedSymbolList) filterNot (s => isInternalTermName(s.name))
+
+ // Terms with user-given names (i.e. not res0 and not synthetic)
+ def namedDefinedTerms = definedTerms filterNot (x => isUserVarName("" + x) || directlyBoundNames(x))
+
+ private def findName(name: Name) = definedSymbols find (_.name == name) getOrElse NoSymbol
+
+ /** Translate a repl-defined identifier into a Symbol.
+ */
+ def apply(name: String): Symbol =
+ types(name) orElse terms(name)
+
+ def types(name: String): Symbol = {
+ val tpname = newTypeName(name)
+ findName(tpname) orElse getClassIfDefined(tpname)
+ }
+ def terms(name: String): Symbol = {
+ val termname = newTypeName(name)
+ findName(termname) orElse getModuleIfDefined(termname)
+ }
+ // [Eugene to Paul] possibly you could make use of TypeTags here
+ def types[T: ClassTag] : Symbol = types(classTag[T].runtimeClass.getName)
+ def terms[T: ClassTag] : Symbol = terms(classTag[T].runtimeClass.getName)
+ def apply[T: ClassTag] : Symbol = apply(classTag[T].runtimeClass.getName)
+
+ def classSymbols = allDefSymbols collect { case x: ClassSymbol => x }
+ def methodSymbols = allDefSymbols collect { case x: MethodSymbol => x }
+
/** the previous requests this interpreter has processed */
- private lazy val prevRequests = mutable.ListBuffer[Request]()
- private lazy val referencedNameMap = mutable.Map[Name, Request]()
- private lazy val definedNameMap = mutable.Map[Name, Request]()
- protected def prevRequestList = prevRequests.toList
- private def allHandlers = prevRequestList flatMap (_.handlers)
- def allSeenTypes = prevRequestList flatMap (_.typeOf.values.toList) distinct
- def allImplicits = allHandlers filter (_.definesImplicit) flatMap (_.definedNames)
- def importHandlers = allHandlers collect { case x: ImportHandler => x }
-
+ private var executingRequest: Request = _
+ private val prevRequests = mutable.ListBuffer[Request]()
+ private val referencedNameMap = mutable.Map[Name, Request]()
+ private val definedNameMap = mutable.Map[Name, Request]()
+ private val directlyBoundNames = mutable.Set[Name]()
+
+ def allHandlers = prevRequestList flatMap (_.handlers)
+ def allDefHandlers = allHandlers collect { case x: MemberDefHandler => x }
+ def allDefSymbols = allDefHandlers map (_.symbol) filter (_ ne NoSymbol)
+
+ def lastRequest = if (prevRequests.isEmpty) null else prevRequests.last
+ def prevRequestList = prevRequests.toList
+ def allSeenTypes = prevRequestList flatMap (_.typeOf.values.toList) distinct
+ def allImplicits = allHandlers filter (_.definesImplicit) flatMap (_.definedNames)
+ def importHandlers = allHandlers collect { case x: ImportHandler => x }
+
def visibleTermNames: List[Name] = definedTerms ++ importedTerms distinct
/** Another entry point for tab-completion, ids in scope */
def unqualifiedIds = visibleTermNames map (_.toString) filterNot (_ contains "$") sorted
-
+
/** Parse the ScalaSig to find type aliases */
def aliasForType(path: String) = ByteCode.aliasForType(path)
-
+
def withoutUnwrapping(op: => Unit): Unit = {
val saved = isettings.unwrapStrings
isettings.unwrapStrings = false
try op
finally isettings.unwrapStrings = saved
}
-
+
def symbolDefString(sym: Symbol) = {
TypeStrings.quieter(
afterTyper(sym.defString),
@@ -1059,38 +1217,41 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
sym.owner.fullName + "."
)
}
-
+
def showCodeIfDebugging(code: String) {
/** Secret bookcase entrance for repl debuggers: end the line
* with "// show" and see what's going on.
*/
- if (SPARK_DEBUG_REPL || code.lines.exists(_.trim endsWith "// show")) {
- echo(code)
- parse(code) foreach (ts => ts foreach (t => withoutUnwrapping(DBG(asCompactString(t)))))
+ def isShow = code.lines exists (_.trim endsWith "// show")
+ def isShowRaw = code.lines exists (_.trim endsWith "// raw")
+
+ // old style
+ beSilentDuring(parse(code)) foreach { ts =>
+ ts foreach { t =>
+ withoutUnwrapping(logDebug(asCompactString(t)))
+ }
}
}
+
// debugging
def debugging[T](msg: String)(res: T) = {
- DBG(msg + " " + res)
+ logDebug(msg + " " + res)
res
}
- def DBG(s: => String) = if (isReplDebug) {
- //try repldbg(s)
- //catch { case x: AssertionError => repldbg("Assertion error printing debug string:\n " + x) }
- }
}
/** Utility methods for the Interpreter. */
object SparkIMain {
// The two name forms this is catching are the two sides of this assignment:
//
- // $line3.$read.$iw.$iw.Bippy =
+ // $line3.$read.$iw.$iw.Bippy =
// $line3.$read$$iw$$iw$Bippy@4a6a00ca
private def removeLineWrapper(s: String) = s.replaceAll("""\$line\d+[./]\$(read|eval|print)[$.]""", "")
private def removeIWPackages(s: String) = s.replaceAll("""\$(iw|iwC|read|eval|print)[$.]""", "")
private def removeSparkVals(s: String) = s.replaceAll("""\$VAL[0-9]+[$.]""", "")
+
def stripString(s: String) = removeSparkVals(removeIWPackages(removeLineWrapper(s)))
-
+
trait CodeAssembler[T] {
def preamble: String
def generate: T => String
@@ -1102,7 +1263,7 @@ object SparkIMain {
code println postamble
}
}
-
+
trait StrippingWriter {
def isStripping: Boolean
def stripImpl(str: String): String
@@ -1112,17 +1273,17 @@ object SparkIMain {
def maxStringLength: Int
def isTruncating: Boolean
def truncate(str: String): String = {
- if (isTruncating && str.length > maxStringLength)
+ if (isTruncating && (maxStringLength != 0 && str.length > maxStringLength))
(str take maxStringLength - 3) + "..."
else str
}
}
- abstract class StrippingTruncatingWriter(out: PrintWriter)
- extends PrintWriter(out)
+ abstract class StrippingTruncatingWriter(out: JPrintWriter)
+ extends JPrintWriter(out)
with StrippingWriter
with TruncatingWriter {
self =>
-
+
def clean(str: String): String = truncate(strip(str))
override def write(str: String) = super.write(clean(str))
}
@@ -1132,18 +1293,7 @@ object SparkIMain {
def isStripping = isettings.unwrapStrings
def isTruncating = reporter.truncationOK
- def stripImpl(str: String): String = {
- val cleaned = stripString(str)
- var ctrlChars = 0
- cleaned map { ch =>
- if (ch.isControl && !ch.isWhitespace) {
- ctrlChars += 1
- if (ctrlChars > 5) return "[line elided for control chars: possibly a scala signature]"
- else '?'
- }
- else ch
- }
- }
+ def stripImpl(str: String): String = naming.unmangle(str)
}
class ReplReporter(intp: SparkIMain) extends ConsoleReporter(intp.settings, null, new ReplStrippingWriter(intp)) {
@@ -1156,5 +1306,55 @@ object SparkIMain {
}
else Console.println(msg)
}
- }
+ }
+}
+
+class SparkISettings(intp: SparkIMain) extends Logging {
+ /** A list of paths where :load should look */
+ var loadPath = List(".")
+
+ /** Set this to true to see repl machinery under -Yrich-exceptions.
+ */
+ var showInternalStackTraces = false
+
+ /** The maximum length of toString to use when printing the result
+ * of an evaluation. 0 means no maximum. If a printout requires
+ * more than this number of characters, then the printout is
+ * truncated.
+ */
+ var maxPrintString = 800
+
+ /** The maximum number of completion candidates to print for tab
+ * completion without requiring confirmation.
+ */
+ var maxAutoprintCompletion = 250
+
+ /** String unwrapping can be disabled if it is causing issues.
+ * Settings this to false means you will see Strings like "$iw.$iw.".
+ */
+ var unwrapStrings = true
+
+ def deprecation_=(x: Boolean) = {
+ val old = intp.settings.deprecation.value
+ intp.settings.deprecation.value = x
+ if (!old && x) logDebug("Enabled -deprecation output.")
+ else if (old && !x) logDebug("Disabled -deprecation output.")
+ }
+
+ def deprecation: Boolean = intp.settings.deprecation.value
+
+ def allSettings = Map(
+ "maxPrintString" -> maxPrintString,
+ "maxAutoprintCompletion" -> maxAutoprintCompletion,
+ "unwrapStrings" -> unwrapStrings,
+ "deprecation" -> deprecation
+ )
+
+ private def allSettingsString =
+ allSettings.toList sortBy (_._1) map { case (k, v) => " " + k + " = " + v + "\n" } mkString
+
+ override def toString = """
+ | SparkISettings {
+ | %s
+ | }""".stripMargin.format(allSettingsString)
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
deleted file mode 100644
index 605b7b259b..0000000000
--- a/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
- * @author Alexander Spoon
- */
-
-package org.apache.spark.repl
-
-import scala.tools.nsc._
-import scala.tools.nsc.interpreter._
-
-/** Settings for the interpreter
- *
- * @version 1.0
- * @author Lex Spoon, 2007/3/24
- **/
-class SparkISettings(intp: SparkIMain) {
- /** A list of paths where :load should look */
- var loadPath = List(".")
-
- /** Set this to true to see repl machinery under -Yrich-exceptions.
- */
- var showInternalStackTraces = false
-
- /** The maximum length of toString to use when printing the result
- * of an evaluation. 0 means no maximum. If a printout requires
- * more than this number of characters, then the printout is
- * truncated.
- */
- var maxPrintString = 800
-
- /** The maximum number of completion candidates to print for tab
- * completion without requiring confirmation.
- */
- var maxAutoprintCompletion = 250
-
- /** String unwrapping can be disabled if it is causing issues.
- * Settings this to false means you will see Strings like "$iw.$iw.".
- */
- var unwrapStrings = true
-
- def deprecation_=(x: Boolean) = {
- val old = intp.settings.deprecation.value
- intp.settings.deprecation.value = x
- if (!old && x) println("Enabled -deprecation output.")
- else if (old && !x) println("Disabled -deprecation output.")
- }
- def deprecation: Boolean = intp.settings.deprecation.value
-
- def allSettings = Map(
- "maxPrintString" -> maxPrintString,
- "maxAutoprintCompletion" -> maxAutoprintCompletion,
- "unwrapStrings" -> unwrapStrings,
- "deprecation" -> deprecation
- )
-
- private def allSettingsString =
- allSettings.toList sortBy (_._1) map { case (k, v) => " " + k + " = " + v + "\n" } mkString
-
- override def toString = """
- | SparkISettings {
- | %s
- | }""".stripMargin.format(allSettingsString)
-}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
index 41a1731d60..64084209e8 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Paul Phillips
*/
@@ -12,30 +12,34 @@ import scala.collection.{ mutable, immutable }
trait SparkImports {
self: SparkIMain =>
-
+
import global._
import definitions.{ ScalaPackage, JavaLangPackage, PredefModule }
import memberHandlers._
-
+
+ def isNoImports = settings.noimports.value
+ def isNoPredef = settings.nopredef.value
+
/** Synthetic import handlers for the language defined imports. */
private def makeWildcardImportHandler(sym: Symbol): ImportHandler = {
val hd :: tl = sym.fullName.split('.').toList map newTermName
val tree = Import(
tl.foldLeft(Ident(hd): Tree)((x, y) => Select(x, y)),
- List(ImportSelector(nme.WILDCARD, -1, null, -1))
+ ImportSelector.wildList
)
tree setSymbol sym
new ImportHandler(tree)
}
-
+
/** Symbols whose contents are language-defined to be imported. */
def languageWildcardSyms: List[Symbol] = List(JavaLangPackage, ScalaPackage, PredefModule)
def languageWildcards: List[Type] = languageWildcardSyms map (_.tpe)
def languageWildcardHandlers = languageWildcardSyms map makeWildcardImportHandler
-
- def importedTerms = onlyTerms(importHandlers flatMap (_.importedNames))
- def importedTypes = onlyTypes(importHandlers flatMap (_.importedNames))
-
+
+ def allImportedNames = importHandlers flatMap (_.importedNames)
+ def importedTerms = onlyTerms(allImportedNames)
+ def importedTypes = onlyTypes(allImportedNames)
+
/** Types which have been wildcard imported, such as:
* val x = "abc" ; import x._ // type java.lang.String
* import java.lang.String._ // object java.lang.String
@@ -48,30 +52,28 @@ trait SparkImports {
* into the compiler scopes.
*/
def sessionWildcards: List[Type] = {
- importHandlers flatMap {
- case x if x.importsWildcard => x.targetType
- case _ => None
- } distinct
+ importHandlers filter (_.importsWildcard) map (_.targetType) distinct
}
def wildcardTypes = languageWildcards ++ sessionWildcards
-
+
def languageSymbols = languageWildcardSyms flatMap membersAtPickler
def sessionImportedSymbols = importHandlers flatMap (_.importedSymbols)
def importedSymbols = languageSymbols ++ sessionImportedSymbols
def importedTermSymbols = importedSymbols collect { case x: TermSymbol => x }
def importedTypeSymbols = importedSymbols collect { case x: TypeSymbol => x }
def implicitSymbols = importedSymbols filter (_.isImplicit)
-
- def importedTermNamed(name: String) = importedTermSymbols find (_.name.toString == name)
+
+ def importedTermNamed(name: String): Symbol =
+ importedTermSymbols find (_.name.toString == name) getOrElse NoSymbol
/** Tuples of (source, imported symbols) in the order they were imported.
*/
def importedSymbolsBySource: List[(Symbol, List[Symbol])] = {
val lang = languageWildcardSyms map (sym => (sym, membersAtPickler(sym)))
- val session = importHandlers filter (_.targetType.isDefined) map { mh =>
- (mh.targetType.get.typeSymbol, mh.importedSymbols)
+ val session = importHandlers filter (_.targetType != NoType) map { mh =>
+ (mh.targetType.typeSymbol, mh.importedSymbols)
}
-
+
lang ++ session
}
def implicitSymbolsBySource: List[(Symbol, List[Symbol])] = {
@@ -79,7 +81,7 @@ trait SparkImports {
case (k, vs) => (k, vs filter (_.isImplicit))
} filterNot (_._2.isEmpty)
}
-
+
/** Compute imports that allow definitions from previous
* requests to be visible in a new request. Returns
* three pieces of related code:
@@ -90,7 +92,7 @@ trait SparkImports {
* 2. A code fragment that should go after the code
* of the new request.
*
- * 3. An access path which can be traverested to access
+ * 3. An access path which can be traversed to access
* any bindings inside code wrapped by #1 and #2 .
*
* The argument is a set of Names that need to be imported.
@@ -103,27 +105,27 @@ trait SparkImports {
* (3) It imports multiple same-named implicits, but only the
* last one imported is actually usable.
*/
- case class ComputedImports(prepend: String, append: String, access: String)
- protected def importsCode(wanted: Set[Name]): ComputedImports = {
- /** Narrow down the list of requests from which imports
+ case class SparkComputedImports(prepend: String, append: String, access: String)
+
+ protected def importsCode(wanted: Set[Name]): SparkComputedImports = {
+ /** Narrow down the list of requests from which imports
* should be taken. Removes requests which cannot contribute
* useful imports for the specified set of wanted names.
*/
case class ReqAndHandler(req: Request, handler: MemberHandler) { }
-
- def reqsToUse: List[ReqAndHandler] = {
+
+ def reqsToUse: List[ReqAndHandler] = {
/** Loop through a list of MemberHandlers and select which ones to keep.
* 'wanted' is the set of names that need to be imported.
*/
def select(reqs: List[ReqAndHandler], wanted: Set[Name]): List[ReqAndHandler] = {
- val isWanted = wanted contains _
// Single symbol imports might be implicits! See bug #1752. Rather than
// try to finesse this, we will mimic all imports for now.
def keepHandler(handler: MemberHandler) = handler match {
case _: ImportHandler => true
- case x => x.definesImplicit || (x.definedNames exists isWanted)
+ case x => x.definesImplicit || (x.definedNames exists wanted)
}
-
+
reqs match {
case Nil => Nil
case rh :: rest if !keepHandler(rh.handler) => select(rest, wanted)
@@ -133,7 +135,7 @@ trait SparkImports {
rh :: select(rest, newWanted)
}
}
-
+
/** Flatten the handlers out and pair each with the original request */
select(allReqAndHandlers reverseMap { case (r, h) => ReqAndHandler(r, h) }, wanted).reverse
}
@@ -147,8 +149,13 @@ trait SparkImports {
code append "class %sC extends Serializable {\n".format(impname)
trailingBraces append "}\nval " + impname + " = new " + impname + "C;\n"
accessPath append ("." + impname)
-
+
currentImps.clear
+ // code append "object %s {\n".format(impname)
+ // trailingBraces append "}\n"
+ // accessPath append ("." + impname)
+
+ // currentImps.clear
}
addWrapper()
@@ -159,36 +166,33 @@ trait SparkImports {
// If the user entered an import, then just use it; add an import wrapping
// level if the import might conflict with some other import
case x: ImportHandler =>
- if (x.importsWildcard || (currentImps exists (x.importedNames contains _)))
+ if (x.importsWildcard || currentImps.exists(x.importedNames contains _))
addWrapper()
-
+
code append (x.member + "\n")
-
+
// give wildcard imports a import wrapper all to their own
if (x.importsWildcard) addWrapper()
else currentImps ++= x.importedNames
// For other requests, import each defined name.
// import them explicitly instead of with _, so that
- // ambiguity errors will not be generated. Also, quote
- // the name of the variable, so that we don't need to
- // handle quoting keywords separately.
+ // ambiguity errors will not be generated. Also, quote
+ // the name of the variable, so that we don't need to
+ // handle quoting keywords separately.
case x =>
for (imv <- x.definedNames) {
- // MATEI: Changed this check because it was messing up for case classes
- // (trying to import them twice within the same wrapper), but that is more likely
- // due to a miscomputation of names that makes the code think they're unique.
- // Need to evaluate whether having so many wrappers is a bad thing.
- /*if (currentImps contains imv)*/
- val imvName = imv.toString
- if (currentImps exists (_.toString == imvName)) addWrapper()
-
+ if (currentImps contains imv) addWrapper()
val objName = req.lineRep.readPath
val valName = "$VAL" + newValId();
- code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
- code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
-
- //code append ("import %s\n" format (req fullPath imv))
+
+ if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported
+ code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
+ code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
+ }
+ // code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
+ // code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
+ // code append ("import " + (req fullPath imv) + "\n")
currentImps += imv
}
}
@@ -196,14 +200,14 @@ trait SparkImports {
// add one extra wrapper, to prevent warnings in the common case of
// redefining the value bound in the last interpreter request.
addWrapper()
- ComputedImports(code.toString, trailingBraces.toString, accessPath.toString)
+ SparkComputedImports(code.toString, trailingBraces.toString, accessPath.toString)
}
-
+
private def allReqAndHandlers =
prevRequestList flatMap (req => req.handlers map (req -> _))
private def membersAtPickler(sym: Symbol): List[Symbol] =
- atPickler(sym.info.nonPrivateMembers)
+ beforePickler(sym.info.nonPrivateMembers.toList)
private var curValId = 0
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
index fdc172d753..8865f82bc0 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Paul Phillips
*/
@@ -11,30 +11,31 @@ import scala.tools.nsc.interpreter._
import scala.tools.jline._
import scala.tools.jline.console.completer._
import Completion._
-import collection.mutable.ListBuffer
+import scala.collection.mutable.ListBuffer
+import org.apache.spark.Logging
// REPL completor - queries supplied interpreter for valid
// completions based on current contents of buffer.
-class SparkJLineCompletion(val intp: SparkIMain) extends Completion with CompletionOutput {
+class SparkJLineCompletion(val intp: SparkIMain) extends Completion with CompletionOutput with Logging {
val global: intp.global.type = intp.global
import global._
- import definitions.{ PredefModule, RootClass, AnyClass, AnyRefClass, ScalaPackage, JavaLangPackage }
+ import definitions.{ PredefModule, AnyClass, AnyRefClass, ScalaPackage, JavaLangPackage }
+ import rootMirror.{ RootClass, getModuleIfDefined }
type ExecResult = Any
- import intp.{ DBG, debugging, afterTyper }
-
+ import intp.{ debugging }
+
// verbosity goes up with consecutive tabs
private var verbosity: Int = 0
def resetVerbosity() = verbosity = 0
-
- def getType(name: String, isModule: Boolean) = {
- val f = if (isModule) definitions.getModule(_: Name) else definitions.getClass(_: Name)
- try Some(f(name).tpe)
- catch { case _: MissingRequirementError => None }
- }
-
- def typeOf(name: String) = getType(name, false)
- def moduleOf(name: String) = getType(name, true)
-
+
+ def getSymbol(name: String, isModule: Boolean) = (
+ if (isModule) getModuleIfDefined(name)
+ else getModuleIfDefined(name)
+ )
+ def getType(name: String, isModule: Boolean) = getSymbol(name, isModule).tpe
+ def typeOf(name: String) = getType(name, false)
+ def moduleOf(name: String) = getType(name, true)
+
trait CompilerCompletion {
def tp: Type
def effectiveTp = tp match {
@@ -48,16 +49,16 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
private def anyMembers = AnyClass.tpe.nonPrivateMembers
def anyRefMethodsToShow = Set("isInstanceOf", "asInstanceOf", "toString")
- def tos(sym: Symbol) = sym.name.decode.toString
- def memberNamed(s: String) = members find (x => tos(x) == s)
- def hasMethod(s: String) = methods exists (x => tos(x) == s)
+ def tos(sym: Symbol): String = sym.decodedName
+ def memberNamed(s: String) = afterTyper(effectiveTp member newTermName(s))
+ def hasMethod(s: String) = memberNamed(s).isMethod
// XXX we'd like to say "filterNot (_.isDeprecated)" but this causes the
// compiler to crash for reasons not yet known.
- def members = afterTyper((effectiveTp.nonPrivateMembers ++ anyMembers) filter (_.isPublic))
- def methods = members filter (_.isMethod)
- def packages = members filter (_.isPackage)
- def aliases = members filter (_.isAliasType)
+ def members = afterTyper((effectiveTp.nonPrivateMembers.toList ++ anyMembers) filter (_.isPublic))
+ def methods = members.toList filter (_.isMethod)
+ def packages = members.toList filter (_.isPackage)
+ def aliases = members.toList filter (_.isAliasType)
def memberNames = members map tos
def methodNames = methods map tos
@@ -65,12 +66,19 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
def aliasNames = aliases map tos
}
+ object NoTypeCompletion extends TypeMemberCompletion(NoType) {
+ override def memberNamed(s: String) = NoSymbol
+ override def members = Nil
+ override def follow(s: String) = None
+ override def alternativesFor(id: String) = Nil
+ }
+
object TypeMemberCompletion {
def apply(tp: Type, runtimeType: Type, param: NamedParam): TypeMemberCompletion = {
new TypeMemberCompletion(tp) {
var upgraded = false
lazy val upgrade = {
- intp rebind param
+ intp rebind param
intp.reporter.printMessage("\nRebinding stable value %s from %s to %s".format(param.name, tp, param.tpe))
upgraded = true
new TypeMemberCompletion(runtimeType)
@@ -92,7 +100,8 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
}
}
def apply(tp: Type): TypeMemberCompletion = {
- if (tp.typeSymbol.isPackageClass) new PackageCompletion(tp)
+ if (tp eq NoType) NoTypeCompletion
+ else if (tp.typeSymbol.isPackageClass) new PackageCompletion(tp)
else new TypeMemberCompletion(tp)
}
def imported(tp: Type) = new ImportCompletion(tp)
@@ -103,9 +112,9 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
def excludeEndsWith: List[String] = Nil
def excludeStartsWith: List[String] = List("<") // <byname>, <repeated>, etc.
def excludeNames: List[String] = (anyref.methodNames filterNot anyRefMethodsToShow) :+ "_root_"
-
+
def methodSignatureString(sym: Symbol) = {
- SparkIMain stripString afterTyper(new MethodSymbolOutput(sym).methodString())
+ IMain stripString afterTyper(new MethodSymbolOutput(sym).methodString())
}
def exclude(name: String): Boolean = (
@@ -118,10 +127,10 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
def completions(verbosity: Int) =
debugging(tp + " completions ==> ")(filtered(memberNames))
-
+
override def follow(s: String): Option[CompletionAware] =
- debugging(tp + " -> '" + s + "' ==> ")(memberNamed(s) map (x => TypeMemberCompletion(x.tpe)))
-
+ debugging(tp + " -> '" + s + "' ==> ")(Some(TypeMemberCompletion(memberNamed(s).tpe)) filterNot (_ eq NoTypeCompletion))
+
override def alternativesFor(id: String): List[String] =
debugging(id + " alternatives ==> ") {
val alts = members filter (x => x.isMethod && tos(x) == id) map methodSignatureString
@@ -131,7 +140,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
override def toString = "%s (%d members)".format(tp, members.size)
}
-
+
class PackageCompletion(tp: Type) extends TypeMemberCompletion(tp) {
override def excludeNames = anyref.methodNames
}
@@ -142,43 +151,44 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
case _ => memberNames
}
}
-
+
class ImportCompletion(tp: Type) extends TypeMemberCompletion(tp) {
override def completions(verbosity: Int) = verbosity match {
case 0 => filtered(members filterNot (_.isSetter) map tos)
case _ => super.completions(verbosity)
}
}
-
+
// not for completion but for excluding
object anyref extends TypeMemberCompletion(AnyRefClass.tpe) { }
-
+
// the unqualified vals/defs/etc visible in the repl
object ids extends CompletionAware {
override def completions(verbosity: Int) = intp.unqualifiedIds ++ List("classOf") //, "_root_")
// now we use the compiler for everything.
- override def follow(id: String) = {
- if (completions(0) contains id) {
- intp typeOfExpression id map { tpe =>
- def default = TypeMemberCompletion(tpe)
-
- // only rebinding vals in power mode for now.
- if (!isReplPower) default
- else intp runtimeClassAndTypeOfTerm id match {
- case Some((clazz, runtimeType)) =>
- val sym = intp.symbolOfTerm(id)
- if (sym.isStable) {
- val param = new NamedParam.Untyped(id, intp valueOfTerm id getOrElse null)
- TypeMemberCompletion(tpe, runtimeType, param)
- }
- else default
- case _ =>
- default
+ override def follow(id: String): Option[CompletionAware] = {
+ if (!completions(0).contains(id))
+ return None
+
+ val tpe = intp typeOfExpression id
+ if (tpe == NoType)
+ return None
+
+ def default = Some(TypeMemberCompletion(tpe))
+
+ // only rebinding vals in power mode for now.
+ if (!isReplPower) default
+ else intp runtimeClassAndTypeOfTerm id match {
+ case Some((clazz, runtimeType)) =>
+ val sym = intp.symbolOfTerm(id)
+ if (sym.isStable) {
+ val param = new NamedParam.Untyped(id, intp valueOfTerm id getOrElse null)
+ Some(TypeMemberCompletion(tpe, runtimeType, param))
}
- }
+ else default
+ case _ =>
+ default
}
- else
- None
}
override def toString = "<repl ids> (%s)".format(completions(0).size)
}
@@ -187,17 +197,10 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
private def imported = intp.sessionWildcards map TypeMemberCompletion.imported
// literal Ints, Strings, etc.
- object literals extends CompletionAware {
- def simpleParse(code: String): Tree = {
- val unit = new CompilationUnit(new util.BatchSourceFile("<console>", code))
- val scanner = new syntaxAnalyzer.UnitParser(unit)
- val tss = scanner.templateStatSeq(false)._2
-
- if (tss.size == 1) tss.head else EmptyTree
- }
-
+ object literals extends CompletionAware {
+ def simpleParse(code: String): Tree = newUnitParser(code).templateStats().last
def completions(verbosity: Int) = Nil
-
+
override def follow(id: String) = simpleParse(id) match {
case x: Literal => Some(new LiteralCompletion(x))
case _ => None
@@ -210,18 +213,18 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
override def follow(id: String) = id match {
case "_root_" => Some(this)
case _ => super.follow(id)
- }
+ }
}
// members of Predef
object predef extends TypeMemberCompletion(PredefModule.tpe) {
override def excludeEndsWith = super.excludeEndsWith ++ List("Wrapper", "ArrayOps")
override def excludeStartsWith = super.excludeStartsWith ++ List("wrap")
override def excludeNames = anyref.methodNames
-
+
override def exclude(name: String) = super.exclude(name) || (
(name contains "2")
)
-
+
override def completions(verbosity: Int) = verbosity match {
case 0 => Nil
case _ => super.completions(verbosity)
@@ -234,7 +237,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
override def exclude(name: String) = super.exclude(name) || (
skipArity(name)
)
-
+
override def completions(verbosity: Int) = verbosity match {
case 0 => filtered(packageNames ++ aliasNames)
case _ => super.completions(verbosity)
@@ -244,7 +247,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
object javalang extends PackageCompletion(JavaLangPackage.tpe) {
override lazy val excludeEndsWith = super.excludeEndsWith ++ List("Exception", "Error")
override lazy val excludeStartsWith = super.excludeStartsWith ++ List("CharacterData")
-
+
override def completions(verbosity: Int) = verbosity match {
case 0 => filtered(packageNames)
case _ => super.completions(verbosity)
@@ -256,7 +259,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
lazy val topLevelBase: List[CompletionAware] = List(ids, rootClass, predef, scalalang, javalang, literals)
def topLevel = topLevelBase ++ imported
def topLevelThreshold = 50
-
+
// the first tier of top level objects (doesn't include file completion)
def topLevelFor(parsed: Parsed): List[String] = {
val buf = new ListBuffer[String]
@@ -280,19 +283,6 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
if (parsed.isEmpty) xs map ("." + _) else xs
}
- // chasing down results which won't parse
- def execute(line: String): Option[ExecResult] = {
- val parsed = Parsed(line)
- def noDotOrSlash = line forall (ch => ch != '.' && ch != '/')
-
- if (noDotOrSlash) None // we defer all unqualified ids to the repl.
- else {
- (ids executionFor parsed) orElse
- (rootClass executionFor parsed) orElse
- (FileCompletion executionFor line)
- }
- }
-
// generic interface for querying (e.g. interpreter loop, testing)
def completions(buf: String): List[String] =
topLevelFor(Parsed.dotted(buf + ".", buf.length + 1))
@@ -327,11 +317,11 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
// This is jline's entry point for completion.
override def complete(buf: String, cursor: Int): Candidates = {
verbosity = if (isConsecutiveTabs(buf, cursor)) verbosity + 1 else 0
- DBG("\ncomplete(%s, %d) last = (%s, %d), verbosity: %s".format(buf, cursor, lastBuf, lastCursor, verbosity))
+ logDebug("\ncomplete(%s, %d) last = (%s, %d), verbosity: %s".format(buf, cursor, lastBuf, lastCursor, verbosity))
// we don't try lower priority completions unless higher ones return no results.
def tryCompletion(p: Parsed, completionFunction: Parsed => List[String]): Option[Candidates] = {
- val winners = completionFunction(p)
+ val winners = completionFunction(p)
if (winners.isEmpty)
return None
val newCursor =
@@ -340,39 +330,45 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
val advance = commonPrefix(winners)
lastCursor = p.position + advance.length
lastBuf = (buf take p.position) + advance
- DBG("tryCompletion(%s, _) lastBuf = %s, lastCursor = %s, p.position = %s".format(
+ logDebug("tryCompletion(%s, _) lastBuf = %s, lastCursor = %s, p.position = %s".format(
p, lastBuf, lastCursor, p.position))
p.position
}
-
+
Some(Candidates(newCursor, winners))
}
-
+
def mkDotted = Parsed.dotted(buf, cursor) withVerbosity verbosity
def mkUndelimited = Parsed.undelimited(buf, cursor) withVerbosity verbosity
// a single dot is special cased to completion on the previous result
def lastResultCompletion =
- if (!looksLikeInvocation(buf)) None
+ if (!looksLikeInvocation(buf)) None
else tryCompletion(Parsed.dotted(buf drop 1, cursor), lastResultFor)
- def regularCompletion = tryCompletion(mkDotted, topLevelFor)
- def fileCompletion =
- if (!looksLikePath(buf)) None
- else tryCompletion(mkUndelimited, FileCompletion completionsFor _.buffer)
-
- /** This is the kickoff point for all manner of theoretically possible compiler
- * unhappiness - fault may be here or elsewhere, but we don't want to crash the
- * repl regardless. Hopefully catching Exception is enough, but because the
- * compiler still throws some Errors it may not be.
+ def tryAll = (
+ lastResultCompletion
+ orElse tryCompletion(mkDotted, topLevelFor)
+ getOrElse Candidates(cursor, Nil)
+ )
+
+ /**
+ * This is the kickoff point for all manner of theoretically
+ * possible compiler unhappiness. The fault may be here or
+ * elsewhere, but we don't want to crash the repl regardless.
+ * The compiler makes it impossible to avoid catching Throwable
+ * with its unfortunate tendency to throw java.lang.Errors and
+ * AssertionErrors as the hats drop. We take two swings at it
+ * because there are some spots which like to throw an assertion
+ * once, then work after that. Yeah, what can I say.
*/
- try {
- (lastResultCompletion orElse regularCompletion orElse fileCompletion) getOrElse Candidates(cursor, Nil)
- }
- catch {
- case ex: Exception =>
- DBG("Error: complete(%s, %s) provoked %s".format(buf, cursor, ex))
- Candidates(cursor, List(" ", "<completion error: " + ex.getMessage + ">"))
+ try tryAll
+ catch { case ex: Throwable =>
+ logWarning("Error: complete(%s, %s) provoked".format(buf, cursor) + ex)
+ Candidates(cursor,
+ if (isReplDebug) List("<error:" + ex + ">")
+ else Nil
+ )
}
}
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
index d9e1de105c..60a4d7841e 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Stepan Koltsov
*/
@@ -15,28 +15,33 @@ import scala.collection.JavaConverters._
import Completion._
import io.Streamable.slurp
-/** Reads from the console using JLine */
-class SparkJLineReader(val completion: Completion) extends InteractiveReader {
+/**
+ * Reads from the console using JLine.
+ */
+class SparkJLineReader(_completion: => Completion) extends InteractiveReader {
val interactive = true
+ val consoleReader = new JLineConsoleReader()
+
+ lazy val completion = _completion
lazy val history: JLineHistory = JLineHistory()
- lazy val keyBindings =
- try KeyBinding parse slurp(term.getDefaultBindings)
- catch { case _: Exception => Nil }
private def term = consoleReader.getTerminal()
def reset() = term.reset()
def init() = term.init()
-
+
def scalaToJline(tc: ScalaCompleter): Completer = new Completer {
def complete(_buf: String, cursor: Int, candidates: JList[CharSequence]): Int = {
- val buf = if (_buf == null) "" else _buf
+ val buf = if (_buf == null) "" else _buf
val Candidates(newCursor, newCandidates) = tc.complete(buf, cursor)
newCandidates foreach (candidates add _)
newCursor
}
}
-
+
class JLineConsoleReader extends ConsoleReader with ConsoleReaderHelper {
+ if ((history: History) ne NoHistory)
+ this setHistory history
+
// working around protected/trait/java insufficiencies.
def goBack(num: Int): Unit = back(num)
def readOneKey(prompt: String) = {
@@ -46,34 +51,28 @@ class SparkJLineReader(val completion: Completion) extends InteractiveReader {
}
def eraseLine() = consoleReader.resetPromptLine("", "", 0)
def redrawLineAndFlush(): Unit = { flush() ; drawLine() ; flush() }
-
- this setBellEnabled false
- if (history ne NoHistory)
- this setHistory history
-
- if (completion ne NoCompletion) {
- val argCompletor: ArgumentCompleter =
- new ArgumentCompleter(new JLineDelimiter, scalaToJline(completion.completer()))
- argCompletor setStrict false
-
- this addCompleter argCompletor
- this setAutoprintThreshold 400 // max completion candidates without warning
+ // override def readLine(prompt: String): String
+
+ // A hook for running code after the repl is done initializing.
+ lazy val postInit: Unit = {
+ this setBellEnabled false
+
+ if (completion ne NoCompletion) {
+ val argCompletor: ArgumentCompleter =
+ new ArgumentCompleter(new JLineDelimiter, scalaToJline(completion.completer()))
+ argCompletor setStrict false
+
+ this addCompleter argCompletor
+ this setAutoprintThreshold 400 // max completion candidates without warning
+ }
}
}
-
- val consoleReader: JLineConsoleReader = new JLineConsoleReader()
- def currentLine: String = consoleReader.getCursorBuffer.buffer.toString
+ def currentLine = consoleReader.getCursorBuffer.buffer.toString
def redrawLine() = consoleReader.redrawLineAndFlush()
- def eraseLine() = {
- while (consoleReader.delete()) { }
- // consoleReader.eraseLine()
- }
+ def eraseLine() = consoleReader.eraseLine()
+ // Alternate implementation, not sure if/when I need this.
+ // def eraseLine() = while (consoleReader.delete()) { }
def readOneLine(prompt: String) = consoleReader readLine prompt
def readOneKey(prompt: String) = consoleReader readOneKey prompt
}
-
-object SparkJLineReader {
- def apply(intp: SparkIMain): SparkJLineReader = apply(new SparkJLineCompletion(intp))
- def apply(comp: Completion): SparkJLineReader = new SparkJLineReader(comp)
-}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
index a3409bf665..382f8360a7 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Martin Odersky
*/
@@ -10,13 +10,14 @@ import scala.tools.nsc.interpreter._
import scala.collection.{ mutable, immutable }
import scala.PartialFunction.cond
-import scala.reflect.NameTransformer
-import util.Chars
+import scala.reflect.internal.Chars
+import scala.reflect.internal.Flags._
+import scala.language.implicitConversions
trait SparkMemberHandlers {
val intp: SparkIMain
- import intp.{ Request, global, naming, atPickler }
+ import intp.{ Request, global, naming }
import global._
import naming._
@@ -29,7 +30,7 @@ trait SparkMemberHandlers {
front + (xs map string2codeQuoted mkString " + ")
}
private implicit def name2string(name: Name) = name.toString
-
+
/** A traverser that finds all mentioned identifiers, i.e. things
* that need to be imported. It might return extra names.
*/
@@ -54,26 +55,28 @@ trait SparkMemberHandlers {
}
def chooseHandler(member: Tree): MemberHandler = member match {
- case member: DefDef => new DefHandler(member)
- case member: ValDef => new ValHandler(member)
- case member@Assign(Ident(_), _) => new AssignHandler(member)
- case member: ModuleDef => new ModuleHandler(member)
- case member: ClassDef => new ClassHandler(member)
- case member: TypeDef => new TypeAliasHandler(member)
- case member: Import => new ImportHandler(member)
- case DocDef(_, documented) => chooseHandler(documented)
- case member => new GenericHandler(member)
+ case member: DefDef => new DefHandler(member)
+ case member: ValDef => new ValHandler(member)
+ case member: Assign => new AssignHandler(member)
+ case member: ModuleDef => new ModuleHandler(member)
+ case member: ClassDef => new ClassHandler(member)
+ case member: TypeDef => new TypeAliasHandler(member)
+ case member: Import => new ImportHandler(member)
+ case DocDef(_, documented) => chooseHandler(documented)
+ case member => new GenericHandler(member)
}
-
+
sealed abstract class MemberDefHandler(override val member: MemberDef) extends MemberHandler(member) {
+ def symbol = if (member.symbol eq null) NoSymbol else member.symbol
def name: Name = member.name
def mods: Modifiers = member.mods
def keyword = member.keyword
- def prettyName = NameTransformer.decode(name)
-
+ def prettyName = name.decode
+
override def definesImplicit = member.mods.isImplicit
override def definesTerm: Option[TermName] = Some(name.toTermName) filter (_ => name.isTermName)
override def definesType: Option[TypeName] = Some(name.toTypeName) filter (_ => name.isTypeName)
+ override def definedSymbols = if (symbol eq NoSymbol) Nil else List(symbol)
}
/** Class to handle one member among all the members included
@@ -82,11 +85,8 @@ trait SparkMemberHandlers {
sealed abstract class MemberHandler(val member: Tree) {
def definesImplicit = false
def definesValue = false
- def isLegalTopLevel = member match {
- case _: ModuleDef | _: ClassDef | _: Import => true
- case _ => false
- }
-
+ def isLegalTopLevel = false
+
def definesTerm = Option.empty[TermName]
def definesType = Option.empty[TypeName]
@@ -94,6 +94,7 @@ trait SparkMemberHandlers {
def importedNames = List[Name]()
def definedNames = definesTerm.toList ++ definesType.toList
def definedOrImported = definedNames ++ importedNames
+ def definedSymbols = List[Symbol]()
def extraCodeToEvaluate(req: Request): String = ""
def resultExtractionCode(req: Request): String = ""
@@ -103,11 +104,11 @@ trait SparkMemberHandlers {
}
class GenericHandler(member: Tree) extends MemberHandler(member)
-
+
class ValHandler(member: ValDef) extends MemberDefHandler(member) {
- val maxStringElements = 1000 // no need to mkString billions of elements
+ val maxStringElements = 1000 // no need to mkString billions of elements
override def definesValue = true
-
+
override def resultExtractionCode(req: Request): String = {
val isInternal = isUserVarName(name) && req.lookupTypeOf(name) == "Unit"
if (!mods.isPublic || isInternal) ""
@@ -116,22 +117,27 @@ trait SparkMemberHandlers {
val resultString =
if (mods.isLazy) codegenln(false, "<lazy>")
else any2stringOf(req fullPath name, maxStringElements)
-
- """ + "%s: %s = " + %s""".format(prettyName, string2code(req typeOf name), resultString)
+
+ val vidString =
+ if (replProps.vids) """" + " @ " + "%%8x".format(System.identityHashCode(%s)) + " """.trim.format(req fullPath name)
+ else ""
+
+ """ + "%s%s: %s = " + %s""".format(string2code(prettyName), vidString, string2code(req typeOf name), resultString)
}
}
}
class DefHandler(member: DefDef) extends MemberDefHandler(member) {
private def vparamss = member.vparamss
- // true if 0-arity
- override def definesValue = vparamss.isEmpty || vparamss.head.isEmpty
+ private def isMacro = member.symbol hasFlag MACRO
+ // true if not a macro and 0-arity
+ override def definesValue = !isMacro && flattensToEmpty(vparamss)
override def resultExtractionCode(req: Request) =
if (mods.isPublic) codegenln(name, ": ", req.typeOf(name)) else ""
}
class AssignHandler(member: Assign) extends MemberHandler(member) {
- val lhs = member.lhs.asInstanceOf[Ident] // an unfortunate limitation
+ val Assign(lhs, rhs) = member
val name = newTermName(freshInternalVarName())
override def definesTerm = Some(name)
@@ -142,15 +148,15 @@ trait SparkMemberHandlers {
/** Print out lhs instead of the generated varName */
override def resultExtractionCode(req: Request) = {
val lhsType = string2code(req lookupTypeOf name)
- val res = string2code(req fullPath name)
-
- """ + "%s: %s = " + %s + "\n" """.format(lhs, lhsType, res) + "\n"
+ val res = string2code(req fullPath name)
+ """ + "%s: %s = " + %s + "\n" """.format(string2code(lhs.toString), lhsType, res) + "\n"
}
}
class ModuleHandler(module: ModuleDef) extends MemberDefHandler(module) {
override def definesTerm = Some(name)
override def definesValue = true
+ override def isLegalTopLevel = true
override def resultExtractionCode(req: Request) = codegenln("defined module ", name)
}
@@ -158,7 +164,8 @@ trait SparkMemberHandlers {
class ClassHandler(member: ClassDef) extends MemberDefHandler(member) {
override def definesType = Some(name.toTypeName)
override def definesTerm = Some(name.toTermName) filter (_ => mods.isCase)
-
+ override def isLegalTopLevel = true
+
override def resultExtractionCode(req: Request) =
codegenln("defined %s %s".format(keyword, name))
}
@@ -173,26 +180,42 @@ trait SparkMemberHandlers {
class ImportHandler(imp: Import) extends MemberHandler(imp) {
val Import(expr, selectors) = imp
- def targetType = intp.typeOfExpression("" + expr)
-
+ def targetType: Type = intp.typeOfExpression("" + expr)
+ override def isLegalTopLevel = true
+
+ def createImportForName(name: Name): String = {
+ selectors foreach {
+ case sel @ ImportSelector(old, _, `name`, _) => return "import %s.{ %s }".format(expr, sel)
+ case _ => ()
+ }
+ "import %s.%s".format(expr, name)
+ }
+ // TODO: Need to track these specially to honor Predef masking attempts,
+ // because they must be the leading imports in the code generated for each
+ // line. We can use the same machinery as Contexts now, anyway.
+ def isPredefImport = isReferenceToPredef(expr)
+
// wildcard imports, e.g. import foo._
private def selectorWild = selectors filter (_.name == nme.USCOREkw)
// renamed imports, e.g. import foo.{ bar => baz }
private def selectorRenames = selectors map (_.rename) filterNot (_ == null)
-
+
/** Whether this import includes a wildcard import */
val importsWildcard = selectorWild.nonEmpty
-
+
+ /** Whether anything imported is implicit .*/
+ def importsImplicit = implicitSymbols.nonEmpty
+
def implicitSymbols = importedSymbols filter (_.isImplicit)
def importedSymbols = individualSymbols ++ wildcardSymbols
-
+
lazy val individualSymbols: List[Symbol] =
- atPickler(targetType.toList flatMap (tp => individualNames map (tp nonPrivateMember _)))
+ beforePickler(individualNames map (targetType nonPrivateMember _))
lazy val wildcardSymbols: List[Symbol] =
- if (importsWildcard) atPickler(targetType.toList flatMap (_.nonPrivateMembers))
+ if (importsWildcard) beforePickler(targetType.nonPrivateMembers.toList)
else Nil
-
+
/** Complete list of names imported by a wildcard */
lazy val wildcardNames: List[Name] = wildcardSymbols map (_.name)
lazy val individualNames: List[Name] = selectorRenames filterNot (_ == nme.USCOREkw) flatMap (_.bothNames)
@@ -200,7 +223,7 @@ trait SparkMemberHandlers {
/** The names imported by this statement */
override lazy val importedNames: List[Name] = wildcardNames ++ individualNames
lazy val importsSymbolNamed: Set[String] = importedNames map (_.toString) toSet
-
+
def importString = imp.toString
override def resultExtractionCode(req: Request) = codegenln(importString) + "\n"
}
diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index 8f9b632c0e..fccb6e652c 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -1,27 +1,9 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
package org.apache.spark.repl
import java.io._
import java.net.URLClassLoader
import scala.collection.mutable.ArrayBuffer
-import scala.collection.JavaConversions._
import org.scalatest.FunSuite
import com.google.common.io.Files
@@ -56,92 +38,92 @@ class ReplSuite extends FunSuite {
def assertContains(message: String, output: String) {
assert(output.contains(message),
- "Interpreter output did not contain '" + message + "':\n" + output)
+ "Interpreter output did not contain '" + message + "':\n" + output)
}
def assertDoesNotContain(message: String, output: String) {
assert(!output.contains(message),
- "Interpreter output contained '" + message + "':\n" + output)
+ "Interpreter output contained '" + message + "':\n" + output)
}
- test ("simple foreach with accumulator") {
+ test("simple foreach with accumulator") {
val output = runInterpreter("local", """
- val accum = sc.accumulator(0)
- sc.parallelize(1 to 10).foreach(x => accum += x)
- accum.value
- """)
+ |val accum = sc.accumulator(0)
+ |sc.parallelize(1 to 10).foreach(x => accum += x)
+ |accum.value
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res1: Int = 55", output)
}
- test ("external vars") {
+ test("external vars") {
val output = runInterpreter("local", """
- var v = 7
- sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
- """)
+ |var v = 7
+ |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
assertContains("res1: Int = 100", output)
}
- test ("external classes") {
+ test("external classes") {
val output = runInterpreter("local", """
- class C {
- def foo = 5
- }
- sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
- """)
+ |class C {
+ |def foo = 5
+ |}
+ |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 50", output)
}
- test ("external functions") {
+ test("external functions") {
val output = runInterpreter("local", """
- def double(x: Int) = x + x
- sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
- """)
+ |def double(x: Int) = x + x
+ |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 110", output)
}
- test ("external functions that access vars") {
+ test("external functions that access vars") {
val output = runInterpreter("local", """
- var v = 7
- def getV() = v
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- """)
+ |var v = 7
+ |def getV() = v
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
assertContains("res1: Int = 100", output)
}
- test ("broadcast vars") {
+ test("broadcast vars") {
// Test that the value that a broadcast var had when it was created is used,
// even if that variable is then modified in the driver program
// TODO: This doesn't actually work for arrays when we run in local mode!
val output = runInterpreter("local", """
- var array = new Array[Int](5)
- val broadcastArray = sc.broadcast(array)
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- array(0) = 5
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- """)
+ |var array = new Array[Int](5)
+ |val broadcastArray = sc.broadcast(array)
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ |array(0) = 5
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output)
assertContains("res2: Array[Int] = Array(5, 0, 0, 0, 0)", output)
}
- test ("interacting with files") {
+ test("interacting with files") {
val tempDir = Files.createTempDir()
val out = new FileWriter(tempDir + "/input")
out.write("Hello world!\n")
@@ -149,11 +131,11 @@ class ReplSuite extends FunSuite {
out.write("Goodbye\n")
out.close()
val output = runInterpreter("local", """
- var file = sc.textFile("%s/input").cache()
- file.count()
- file.count()
- file.count()
- """.format(tempDir.getAbsolutePath))
+ |var file = sc.textFile("%s/input").cache()
+ |file.count()
+ |file.count()
+ |file.count()
+ """.stripMargin.format(tempDir.getAbsolutePath))
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Long = 3", output)
@@ -161,19 +143,19 @@ class ReplSuite extends FunSuite {
assertContains("res2: Long = 3", output)
}
- test ("local-cluster mode") {
+ test("local-cluster mode") {
val output = runInterpreter("local-cluster[1,1,512]", """
- var v = 7
- def getV() = v
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- var array = new Array[Int](5)
- val broadcastArray = sc.broadcast(array)
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- array(0) = 5
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- """)
+ |var v = 7
+ |def getV() = v
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |var array = new Array[Int](5)
+ |val broadcastArray = sc.broadcast(array)
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ |array(0) = 5
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
@@ -183,19 +165,19 @@ class ReplSuite extends FunSuite {
}
if (System.getenv("MESOS_NATIVE_LIBRARY") != null) {
- test ("running on Mesos") {
+ test("running on Mesos") {
val output = runInterpreter("localquiet", """
- var v = 7
- def getV() = v
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- var array = new Array[Int](5)
- val broadcastArray = sc.broadcast(array)
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- array(0) = 5
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- """)
+ |var v = 7
+ |def getV() = v
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |var array = new Array[Int](5)
+ |val broadcastArray = sc.broadcast(array)
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ |array(0) = 5
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
diff --git a/run-example b/run-example
index 08ec717ca5..feade6589a 100755
--- a/run-example
+++ b/run-example
@@ -17,7 +17,7 @@
# limitations under the License.
#
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Figure out where the Scala framework is installed
FWDIR="$(cd `dirname $0`; pwd)"
diff --git a/run-example2.cmd b/run-example2.cmd
index dbb371ecfc..d4ad98d6e7 100644
--- a/run-example2.cmd
+++ b/run-example2.cmd
@@ -17,7 +17,7 @@ rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
set FWDIR=%~dp0
diff --git a/spark-class b/spark-class
index e111ef6da7..5305b3d025 100755
--- a/spark-class
+++ b/spark-class
@@ -17,7 +17,7 @@
# limitations under the License.
#
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Figure out where the Scala framework is installed
FWDIR="$(cd `dirname $0`; pwd)"
@@ -115,3 +115,4 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
fi
exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+
diff --git a/streaming/pom.xml b/streaming/pom.xml
index b260a72abb..c74e76d526 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -86,17 +86,17 @@
</dependency>
<dependency>
<groupId>com.typesafe.akka</groupId>
- <artifactId>akka-zeromq</artifactId>
- <version>2.0.3</version>
+ <artifactId>akka-zeromq_${scala-short.version}</artifactId>
+ <version>${akka.version}</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -109,10 +109,14 @@
<artifactId>slf4j-log4j12</artifactId>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ </dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index 80da6bd30b..cd404fd408 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -29,6 +29,7 @@ import org.apache.spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
+import scala.reflect.ClassTag
import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
@@ -56,7 +57,7 @@ import org.apache.hadoop.conf.Configuration
* - A function that is used to generate an RDD after each time interval
*/
-abstract class DStream[T: ClassManifest] (
+abstract class DStream[T: ClassTag] (
@transient protected[streaming] var ssc: StreamingContext
) extends Serializable with Logging {
@@ -82,7 +83,7 @@ abstract class DStream[T: ClassManifest] (
// RDDs generated, marked as protected[streaming] so that testsuites can access it
@transient
protected[streaming] var generatedRDDs = new HashMap[Time, RDD[T]] ()
-
+
// Time zero for the DStream
protected[streaming] var zeroTime: Time = null
@@ -274,16 +275,16 @@ abstract class DStream[T: ClassManifest] (
/**
* Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal
* method that should not be called directly.
- */
+ */
protected[streaming] def getOrCompute(time: Time): Option[RDD[T]] = {
// If this DStream was not initialized (i.e., zeroTime not set), then do it
// If RDD was already generated, then retrieve it from HashMap
generatedRDDs.get(time) match {
-
- // If an RDD was already generated and is being reused, then
+
+ // If an RDD was already generated and is being reused, then
// probably all RDDs in this DStream will be reused and hence should be cached
case Some(oldRDD) => Some(oldRDD)
-
+
// if RDD was not generated, and if the time is valid
// (based on sliding time of this DStream), then generate the RDD
case None => {
@@ -300,7 +301,7 @@ abstract class DStream[T: ClassManifest] (
}
generatedRDDs.put(time, newRDD)
Some(newRDD)
- case None =>
+ case None =>
None
}
} else {
@@ -344,7 +345,7 @@ abstract class DStream[T: ClassManifest] (
dependencies.foreach(_.clearOldMetadata(time))
}
- /* Adds metadata to the Stream while it is running.
+ /* Adds metadata to the Stream while it is running.
* This methd should be overwritten by sublcasses of InputDStream.
*/
protected[streaming] def addMetadata(metadata: Any) {
@@ -416,7 +417,7 @@ abstract class DStream[T: ClassManifest] (
// =======================================================================
/** Return a new DStream by applying a function to all elements of this DStream. */
- def map[U: ClassManifest](mapFunc: T => U): DStream[U] = {
+ def map[U: ClassTag](mapFunc: T => U): DStream[U] = {
new MappedDStream(this, context.sparkContext.clean(mapFunc))
}
@@ -424,7 +425,7 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream by applying a function to all elements of this DStream,
* and then flattening the results
*/
- def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = {
+ def flatMap[U: ClassTag](flatMapFunc: T => Traversable[U]): DStream[U] = {
new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc))
}
@@ -443,7 +444,7 @@ abstract class DStream[T: ClassManifest] (
* of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
* of the RDD.
*/
- def mapPartitions[U: ClassManifest](
+ def mapPartitions[U: ClassTag](
mapPartFunc: Iterator[T] => Iterator[U],
preservePartitioning: Boolean = false
): DStream[U] = {
@@ -490,16 +491,14 @@ abstract class DStream[T: ClassManifest] (
* this DStream will be registered as an output stream and therefore materialized.
*/
def foreach(foreachFunc: (RDD[T], Time) => Unit) {
- val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
- ssc.registerOutputStream(newStream)
- newStream
+ ssc.registerOutputStream(new ForEachDStream(this, context.sparkContext.clean(foreachFunc)))
}
/**
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of this DStream.
*/
- def transform[U: ClassManifest](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
+ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
transform((r: RDD[T], t: Time) => transformFunc(r))
}
@@ -507,7 +506,7 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of this DStream.
*/
- def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
+ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
new TransformedDStream(this, context.sparkContext.clean(transformFunc))
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
index 58a0da2870..3fd5d52403 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
@@ -20,13 +20,16 @@ package org.apache.spark.streaming
import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.conf.Configuration
+
import collection.mutable.HashMap
import org.apache.spark.Logging
+import scala.collection.mutable.HashMap
+import scala.reflect.ClassTag
private[streaming]
-class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T])
+class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T])
extends Serializable with Logging {
protected val data = new HashMap[Time, AnyRef]()
@@ -107,4 +110,3 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T])
"[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]"
}
}
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
index aae79a4e6f..b761646dff 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
@@ -28,8 +28,7 @@ import scala.collection.mutable.Queue
import akka.actor._
import akka.pattern.ask
-import akka.util.duration._
-import akka.dispatch._
+import scala.concurrent.duration._
private[streaming] sealed trait NetworkInputTrackerMessage
private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
index 757bc98981..f021e29619 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -24,10 +24,11 @@ import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStrea
import org.apache.spark.{Partitioner, HashPartitioner}
import org.apache.spark.SparkContext._
-import org.apache.spark.rdd.{Manifests, RDD, PairRDDFunctions}
+import org.apache.spark.rdd.{ClassTags, RDD, PairRDDFunctions}
import org.apache.spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.mapred.{JobConf, OutputFormat}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
@@ -35,7 +36,7 @@ import org.apache.hadoop.mapred.OutputFormat
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.conf.Configuration
-class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)])
+class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)])
extends Serializable {
private[streaming] def ssc = self.ssc
@@ -105,7 +106,7 @@ extends Serializable {
* combineByKey for RDDs. Please refer to combineByKey in
* [[org.apache.spark.rdd.PairRDDFunctions]] for more information.
*/
- def combineByKey[C: ClassManifest](
+ def combineByKey[C: ClassTag](
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
@@ -205,7 +206,7 @@ extends Serializable {
* DStream's batching interval
*/
def reduceByKeyAndWindow(
- reduceFunc: (V, V) => V,
+ reduceFunc: (V, V) => V,
windowDuration: Duration,
slideDuration: Duration
): DStream[(K, V)] = {
@@ -336,7 +337,7 @@ extends Serializable {
* corresponding state key-value pair will be eliminated.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Seq[V], Option[S]) => Option[S]
): DStream[(K, S)] = {
updateStateByKey(updateFunc, defaultPartitioner())
@@ -351,7 +352,7 @@ extends Serializable {
* @param numPartitions Number of partitions of each RDD in the new DStream.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Seq[V], Option[S]) => Option[S],
numPartitions: Int
): DStream[(K, S)] = {
@@ -367,7 +368,7 @@ extends Serializable {
* @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Seq[V], Option[S]) => Option[S],
partitioner: Partitioner
): DStream[(K, S)] = {
@@ -390,7 +391,7 @@ extends Serializable {
* @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
partitioner: Partitioner,
rememberPartitioner: Boolean
@@ -399,11 +400,11 @@ extends Serializable {
}
- def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = {
+ def mapValues[U: ClassTag](mapValuesFunc: V => U): DStream[(K, U)] = {
new MapValuedDStream[K, V, U](self, mapValuesFunc)
}
- def flatMapValues[U: ClassManifest](
+ def flatMapValues[U: ClassTag](
flatMapValuesFunc: V => TraversableOnce[U]
): DStream[(K, U)] = {
new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc)
@@ -415,7 +416,7 @@ extends Serializable {
* key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
* of partitions.
*/
- def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
+ def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner())
}
@@ -424,7 +425,7 @@ extends Serializable {
* or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
* key in both RDDs. Partitioner is used to partition each generated RDD.
*/
- def cogroup[W: ClassManifest](
+ def cogroup[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (Seq[V], Seq[W]))] = {
@@ -434,8 +435,8 @@ extends Serializable {
partitioner
)
val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)(
- classManifest[K],
- Manifests.seqSeqManifest
+ classTag[K],
+ ClassTags.seqSeqClassTag
)
pdfs.mapValues {
case Seq(vs, ws) =>
@@ -447,7 +448,7 @@ extends Serializable {
* Join `this` DStream with `other` DStream. HashPartitioner is used
* to partition each generated RDD into default number of partitions.
*/
- def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
+ def join[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
join[W](other, defaultPartitioner())
}
@@ -456,7 +457,7 @@ extends Serializable {
* be generated by joining RDDs from `this` and other DStream. Uses the given
* Partitioner to partition each generated RDD.
*/
- def join[W: ClassManifest](
+ def join[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (V, W))] = {
@@ -474,8 +475,8 @@ extends Serializable {
def saveAsHadoopFiles[F <: OutputFormat[K, V]](
prefix: String,
suffix: String
- )(implicit fm: ClassManifest[F]) {
- saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ )(implicit fm: ClassTag[F]) {
+ saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -504,8 +505,8 @@ extends Serializable {
def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](
prefix: String,
suffix: String
- )(implicit fm: ClassManifest[F]) {
- saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ )(implicit fm: ClassTag[F]) {
+ saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -527,9 +528,7 @@ extends Serializable {
self.foreach(saveFunc)
}
- private def getKeyClass() = implicitly[ClassManifest[K]].erasure
+ private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass
- private def getValueClass() = implicitly[ClassManifest[V]].erasure
+ private def getValueClass() = implicitly[ClassTag[V]].runtimeClass
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 878725c705..c722aa15ab 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -34,6 +34,7 @@ import org.apache.spark.streaming.receivers.ActorReceiver
import scala.collection.mutable.Queue
import scala.collection.Map
+import scala.reflect.ClassTag
import java.io.InputStream
import java.util.concurrent.atomic.AtomicInteger
@@ -46,6 +47,7 @@ import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
import org.apache.hadoop.fs.Path
import twitter4j.Status
import twitter4j.auth.Authorization
+import akka.util.ByteString
/**
@@ -187,7 +189,7 @@ class StreamingContext private (
* Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html
* @param receiver Custom implementation of NetworkReceiver
*/
- def networkStream[T: ClassManifest](
+ def networkStream[T: ClassTag](
receiver: NetworkReceiver[T]): DStream[T] = {
val inputStream = new PluggableInputDStream[T](this,
receiver)
@@ -207,7 +209,7 @@ class StreamingContext private (
* to ensure the type safety, i.e parametrized type of data received and actorStream
* should be same.
*/
- def actorStream[T: ClassManifest](
+ def actorStream[T: ClassTag](
props: Props,
name: String,
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
@@ -227,14 +229,14 @@ class StreamingContext private (
* and sub sequence refer to its payload.
* @param storageLevel RDD storage level. Defaults to memory-only.
*/
- def zeroMQStream[T: ClassManifest](
+ def zeroMQStream[T: ClassTag](
publisherUrl:String,
subscribe: Subscribe,
- bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T],
+ bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
): DStream[T] = {
- actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)),
+ actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
"ZeroMQReceiver", storageLevel, supervisorStrategy)
}
@@ -266,7 +268,7 @@ class StreamingContext private (
* in its own thread.
* @param storageLevel Storage level to use for storing the received objects
*/
- def kafkaStream[T: ClassManifest, D <: kafka.serializer.Decoder[_]: Manifest](
+ def kafkaStream[T: ClassTag, D <: kafka.serializer.Decoder[_]: Manifest](
kafkaParams: Map[String, String],
topics: Map[String, Int],
storageLevel: StorageLevel
@@ -303,7 +305,7 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects received (after converting bytes to objects)
*/
- def socketStream[T: ClassManifest](
+ def socketStream[T: ClassTag](
hostname: String,
port: Int,
converter: (InputStream) => Iterator[T],
@@ -325,7 +327,7 @@ class StreamingContext private (
port: Int,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[SparkFlumeEvent] = {
- val inputStream = new FlumeInputDStream(this, hostname, port, storageLevel)
+ val inputStream = new FlumeInputDStream[SparkFlumeEvent](this, hostname, port, storageLevel)
registerInputStream(inputStream)
inputStream
}
@@ -340,7 +342,7 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects in the received blocks
*/
- def rawSocketStream[T: ClassManifest](
+ def rawSocketStream[T: ClassTag](
hostname: String,
port: Int,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
@@ -360,9 +362,9 @@ class StreamingContext private (
* @tparam F Input format for reading HDFS file
*/
def fileStream[
- K: ClassManifest,
- V: ClassManifest,
- F <: NewInputFormat[K, V]: ClassManifest
+ K: ClassTag,
+ V: ClassTag,
+ F <: NewInputFormat[K, V]: ClassTag
] (directory: String): DStream[(K, V)] = {
val inputStream = new FileInputDStream[K, V, F](this, directory)
registerInputStream(inputStream)
@@ -380,9 +382,9 @@ class StreamingContext private (
* @tparam F Input format for reading HDFS file
*/
def fileStream[
- K: ClassManifest,
- V: ClassManifest,
- F <: NewInputFormat[K, V]: ClassManifest
+ K: ClassTag,
+ V: ClassTag,
+ F <: NewInputFormat[K, V]: ClassTag
] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = {
val inputStream = new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
registerInputStream(inputStream)
@@ -424,7 +426,7 @@ class StreamingContext private (
* @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
* @tparam T Type of objects in the RDD
*/
- def queueStream[T: ClassManifest](
+ def queueStream[T: ClassTag](
queue: Queue[RDD[T]],
oneAtATime: Boolean = true
): DStream[T] = {
@@ -440,7 +442,7 @@ class StreamingContext private (
* Set as null if no RDD should be returned when empty
* @tparam T Type of objects in the RDD
*/
- def queueStream[T: ClassManifest](
+ def queueStream[T: ClassTag](
queue: Queue[RDD[T]],
oneAtATime: Boolean,
defaultRDD: RDD[T]
@@ -453,7 +455,7 @@ class StreamingContext private (
/**
* Create a unified DStream from multiple DStreams of the same type and same interval
*/
- def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = {
+ def union[T: ClassTag](streams: Seq[DStream[T]]): DStream[T] = {
new UnionDStream[T](streams.toArray)
}
@@ -530,7 +532,7 @@ class StreamingContext private (
object StreamingContext {
- implicit def toPairDStreamFunctions[K: ClassManifest, V: ClassManifest](stream: DStream[(K,V)]) = {
+ implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = {
new PairDStreamFunctions[K, V](stream)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
index d1932b6b05..0d54d78ed3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
@@ -23,6 +23,8 @@ import org.apache.spark.api.java.JavaRDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
+
/**
* A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
* sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
@@ -41,7 +43,7 @@ import org.apache.spark.rdd.RDD
* - A time interval at which the DStream generates an RDD
* - A function that is used to generate an RDD after each time interval
*/
-class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassManifest[T])
+class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T])
extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] {
override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd)
@@ -97,6 +99,6 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
}
object JavaDStream {
- implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] =
+ implicit def fromDStream[T: ClassTag](dstream: DStream[T]): JavaDStream[T] =
new JavaDStream[T](dstream)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
index 459695b7ca..4508e48590 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
@@ -21,6 +21,7 @@ import java.util.{List => JList}
import java.lang.{Long => JLong}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.spark.streaming._
import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD}
@@ -31,7 +32,7 @@ import JavaDStream._
trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]
extends Serializable {
- implicit val classManifest: ClassManifest[T]
+ implicit val classTag: ClassTag[T]
def dstream: DStream[T]
@@ -133,7 +134,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
/** Return a new DStream by applying a function to all elements of this DStream. */
def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = {
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType())
}
@@ -154,7 +155,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType())
}
@@ -257,8 +258,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* on each RDD of this DStream.
*/
def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
def scalaTransform (in: RDD[T]): RDD[U] =
transformFunc.call(wrapRDD(in)).rdd
dstream.transform(scalaTransform(_))
@@ -269,8 +270,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* on each RDD of this DStream.
*/
def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
def scalaTransform (in: RDD[T], time: Time): RDD[U] =
transformFunc.call(wrapRDD(in), time).rdd
dstream.transform(scalaTransform(_, _))
@@ -282,10 +283,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
*/
def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
- implicit val cmk: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+ implicit val cmk: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
def scalaTransform (in: RDD[T]): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(in)).rdd
dstream.transform(scalaTransform(_))
@@ -297,10 +298,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
*/
def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
- implicit val cmk: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+ implicit val cmk: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(in), time).rdd
dstream.transform(scalaTransform(_, _))
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index 978fca33ad..c80545b530 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -21,6 +21,7 @@ import java.util.{List => JList}
import java.lang.{Long => JLong}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
@@ -36,8 +37,8 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.PairRDDFunctions
class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
- implicit val kManifiest: ClassManifest[K],
- implicit val vManifest: ClassManifest[V])
+ implicit val kTag: ClassTag[K],
+ implicit val vTag: ClassTag[V])
extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] {
override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
@@ -156,8 +157,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
mergeCombiners: JFunction2[C, C, C],
partitioner: Partitioner
): JavaPairDStream[K, C] = {
- implicit val cm: ClassManifest[C] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ implicit val cm: ClassTag[C] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner)
}
@@ -422,8 +423,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
*/
def updateStateByKey[S](updateFunc: JFunction2[JList[V], Optional[S], Optional[S]])
: JavaPairDStream[K, S] = {
- implicit val cm: ClassManifest[S] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
+ implicit val cm: ClassTag[S] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]]
dstream.updateStateByKey(convertUpdateStateFunction(updateFunc))
}
@@ -436,7 +437,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* @param numPartitions Number of partitions of each RDD in the new DStream.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
numPartitions: Int)
: JavaPairDStream[K, S] = {
@@ -452,7 +453,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
partitioner: Partitioner
): JavaPairDStream[K, S] = {
@@ -460,16 +461,16 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
dstream.mapValues(f)
}
def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = {
import scala.collection.JavaConverters._
def fn = (x: V) => f.apply(x).asScala
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
dstream.flatMapValues(fn)
}
@@ -480,8 +481,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* of partitions.
*/
def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
}
@@ -492,8 +493,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
*/
def cogroup[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
: JavaPairDStream[K, (JList[V], JList[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.cogroup(other.dstream, partitioner)
.mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
}
@@ -503,8 +504,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* to partition each generated RDD into default number of partitions.
*/
def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.join(other.dstream)
}
@@ -515,8 +516,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
*/
def join[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
: JavaPairDStream[K, (V, W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.join(other.dstream, partitioner)
}
@@ -590,24 +591,24 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf)
}
- override val classManifest: ClassManifest[(K, V)] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+ override val classTag: ClassTag[(K, V)] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]]
}
object JavaPairDStream {
- implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)])
+ implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)])
:JavaPairDStream[K, V] =
new JavaPairDStream[K, V](dstream)
def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = {
- implicit val cmk: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val cmv: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cmk: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val cmv: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
new JavaPairDStream[K, V](dstream.dstream)
}
- def scalaToJavaLong[K: ClassManifest](dstream: JavaPairDStream[K, Long])
+ def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long])
: JavaPairDStream[K, JLong] = {
StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_))
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 54ba3e6025..8242af6d5f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -22,12 +22,15 @@ import java.io.InputStream
import java.util.{Map => JMap}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import twitter4j.Status
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.zeromq.Subscribe
+import akka.util.ByteString
+
import twitter4j.auth.Authorization
import org.apache.spark.rdd.RDD
@@ -142,10 +145,11 @@ class JavaStreamingContext(val ssc: StreamingContext) {
groupId: String,
topics: JMap[String, JInt])
: JavaDStream[String] = {
- implicit val cmt: ClassManifest[String] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]]
+ implicit val cmt: ClassTag[String] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
StorageLevel.MEMORY_ONLY_SER_2)
+
}
/**
@@ -163,8 +167,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
topics: JMap[String, JInt],
storageLevel: StorageLevel)
: JavaDStream[String] = {
- implicit val cmt: ClassManifest[String] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]]
+ implicit val cmt: ClassTag[String] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
storageLevel)
}
@@ -186,8 +190,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
topics: JMap[String, JInt],
storageLevel: StorageLevel)
: JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]]
ssc.kafkaStream[T, D](
kafkaParams.toMap,
@@ -237,8 +241,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
storageLevel: StorageLevel)
: JavaDStream[T] = {
def fn = (x: InputStream) => converter.apply(x).toIterator
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.socketStream(hostname, port, fn, storageLevel)
}
@@ -266,8 +270,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
hostname: String,
port: Int,
storageLevel: StorageLevel): JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel))
}
@@ -281,8 +285,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam T Type of the objects in the received blocks
*/
def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port))
}
@@ -296,12 +300,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam F Input format for reading HDFS file
*/
def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = {
- implicit val cmk: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val cmv: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
- implicit val cmf: ClassManifest[F] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]]
+ implicit val cmk: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val cmv: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+ implicit val cmf: ClassTag[F] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[F]]
ssc.fileStream[K, V, F](directory);
}
@@ -396,7 +400,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
def twitterStream(): JavaDStream[Status] = {
ssc.twitterStream()
}
-
+
/**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
@@ -414,8 +418,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
storageLevel: StorageLevel,
supervisorStrategy: SupervisorStrategy
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.actorStream[T](props, name, storageLevel, supervisorStrategy)
}
@@ -435,8 +439,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
name: String,
storageLevel: StorageLevel
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.actorStream[T](props, name, storageLevel)
}
@@ -454,8 +458,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
props: Props,
name: String
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.actorStream[T](props, name)
}
@@ -472,12 +476,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
def zeroMQStream[T](
publisherUrl:String,
subscribe: Subscribe,
- bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T],
+ bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
storageLevel: StorageLevel,
supervisorStrategy: SupervisorStrategy
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy)
}
@@ -497,9 +501,9 @@ class JavaStreamingContext(val ssc: StreamingContext) {
bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
storageLevel: StorageLevel
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
- def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+ def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
}
@@ -517,9 +521,9 @@ class JavaStreamingContext(val ssc: StreamingContext) {
subscribe: Subscribe,
bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
- def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+ def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
}
@@ -539,8 +543,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam T Type of objects in the RDD
*/
def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val sQueue = new scala.collection.mutable.Queue[RDD[T]]
sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
ssc.queueStream(sQueue)
@@ -556,8 +560,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam T Type of objects in the RDD
*/
def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val sQueue = new scala.collection.mutable.Queue[RDD[T]]
sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
ssc.queueStream(sQueue, oneAtATime)
@@ -577,8 +581,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
queue: java.util.Queue[JavaRDD[T]],
oneAtATime: Boolean,
defaultRDD: JavaRDD[T]): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val sQueue = new scala.collection.mutable.Queue[RDD[T]]
sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index fea0573b77..39e25239bf 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -26,14 +26,16 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import scala.collection.mutable.{HashSet, HashMap}
+import scala.reflect.ClassTag
+
import java.io.{ObjectInputStream, IOException}
private[streaming]
-class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest](
+class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag](
@transient ssc_ : StreamingContext,
directory: String,
filter: Path => Boolean = FileInputDStream.defaultFilter,
- newFilesOnly: Boolean = true)
+ newFilesOnly: Boolean = true)
extends InputDStream[(K, V)](ssc_) {
protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
@@ -54,7 +56,7 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
}
logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly)
}
-
+
override def stop() { }
/**
@@ -100,7 +102,7 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
latestModTimeFiles += path.toString
logDebug("Accepted " + path)
return true
- }
+ }
}
}
logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime)
@@ -195,5 +197,3 @@ private[streaming]
object FileInputDStream {
def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".")
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
index 18de772946..b72ab90e60 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
@@ -22,6 +22,7 @@ import java.io.{ObjectInput, ObjectOutput, Externalizable}
import java.nio.ByteBuffer
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.flume.source.avro.AvroSourceProtocol
import org.apache.flume.source.avro.AvroFlumeEvent
@@ -34,7 +35,7 @@ import org.apache.spark.util.Utils
import org.apache.spark.storage.StorageLevel
private[streaming]
-class FlumeInputDStream[T: ClassManifest](
+class FlumeInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
index 674b27118c..f01e67fe13 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
@@ -19,6 +19,8 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream}
+import scala.reflect.ClassTag
+
/**
* This is the abstract base class for all input streams. This class provides to methods
* start() and stop() which called by the scheduler to start and stop receiving data/
@@ -30,7 +32,7 @@ import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream}
* that requires running a receiver on the worker nodes, use NetworkInputDStream
* as the parent class.
*/
-abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext)
+abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext)
extends DStream[T](ssc_) {
var lastValidTime: Time = null
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
index 51e913675d..96134868cc 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
@@ -35,18 +35,18 @@ import org.I0Itec.zkclient._
import scala.collection.Map
import scala.collection.mutable.HashMap
import scala.collection.JavaConversions._
-
+import scala.reflect.ClassTag
/**
* Input stream that pulls messages from a Kafka Broker.
- *
+ *
* @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
* @param storageLevel RDD storage level.
*/
private[streaming]
-class KafkaInputDStream[T: ClassManifest, D <: Decoder[_]: Manifest](
+class KafkaInputDStream[T: ClassTag, D <: Decoder[_]: Manifest](
@transient ssc_ : StreamingContext,
kafkaParams: Map[String, String],
topics: Map[String, Int],
@@ -61,7 +61,7 @@ class KafkaInputDStream[T: ClassManifest, D <: Decoder[_]: Manifest](
}
private[streaming]
-class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
+class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest](
kafkaParams: Map[String, String],
topics: Map[String, Int],
storageLevel: StorageLevel
@@ -102,7 +102,7 @@ class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
}
// Create Threads for each Topic/Message Stream we are listening
- val decoder = manifest[D].erasure.newInstance.asInstanceOf[Decoder[T]]
+ val decoder = manifest[D].runtimeClass.newInstance.asInstanceOf[Decoder[T]]
val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder)
// Start the messages handler for each partition
@@ -112,7 +112,7 @@ class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
}
// Handles Kafka Messages
- private class MessageHandler[T: ClassManifest](stream: KafkaStream[T]) extends Runnable {
+ private class MessageHandler[T: ClassTag](stream: KafkaStream[T]) extends Runnable {
def run() {
logInfo("Starting MessageHandler.")
for (msgAndMetadata <- stream) {
@@ -135,7 +135,7 @@ class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest](
zk.deleteRecursive(dir)
zk.close()
} catch {
- case _ => // swallow
+ case _ : Throwable => // swallow
}
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index 31f9891560..394a39fbb0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
@@ -21,11 +21,12 @@ import java.util.concurrent.ArrayBlockingQueue
import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
import akka.actor.{Props, Actor}
import akka.pattern.ask
-import akka.dispatch.Await
-import akka.util.duration._
import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
import org.apache.spark.streaming._
@@ -42,7 +43,7 @@ import org.apache.spark.storage.StorageLevel
* @param ssc_ Streaming context that will execute this input stream
* @tparam T Class type of the object of this stream
*/
-abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : StreamingContext)
+abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext)
extends InputDStream[T](ssc_) {
// This is an unique identifier that is used to match the network receiver with the
@@ -84,7 +85,7 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe
* Abstract class of a receiver that can be run on worker nodes to receive external data. See
* [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation.
*/
-abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging {
+abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging {
initLogging()
@@ -176,7 +177,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
logInfo("Attempting to register with tracker")
val ip = System.getProperty("spark.driver.host", "localhost")
val port = System.getProperty("spark.driver.port", "7077").toInt
- val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
+ val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
val tracker = env.actorSystem.actorFor(url)
val timeout = 5.seconds
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
index c91f12ecd7..a4746f06ad 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
@@ -21,6 +21,8 @@ import org.apache.spark.Logging
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
+import scala.reflect.ClassTag
+
import java.net.InetSocketAddress
import java.nio.ByteBuffer
import java.nio.channels.{ReadableByteChannel, SocketChannel}
@@ -35,7 +37,7 @@ import java.util.concurrent.ArrayBlockingQueue
* in the format that the system is configured with.
*/
private[streaming]
-class RawInputDStream[T: ClassManifest](
+class RawInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
index b88a4db959..db56345ca8 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -28,8 +28,11 @@ import org.apache.spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
private[streaming]
-class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
+class ReducedWindowedDStream[K: ClassTag, V: ClassTag](
parent: DStream[(K, V)],
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
@@ -49,7 +52,7 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
"must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")"
)
- // Reduce each batch of data using reduceByKey which will be further reduced by window
+ // Reduce each batch of data using reduceByKey which will be further reduced by window
// by ReducedWindowedDStream
val reducedStream = parent.reduceByKey(reduceFunc, partitioner)
@@ -170,5 +173,3 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
}
}
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
index e2539c7396..2cdd13f205 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
@@ -21,11 +21,13 @@ import org.apache.spark.streaming.StreamingContext
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.NextIterator
+import scala.reflect.ClassTag
+
import java.io._
import java.net.Socket
private[streaming]
-class SocketInputDStream[T: ClassManifest](
+class SocketInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
@@ -39,7 +41,7 @@ class SocketInputDStream[T: ClassManifest](
}
private[streaming]
-class SocketReceiver[T: ClassManifest](
+class SocketReceiver[T: ClassTag](
host: String,
port: Int,
bytesToObjects: InputStream => Iterator[T],
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
index 362a6bf4cc..e0ff3ccba4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
@@ -23,8 +23,10 @@ import org.apache.spark.SparkContext._
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Duration, Time, DStream}
+import scala.reflect.ClassTag
+
private[streaming]
-class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
+class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
parent: DStream[(K, V)],
updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
partitioner: Partitioner,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
index c696bb70a8..0d84ec84f2 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
@@ -22,8 +22,11 @@ import org.apache.spark.rdd.RDD
import collection.mutable.ArrayBuffer
import org.apache.spark.rdd.UnionRDD
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
private[streaming]
-class UnionDStream[T: ClassManifest](parents: Array[DStream[T]])
+class UnionDStream[T: ClassTag](parents: Array[DStream[T]])
extends DStream[T](parents.head.ssc) {
if (parents.length == 0) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
index 3c57294269..73d959331a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
@@ -22,8 +22,10 @@ import org.apache.spark.rdd.UnionRDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
+import scala.reflect.ClassTag
+
private[streaming]
-class WindowedDStream[T: ClassManifest](
+class WindowedDStream[T: ClassTag](
parent: DStream[T],
_windowDuration: Duration,
_slideDuration: Duration)
@@ -52,6 +54,3 @@ class WindowedDStream[T: ClassManifest](
Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
}
}
-
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
index 4b5d8c467e..ee087a1cf0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
@@ -20,6 +20,10 @@ package org.apache.spark.streaming.receivers
import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }
import akka.actor.{ actorRef2Scala, ActorRef }
import akka.actor.{ PossiblyHarmful, OneForOneStrategy }
+import akka.actor.SupervisorStrategy._
+
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.NetworkReceiver
@@ -28,12 +32,9 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.ArrayBuffer
-/** A helper with set of defaults for supervisor strategy **/
+/** A helper with set of defaults for supervisor strategy */
object ReceiverSupervisorStrategy {
- import akka.util.duration._
- import akka.actor.SupervisorStrategy._
-
val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange =
15 millis) {
case _: RuntimeException ⇒ Restart
@@ -48,10 +49,10 @@ object ReceiverSupervisorStrategy {
* Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html
*
* @example {{{
- * class MyActor extends Actor with Receiver{
- * def receive {
- * case anything :String ⇒ pushBlock(anything)
- * }
+ * class MyActor extends Actor with Receiver{
+ * def receive {
+ * case anything :String => pushBlock(anything)
+ * }
* }
* //Can be plugged in actorStream as follows
* ssc.actorStream[String](Props(new MyActor),"MyActorReceiver")
@@ -65,11 +66,11 @@ object ReceiverSupervisorStrategy {
*
*/
trait Receiver { self: Actor ⇒
- def pushBlock[T: ClassManifest](iter: Iterator[T]) {
+ def pushBlock[T: ClassTag](iter: Iterator[T]) {
context.parent ! Data(iter)
}
- def pushBlock[T: ClassManifest](data: T) {
+ def pushBlock[T: ClassTag](data: T) {
context.parent ! Data(data)
}
@@ -83,8 +84,8 @@ case class Statistics(numberOfMsgs: Int,
numberOfHiccups: Int,
otherInfo: String)
-/** Case class to receive data sent by child actors **/
-private[streaming] case class Data[T: ClassManifest](data: T)
+/** Case class to receive data sent by child actors */
+private[streaming] case class Data[T: ClassTag](data: T)
/**
* Provides Actors as receivers for receiving stream.
@@ -95,19 +96,19 @@ private[streaming] case class Data[T: ClassManifest](data: T)
* his own Actor to run as receiver for Spark Streaming input source.
*
* This starts a supervisor actor which starts workers and also provides
- * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance].
- *
+ * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance].
+ *
* Here's a way to start more supervisor/workers as its children.
*
* @example {{{
- * context.parent ! Props(new Supervisor)
+ * context.parent ! Props(new Supervisor)
* }}} OR {{{
* context.parent ! Props(new Worker,"Worker")
* }}}
*
*
*/
-private[streaming] class ActorReceiver[T: ClassManifest](
+private[streaming] class ActorReceiver[T: ClassTag](
props: Props,
name: String,
storageLevel: StorageLevel,
@@ -120,7 +121,7 @@ private[streaming] class ActorReceiver[T: ClassManifest](
protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor),
"Supervisor" + streamId)
- private class Supervisor extends Actor {
+ class Supervisor extends Actor {
override val supervisorStrategy = receiverSupervisorStrategy
val worker = context.actorOf(props, name)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
index 043bb8c8bf..ce8c56fa8a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
@@ -18,19 +18,22 @@
package org.apache.spark.streaming.receivers
import akka.actor.Actor
+import akka.util.ByteString
import akka.zeromq._
import org.apache.spark.Logging
+import scala.reflect.ClassTag
+
/**
* A receiver to subscribe to ZeroMQ stream.
*/
-private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String,
+private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String,
subscribe: Subscribe,
- bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T])
+ bytesToObjects: Seq[ByteString] ⇒ Iterator[T])
extends Actor with Receiver with Logging {
- override def preStart() = context.system.newSocket(SocketType.Sub, Listener(self),
+ override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self),
Connect(publisherUrl), subscribe)
def receive: Receive = {
@@ -38,10 +41,10 @@ private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String,
case Connecting ⇒ logInfo("connecting ...")
case m: ZMQMessage ⇒
- logDebug("Received message for:" + m.firstFrameAsString)
+ logDebug("Received message for:" + m.frame(0))
//We ignore first frame for processing as it is the topic
- val bytes = m.frames.tail.map(_.payload)
+ val bytes = m.frames.tail
pushBlock(bytesToObjects(bytes))
case Closed ⇒ logInfo("received closed ")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
index 6977957126..4a3993e3e3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
@@ -25,6 +25,7 @@ import StreamingContext._
import scala.util.Random
import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import scala.reflect.ClassTag
import java.io.{File, ObjectInputStream, IOException}
import java.util.UUID
@@ -120,7 +121,7 @@ object MasterFailureTest extends Logging {
* Tests stream operation with multiple master failures, and verifies whether the
* final set of output values is as expected or not.
*/
- def testOperation[T: ClassManifest](
+ def testOperation[T: ClassTag](
directory: String,
batchDuration: Duration,
input: Seq[String],
@@ -158,7 +159,7 @@ object MasterFailureTest extends Logging {
* and batch duration. Returns the streaming context and the directory to which
* files should be written for testing.
*/
- private def setupStreams[T: ClassManifest](
+ private def setupStreams[T: ClassTag](
directory: String,
batchDuration: Duration,
operation: DStream[String] => DStream[T]
@@ -192,7 +193,7 @@ object MasterFailureTest extends Logging {
* Repeatedly starts and kills the streaming context until timed out or
* the last expected output is generated. Finally, return
*/
- private def runStreams[T: ClassManifest](
+ private def runStreams[T: ClassTag](
ssc_ : StreamingContext,
lastExpectedOutput: T,
maxTimeToRun: Long
@@ -274,7 +275,7 @@ object MasterFailureTest extends Logging {
* duplicate batch outputs of values from the `output`. As a result, the
* expected output should not have consecutive batches with the same values as output.
*/
- private def verifyOutput[T: ClassManifest](output: Seq[T], expectedOutput: Seq[T]) {
+ private def verifyOutput[T: ClassTag](output: Seq[T], expectedOutput: Seq[T]) {
// Verify whether expected outputs do not consecutive batches with same output
for (i <- 0 until expectedOutput.size - 1) {
assert(expectedOutput(i) != expectedOutput(i+1),
@@ -305,7 +306,7 @@ object MasterFailureTest extends Logging {
* ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
*/
private[streaming]
-class TestOutputStream[T: ClassManifest](
+class TestOutputStream[T: ClassTag](
parent: DStream[T],
val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]]
) extends ForEachDStream[T](
@@ -380,24 +381,24 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString)
FileUtils.writeStringToFile(localFile, input(i).toString + "\n")
var tries = 0
- var done = false
- while (!done && tries < maxTries) {
- tries += 1
- try {
- // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
- fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
- fs.rename(tempHadoopFile, hadoopFile)
- done = true
- } catch {
- case ioe: IOException => {
- fs = testDir.getFileSystem(new Configuration())
- logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe)
- }
- }
+ var done = false
+ while (!done && tries < maxTries) {
+ tries += 1
+ try {
+ // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
+ fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
+ fs.rename(tempHadoopFile, hadoopFile)
+ done = true
+ } catch {
+ case ioe: IOException => {
+ fs = testDir.getFileSystem(new Configuration())
+ logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe)
+ }
+ }
}
- if (!done)
+ if (!done)
logError("Could not generate file " + hadoopFile)
- else
+ else
logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis)
Thread.sleep(interval)
localFile.delete()
@@ -411,5 +412,3 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
}
}
}
-
-
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index c0d729ff87..783b8dea31 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -48,7 +48,7 @@ import java.util.*;
import akka.actor.Props;
import akka.zeromq.Subscribe;
-
+import akka.util.ByteString;
// The test suite itself is Serializable so that anonymous Function implementations can be
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
index 8a6604904d..d5cdad4998 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
@@ -17,7 +17,9 @@
package org.apache.spark.streaming
-import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import scala.reflect.ClassTag
+
import java.util.{List => JList}
import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext}
import org.apache.spark.streaming._
@@ -31,15 +33,15 @@ trait JavaTestBase extends TestSuiteBase {
/**
* Create a [[org.apache.spark.streaming.TestInputStream]] and attach it to the supplied context.
* The stream will be derived from the supplied lists of Java objects.
- **/
+ */
def attachTestInputStream[T](
ssc: JavaStreamingContext,
data: JList[JList[T]],
numPartitions: Int) = {
val seqData = data.map(Seq(_:_*))
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val dstream = new TestInputStream[T](ssc.ssc, seqData, numPartitions)
ssc.ssc.registerInputStream(dstream)
new JavaDStream[T](dstream)
@@ -52,8 +54,8 @@ trait JavaTestBase extends TestSuiteBase {
def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]](
dstream: JavaDStreamLike[T, This, R]) =
{
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val ostream = new TestOutputStream(dstream.dstream,
new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]])
dstream.dstream.ssc.registerOutputStream(ostream)
@@ -66,8 +68,8 @@ trait JavaTestBase extends TestSuiteBase {
*/
def runStreams[V](
ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = {
- implicit val cm: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cm: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput)
val out = new ArrayList[JList[V]]()
res.map(entry => out.append(new ArrayList[V](entry)))
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index a327de80b3..07de51bebb 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -20,14 +20,20 @@ package org.apache.spark.streaming
import dstream.FileInputDStream
import org.apache.spark.streaming.StreamingContext._
import java.io.File
-import runtime.RichInt
-import org.scalatest.BeforeAndAfter
+
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
import org.apache.commons.io.FileUtils
-import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
-import util.{Clock, ManualClock}
-import scala.util.Random
+import org.scalatest.BeforeAndAfter
+
import com.google.common.io.Files
+import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
+import org.apache.spark.streaming.dstream.FileInputDStream
+import org.apache.spark.streaming.util.ManualClock
+
+
/**
* This test suites tests the checkpointing functionality of DStreams -
@@ -74,13 +80,13 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// Setup the streams
val input = (1 to 10).map(_ => Seq("a")).toSeq
val operation = (st: DStream[String]) => {
- val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
- Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+ Some((values.foldLeft(0)(_ + _) + state.getOrElse(0)))
}
st.map(x => (x, 1))
- .updateStateByKey[RichInt](updateFunc)
+ .updateStateByKey(updateFunc)
.checkpoint(stateStreamCheckpointInterval)
- .map(t => (t._1, t._2.self))
+ .map(t => (t._1, t._2))
}
var ssc = setupStreams(input, operation)
var stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
@@ -180,13 +186,13 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
val input = (1 to 10).map(_ => Seq("a")).toSeq
val output = (1 to 10).map(x => Seq(("a", x))).toSeq
val operation = (st: DStream[String]) => {
- val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
- Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+ Some((values.foldLeft(0)(_ + _) + state.getOrElse(0)))
}
st.map(x => (x, 1))
- .updateStateByKey[RichInt](updateFunc)
+ .updateStateByKey(updateFunc)
.checkpoint(batchDuration * 2)
- .map(t => (t._1, t._2.self))
+ .map(t => (t._1, t._2))
}
testCheckpointedOperation(input, operation, output, 7)
}
@@ -312,7 +318,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
* NOTE: This takes into consideration that the last batch processed before
* master failure will be re-processed after restart/recovery.
*/
- def testCheckpointedOperation[U: ClassManifest, V: ClassManifest](
+ def testCheckpointedOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
@@ -356,7 +362,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
* Advances the manual clock on the streaming scheduler by given number of batches.
* It also waits for the expected amount of time for each batch.
*/
- def advanceTimeWithRealDelay[V: ClassManifest](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = {
+ def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = {
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
logInfo("Manual clock before advancing = " + clock.time)
for (i <- 1 to numBatches.toInt) {
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 37dd9c4cc6..c91f9ba46d 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -20,8 +20,9 @@ package org.apache.spark.streaming
import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream}
import org.apache.spark.streaming.util.ManualClock
-import collection.mutable.ArrayBuffer
-import collection.mutable.SynchronizedBuffer
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.SynchronizedBuffer
+import scala.reflect.ClassTag
import java.io.{ObjectInputStream, IOException}
@@ -35,7 +36,7 @@ import org.apache.spark.rdd.RDD
* replayable, reliable message queue like Kafka. It requires a sequence as input, and
* returns the i_th element at the i_th batch unde manual clock.
*/
-class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
+class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
extends InputDStream[T](ssc_) {
def start() {}
@@ -61,7 +62,7 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[
* This is a output stream just for the testsuites. All the output is collected into a
* ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
*/
-class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBuffer[Seq[T]])
+class TestOutputStream[T: ClassTag](parent: DStream[T], val output: ArrayBuffer[Seq[T]])
extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.collect()
output += collected
@@ -106,7 +107,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Set up required DStreams to test the DStream operation using the two sequences
* of input collections.
*/
- def setupStreams[U: ClassManifest, V: ClassManifest](
+ def setupStreams[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V]
): StreamingContext = {
@@ -130,7 +131,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Set up required DStreams to test the binary operation using the sequence
* of input collections.
*/
- def setupStreams[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ def setupStreams[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W]
@@ -158,7 +159,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* returns the collected output. It will wait until `numExpectedOutput` number of
* output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
*/
- def runStreams[V: ClassManifest](
+ def runStreams[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int
@@ -214,7 +215,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* is same as the expected output values, by comparing the output
* collections either as lists (order matters) or sets (order does not matter)
*/
- def verifyOutput[V: ClassManifest](
+ def verifyOutput[V: ClassTag](
output: Seq[Seq[V]],
expectedOutput: Seq[Seq[V]],
useSet: Boolean
@@ -244,7 +245,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Test unary DStream operation with a list of inputs, with number of
* batches to run same as the number of expected output values
*/
- def testOperation[U: ClassManifest, V: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
@@ -262,7 +263,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* @param useSet Compare the output values with the expected output values
* as sets (order matters) or as lists (order does not matter)
*/
- def testOperation[U: ClassManifest, V: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
@@ -279,7 +280,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Test binary DStream operation with two lists of inputs, with number of
* batches to run same as the number of expected output values
*/
- def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W],
@@ -299,7 +300,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* @param useSet Compare the output values with the expected output values
* as sets (order matters) or as lists (order does not matter)
*/
- def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W],
diff --git a/tools/pom.xml b/tools/pom.xml
index 29f0014128..2456457722 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -43,14 +43,14 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_${scala-short.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 427fcdf545..7f852d3540 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -64,8 +64,8 @@
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala-short.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala-short.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 6d6ef149cc..d222f412a0 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -209,7 +209,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
else {
// deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter)
val workerId = workerIdCounter.incrementAndGet().toString
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
StandaloneSchedulerBackend.ACTOR_NAME)