aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorNick Pentreath <nick.pentreath@gmail.com>2013-12-15 08:34:05 +0200
committerNick Pentreath <nick.pentreath@gmail.com>2013-12-15 08:34:05 +0200
commitd36ee3b159dff04b3a3222d6c55ee27ba93cd074 (patch)
tree40c2010a53ddc13307e1645bed5de25c2ae2c133
parent8cdfb08c47131ce3438e5faf1222af2039424324 (diff)
parent7db916596144a8b9e55fabf859d921a093d3c59f (diff)
downloadspark-d36ee3b159dff04b3a3222d6c55ee27ba93cd074.tar.gz
spark-d36ee3b159dff04b3a3222d6c55ee27ba93cd074.tar.bz2
spark-d36ee3b159dff04b3a3222d6c55ee27ba93cd074.zip
Merge remote-tracking branch 'upstream/master'
-rw-r--r--.gitignore1
-rw-r--r--README.md5
-rw-r--r--assembly/pom.xml16
-rw-r--r--bagel/pom.xml12
-rw-r--r--bin/compute-classpath.cmd2
-rwxr-xr-xbin/compute-classpath.sh2
-rw-r--r--core/pom.xml29
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileClient.java2
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileServer.java1
-rw-r--r--core/src/main/scala/org/apache/spark/MapOutputTracker.scala29
-rw-r--r--core/src/main/scala/org/apache/spark/Partitioner.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala77
-rw-r--r--core/src/main/scala/org/apache/spark/SparkEnv.scala13
-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.scala77
-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.scala32
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala60
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java1
-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/Function3.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.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala3
-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.scala48
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala69
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala6
-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.scala48
-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.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/executor/Executor.scala2
-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/AsyncRDDActions.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/BlockRDD.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.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala33
-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.scala64
-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/ShuffledRDD.scala6
-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.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala2
-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/TaskResultGetter.scala4
-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.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala2
-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.scala79
-rw-r--r--core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala68
-rw-r--r--core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala3
-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.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala4
-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.scala3
-rw-r--r--core/src/test/scala/org/apache/spark/DriverSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala14
-rw-r--r--core/src/test/scala/org/apache/spark/UnpersistSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala8
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/ui/UISuite.scala1
-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--docs/configuration.md23
-rw-r--r--docs/running-on-yarn.md12
-rw-r--r--examples/pom.xml26
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java2
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaPageRank.java3
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaWordCount.java2
-rw-r--r--examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java1
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala7
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala8
-rw-r--r--mllib/pom.xml12
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala2
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java4
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java2
-rw-r--r--pom.xml169
-rw-r--r--project/SparkBuild.scala145
-rwxr-xr-xpyspark2
-rw-r--r--pyspark2.cmd2
-rw-r--r--python/pyspark/rdd.py4
-rw-r--r--repl-bin/pom.xml8
-rwxr-xr-xrepl-bin/src/deb/bin/run2
-rw-r--r--repl/lib/scala-jline.jarbin158463 -> 0 bytes
-rw-r--r--repl/pom.xml18
-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.scala109
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala944
-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.scala1681
-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.scala178
-rwxr-xr-xrun-example2
-rw-r--r--run-example2.cmd2
-rwxr-xr-xspark-class4
-rw-r--r--streaming/pom.xml22
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStream.scala37
-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.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala63
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala44
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala8
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala79
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala93
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala108
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala3
-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/FilteredDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala3
-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/ForEachDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.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.scala23
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala13
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala4
-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/ShuffledDStream.scala3
-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/TransformedDStream.scala3
-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.java88
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala22
-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.scala29
-rw-r--r--tools/pom.xml12
-rw-r--r--tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala4
-rw-r--r--yarn/pom.xml10
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala17
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala4
197 files changed, 3470 insertions, 2905 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 7faba27420..80bbe311a9 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 09df8c1fd7..c2cda41c6d 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-assembly_2.9.3</artifactId>
+ <artifactId>spark-assembly_2.10</artifactId>
<name>Spark Project Assembly</name>
<url>http://spark.incubator.apache.org/</url>
@@ -41,27 +41,27 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_2.9.3</artifactId>
+ <artifactId>spark-repl_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -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-2.10/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
@@ -128,7 +128,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn_2.9.3</artifactId>
+ <artifactId>spark-yarn_2.10</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
diff --git a/bagel/pom.xml b/bagel/pom.xml
index 0e552c880f..0f550d70d6 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Bagel</name>
<url>http://spark.incubator.apache.org/</url>
@@ -34,7 +34,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -43,18 +43,18 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_2.10</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-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/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 c16afd6b36..40555089fc 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 38f4be1280..3fe48fd2af 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Core</name>
<url>http://spark.incubator.apache.org/</url>
@@ -86,7 +86,7 @@
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>chill_2.9.3</artifactId>
+ <artifactId>chill_2.10</artifactId>
<version>0.3.1</version>
</dependency>
<dependency>
@@ -96,19 +96,15 @@
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-actor</artifactId>
+ <artifactId>akka-actor_2.10</artifactId>
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-remote</artifactId>
+ <artifactId>akka-remote_2.10</artifactId>
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-slf4j</artifactId>
- </dependency>
- <dependency>
- <groupId>org.scala-lang</groupId>
- <artifactId>scalap</artifactId>
+ <artifactId>akka-slf4j_2.10</artifactId>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
@@ -116,7 +112,7 @@
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
- <artifactId>lift-json_2.9.2</artifactId>
+ <artifactId>lift-json_2.10</artifactId>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
@@ -164,13 +160,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_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -190,8 +191,8 @@
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
index 20a7a3aa8c..edd0fc56f8 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
@@ -19,8 +19,6 @@ package org.apache.spark.network.netty;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelOption;
import io.netty.channel.oio.OioEventLoopGroup;
import io.netty.channel.socket.oio.OioSocketChannel;
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
index 666432474d..a99af348ce 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
@@ -20,7 +20,6 @@ package org.apache.spark.network.netty;
import java.net.InetSocketAddress;
import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelOption;
import io.netty.channel.oio.OioEventLoopGroup;
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index b4d0b7017c..10fae5af9f 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -21,12 +21,11 @@ import java.io._
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
import scala.collection.mutable.HashSet
+import scala.concurrent.Await
+import scala.concurrent.duration._
import akka.actor._
-import akka.dispatch._
import akka.pattern.ask
-import akka.util.Duration
-
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
@@ -55,9 +54,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
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 = _
+ var trackerActor: Either[ActorRef, ActorSelection] = _
protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
@@ -73,8 +72,18 @@ private[spark] class MapOutputTracker extends Logging {
// throw a SparkException if this fails.
private def askTracker(message: Any): Any = {
try {
- val future = trackerActor.ask(message)(timeout)
- return Await.result(future, timeout)
+ /*
+ The difference between ActorRef and ActorSelection is well explained here:
+ http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#Use_actorSelection_instead_of_actorFor
+ In spark a map output tracker can be either started on Driver where it is created which
+ is an ActorRef or it can be on executor from where it is looked up which is an
+ actorSelection.
+ */
+ val future = trackerActor match {
+ case Left(a: ActorRef) => a.ask(message)(timeout)
+ case Right(b: ActorSelection) => b.ask(message)(timeout)
+ }
+ Await.result(future, timeout)
} catch {
case e: Exception =>
throw new SparkException("Error communicating with MapOutputTracker", e)
@@ -117,7 +126,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)
@@ -144,7 +153,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)
@@ -312,7 +321,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..bcec41c439 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -17,8 +17,10 @@
package org.apache.spark
-import org.apache.spark.util.Utils
+import scala.reflect.ClassTag
+
import org.apache.spark.rdd.RDD
+import org.apache.spark.util.Utils
/**
* An object that defines how the elements in a key-value pair RDD are partitioned by key.
@@ -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 66006bf212..a0f794edfd 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -26,6 +26,7 @@ import scala.collection.Map
import scala.collection.generic.Growable
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
+import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -267,19 +268,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)
}
@@ -332,7 +333,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,
* {{{
@@ -340,17 +341,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,
* {{{
@@ -358,17 +359,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]])
}
/**
@@ -426,11 +427,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()
@@ -449,7 +450,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] = {
@@ -458,17 +459,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
@@ -711,7 +712,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],
@@ -732,7 +733,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],
@@ -747,7 +748,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],
@@ -759,21 +760,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)
@@ -784,7 +785,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)
@@ -930,16 +931,16 @@ 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 rddToAsyncRDDActions[T: ClassManifest](rdd: RDD[T]) = new AsyncRDDActions(rdd)
+ implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(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)
@@ -964,16 +965,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]))
}
@@ -992,7 +993,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
@@ -1147,12 +1148,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 ff2df8fb6a..826f5c2d8c 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -20,7 +20,7 @@ package org.apache.spark
import collection.mutable
import serializer.Serializer
-import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
+import akka.actor._
import akka.remote.RemoteActorRefProvider
import org.apache.spark.broadcast.BroadcastManager
@@ -74,7 +74,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 = {
@@ -151,17 +152,17 @@ object SparkEnv extends Logging {
val closureSerializer = serializerManager.get(
System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"))
- def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
+ def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = {
if (isDriver) {
logInfo("Registering " + name)
- actorSystem.actorOf(Props(newActor), name = name)
+ Left(actorSystem.actorOf(Props(newActor), name = name))
} else {
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)
+ Right(actorSystem.actorSelection(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 9f02a9b7d3..da30cf619a 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,12 +17,15 @@
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
@@ -30,7 +33,7 @@ import scala.collection.JavaConverters._
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))
@@ -44,7 +47,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.
*/
@@ -108,7 +111,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 2142fd7327..363667fa86 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.
*/
@@ -138,14 +139,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.
@@ -157,8 +158,7 @@ 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,
@@ -195,14 +195,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.
*/
@@ -258,7 +258,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.
*/
@@ -315,15 +315,14 @@ 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)))
}
@@ -414,8 +413,7 @@ 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))
}
@@ -426,8 +424,7 @@ 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))
}
@@ -592,6 +589,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
}
/**
+ * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
+ * `collect` or `save` on the resulting RDD will return or output an ordered list of records
+ * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
+ * order of the keys).
+ */
+ def sortByKey(comp: Comparator[K], ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = {
+ class KeyOrdering(val a: K) extends Ordered[K] {
+ override def compare(b: K) = comp.compare(a, b)
+ }
+ implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x)
+ fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions))
+ }
+
+ /**
* Return an RDD with the keys of each tuple.
*/
def keys(): JavaRDD[K] = JavaRDD.fromRDD[K](rdd.map(_._1))
@@ -603,22 +614,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],
- Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2)))
+ 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
@@ -626,10 +637,8 @@ object JavaPairRDD {
/** Convert a JavaRDD of key-value pairs to JavaPairRDD. */
def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[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 JavaPairRDD[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 3b359a8fd6..c47657f512 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)
@@ -127,8 +129,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..9e912d3adb 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]
@@ -71,7 +72,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* Return a new RDD by applying a function to each partition of this RDD, while tracking the index
* of the original partition.
*/
- def mapPartitionsWithIndex[R: ClassManifest](
+ def mapPartitionsWithIndex[R: ClassTag](
f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]],
preservesPartitioning: Boolean = false): JavaRDD[R] =
new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))),
@@ -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,16 @@ 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 +177,9 @@ 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 +207,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 +222,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 +354,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..acf328aa6a 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,7 @@ 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 +94,8 @@ 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 +130,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 +151,7 @@ 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 +163,7 @@ 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 +179,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 +195,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 +208,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 +220,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 +236,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 +250,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 +266,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 +401,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/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
index c9cbce5624..2090efd3b9 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
@@ -17,7 +17,6 @@
package org.apache.spark.api.java;
-import java.util.Arrays;
import java.util.ArrayList;
import java.util.List;
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 2dfda8b09a..bdb01f7670 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,9 +17,11 @@
package org.apache.spark.api.java.function
+import scala.reflect.ClassTag
+
/**
* A function that returns zero or more output records from each input record.
*/
abstract class FlatMapFunction[T, R] extends Function[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 528e1c0a7c..aae1349c5e 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,9 +17,11 @@
package org.apache.spark.api.java.function
+import scala.reflect.ClassTag
+
/**
* A function that takes two inputs and returns zero or more output records.
*/
abstract class FlatMapFunction2[A, B, C] extends Function2[A, 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 ce368ee01b..537439ef53 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 java.io.Serializable;
@@ -29,8 +29,8 @@ import java.io.Serializable;
* when mapping RDDs of other types.
*/
public abstract class Function<T, R> extends WrappedFunction1<T, R> implements Serializable {
- public ClassManifest<R> returnType() {
- return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<R> returnType() {
+ return 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 44ad559d48..a2d1214fb4 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 java.io.Serializable;
@@ -28,8 +28,8 @@ import java.io.Serializable;
public abstract class Function2<T1, T2, R> extends WrappedFunction2<T1, T2, R>
implements Serializable {
- 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/Function3.java b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
index ac6178924a..fb1deceab5 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function3.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;
@@ -29,8 +29,8 @@ import java.io.Serializable;
public abstract class Function3<T1, T2, T3, R> extends WrappedFunction3<T1, T2, T3, R>
implements Serializable {
- 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 6d76a8f970..ca485b3cc2 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 java.io.Serializable;
@@ -33,11 +33,11 @@ public abstract class PairFlatMapFunction<T, K, V>
extends WrappedFunction1<T, Iterable<Tuple2<K, V>>>
implements Serializable {
- 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 ede7ceefb5..cbe2306026 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 java.io.Serializable;
@@ -31,11 +31,11 @@ import java.io.Serializable;
public abstract class PairFunction<T, K, V> extends WrappedFunction1<T, Tuple2<K, V>>
implements Serializable {
- 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 132e4fb0d2..a659cc06c2 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
@@ -29,8 +30,7 @@ import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.util.Utils
-
-private[spark] class PythonRDD[T: ClassManifest](
+private[spark] class PythonRDD[T: ClassTag](
parent: RDD[T],
command: Array[Byte],
envVars: JMap[String, String],
@@ -148,7 +148,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
}
}
@@ -200,7 +200,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))
}
@@ -236,7 +236,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/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 67d45723ba..f291266fcf 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -64,7 +64,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
startDaemon()
new Socket(daemonHost, daemonPort)
}
- case e => throw e
+ case e: Throwable => throw e
}
}
}
@@ -198,7 +198,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
}
}.start()
} catch {
- case e => {
+ case e: Throwable => {
stopDaemon()
throw e
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
index fcfea96ad6..37dfa7fec0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
@@ -17,8 +17,7 @@
package org.apache.spark.deploy
-private[spark] object ExecutorState
- extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") {
+private[spark] object ExecutorState extends Enumeration {
val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value
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 a724900943..59d12a3e6f 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(): Array[String] = {
logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
@@ -61,10 +61,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 77422f61ec..4d95efa73a 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,17 +19,15 @@ 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.util.duration._
-import akka.remote.RemoteClientDisconnected
-import akka.remote.RemoteClientLifeCycleEvent
-import akka.remote.RemoteClientShutdown
-import akka.dispatch.Await
-
-import org.apache.spark.Logging
+import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent}
+
+import org.apache.spark.{SparkException, Logging}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
@@ -51,18 +49,19 @@ private[spark] class Client(
val REGISTRATION_TIMEOUT = 20.seconds
val REGISTRATION_RETRIES = 3
+ var masterAddress: Address = null
var actor: ActorRef = null
var appId: String = null
var registered = false
var activeMasterUrl: String = null
class ClientActor extends Actor with Logging {
- var master: ActorRef = null
- var masterAddress: Address = null
+ var master: ActorSelection = null
var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times
var alreadyDead = false // To avoid calling listener.dead() multiple times
override def preStart() {
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
try {
registerWithMaster()
} catch {
@@ -76,7 +75,7 @@ private[spark] class Client(
def tryRegisterAllMasters() {
for (masterUrl <- masterUrls) {
logInfo("Connecting to master " + masterUrl + "...")
- val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+ val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
actor ! RegisterApplication(appDescription)
}
}
@@ -84,6 +83,7 @@ private[spark] class Client(
def registerWithMaster() {
tryRegisterAllMasters()
+ import context.dispatcher
var retries = 0
lazy val retryTimer: Cancellable =
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
@@ -102,10 +102,13 @@ private[spark] class Client(
def changeMaster(url: String) {
activeMasterUrl = url
- master = context.actorFor(Master.toAkkaUrl(url))
- masterAddress = master.path.address
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
- context.watch(master) // Doesn't work with remote actors, but useful for testing
+ master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+ masterAddress = activeMasterUrl match {
+ case Master.sparkUrlRegex(host, port) =>
+ Address("akka.tcp", Master.systemName, host, port.toInt)
+ case x =>
+ throw new SparkException("Invalid spark URL: " + x)
+ }
}
override def receive = {
@@ -135,21 +138,12 @@ private[spark] class Client(
case MasterChanged(masterUrl, masterWebUiUrl) =>
logInfo("Master has changed, new master is at " + masterUrl)
- context.unwatch(master)
changeMaster(masterUrl)
alreadyDisconnected = false
sender ! MasterChangeAcknowledged(appId)
- case Terminated(actor_) if actor_ == master =>
- logWarning("Connection to master failed; waiting for master to reconnect...")
- markDisconnected()
-
- case RemoteClientDisconnected(transport, address) if address == masterAddress =>
- logWarning("Connection to master failed; waiting for master to reconnect...")
- markDisconnected()
-
- case RemoteClientShutdown(transport, address) if address == masterAddress =>
- logWarning("Connection to master failed; waiting for master to reconnect...")
+ case DisassociatedEvent(_, address, _) if address == masterAddress =>
+ logWarning(s"Connection to $address failed; waiting for master to reconnect...")
markDisconnected()
case StopClient =>
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
index fedf879eff..67e6c5d66a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
@@ -17,8 +17,7 @@
package org.apache.spark.deploy.master
-private[spark] object ApplicationState
- extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED", "UNKNOWN") {
+private[spark] object ApplicationState extends Enumeration {
type ApplicationState = Value
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
index c0849ef324..043945a211 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -65,7 +65,7 @@ private[spark] class FileSystemPersistenceEngine(
(apps, workers)
}
- private def serializeIntoFile(file: File, value: Serializable) {
+ private def serializeIntoFile(file: File, value: AnyRef) {
val created = file.createNewFile()
if (!created) { throw new IllegalStateException("Could not create file: " + file) }
@@ -77,13 +77,13 @@ private[spark] class FileSystemPersistenceEngine(
out.close()
}
- def deserializeFromFile[T <: Serializable](file: File)(implicit m: Manifest[T]): T = {
+ def deserializeFromFile[T](file: File)(implicit m: Manifest[T]): T = {
val fileData = new Array[Byte](file.length().asInstanceOf[Int])
val dis = new DataInputStream(new FileInputStream(file))
dis.readFully(fileData)
dis.close()
- val clazz = m.erasure.asInstanceOf[Class[T]]
+ val clazz = m.runtimeClass.asInstanceOf[Class[T]]
val serializer = serialization.serializerFor(clazz)
serializer.fromBinary(fileData).asInstanceOf[T]
}
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 cd916672ac..c627dd3806 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
@@ -17,19 +17,20 @@
package org.apache.spark.deploy.master
-import java.util.Date
import java.text.SimpleDateFormat
+import java.util.concurrent.TimeUnit
+import java.util.Date
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.concurrent.duration.{Duration, FiniteDuration}
import akka.actor._
-import akka.actor.Terminated
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+import akka.remote._
import akka.serialization.SerializationExtension
-import akka.util.duration._
-import akka.util.{Duration, Timeout}
+import akka.util.Timeout
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
@@ -37,9 +38,11 @@ import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.MasterMessages._
import org.apache.spark.deploy.master.ui.MasterWebUI
import org.apache.spark.metrics.MetricsSystem
-import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.util.{Utils, AkkaUtils}
private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
+ import context.dispatcher
+
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt
@@ -93,7 +96,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
override def preStart() {
logInfo("Starting Spark master at " + masterUrl)
// 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()
masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get
context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut)
@@ -113,13 +116,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
new BlackHolePersistenceEngine()
}
- leaderElectionAgent = context.actorOf(Props(
- RECOVERY_MODE match {
+ leaderElectionAgent = RECOVERY_MODE match {
case "ZOOKEEPER" =>
- new ZooKeeperLeaderElectionAgent(self, masterUrl)
+ context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl))
case _ =>
- new MonarchyLeaderAgent(self)
- }))
+ context.actorOf(Props(classOf[MonarchyLeaderAgent], self))
+ }
}
override def preRestart(reason: Throwable, message: Option[Any]) {
@@ -142,9 +144,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
RecoveryState.ALIVE
else
RecoveryState.RECOVERING
-
logInfo("I have been elected leader! New state: " + state)
-
if (state == RecoveryState.RECOVERING) {
beginRecovery(storedApps, storedWorkers)
context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() }
@@ -156,7 +156,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
System.exit(0)
}
- case RegisterWorker(id, host, workerPort, cores, memory, webUiPort, publicAddress) => {
+ case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => {
logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
host, workerPort, cores, Utils.megabytesToString(memory)))
if (state == RecoveryState.STANDBY) {
@@ -164,9 +164,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
} else if (idToWorker.contains(id)) {
sender ! RegisterWorkerFailed("Duplicate worker ID")
} else {
- val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+ val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory,
+ sender, workerWebUiPort, publicAddress)
registerWorker(worker)
- context.watch(sender) // This doesn't work with remote actors but helps for testing
persistenceEngine.addWorker(worker)
sender ! RegisteredWorker(masterUrl, masterWebUiUrl)
schedule()
@@ -181,7 +181,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
val app = createApplication(description, sender)
registerApplication(app)
logInfo("Registered app " + description.name + " with ID " + app.id)
- context.watch(sender) // This doesn't work with remote actors but helps for testing
persistenceEngine.addApplication(app)
sender ! RegisteredApplication(app.id, masterUrl)
schedule()
@@ -257,23 +256,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
if (canCompleteRecovery) { completeRecovery() }
}
- case Terminated(actor) => {
- // The disconnected actor could've been either a worker or an app; remove whichever of
- // those we have an entry for in the corresponding actor hashmap
- actorToWorker.get(actor).foreach(removeWorker)
- actorToApp.get(actor).foreach(finishApplication)
- if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
- }
-
- case RemoteClientDisconnected(transport, 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)
- if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
- }
-
- case RemoteClientShutdown(transport, address) => {
+ case DisassociatedEvent(_, address, _) => {
// The disconnected client could've been either a worker or an app; remove whichever it was
+ logInfo(s"$address got disassociated, removing it.")
addressToWorker.get(address).foreach(removeWorker)
addressToApp.get(address).foreach(finishApplication)
if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
@@ -530,9 +515,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
private[spark] object Master {
- private val systemName = "sparkMaster"
+ val systemName = "sparkMaster"
private val actorName = "Master"
- private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
+ val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
def main(argStrings: Array[String]) {
val args = new MasterArguments(argStrings)
@@ -540,11 +525,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)
}
@@ -552,9 +537,9 @@ 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 timeoutDuration = Duration.create(
- System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+ val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName)
+ val timeoutDuration: FiniteDuration = Duration.create(
+ System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS)
implicit val timeout = Timeout(timeoutDuration)
val respFuture = actor ? RequestWebUIPort // ask pattern
val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse]
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
index b91be821f0..256a5a7c28 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
@@ -17,9 +17,7 @@
package org.apache.spark.deploy.master
-private[spark] object RecoveryState
- extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") {
-
+private[spark] object RecoveryState extends Enumeration {
type MasterState = Value
val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
index c8d34f25e2..0b36ef6005 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
@@ -17,9 +17,7 @@
package org.apache.spark.deploy.master
-private[spark] object WorkerState
- extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED", "UNKNOWN") {
-
+private[spark] object WorkerState extends Enumeration {
type WorkerState = Value
val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index a0233a7271..825344b3bb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -70,15 +70,15 @@ class ZooKeeperPersistenceEngine(serialization: Serialization)
(apps, workers)
}
- private def serializeIntoFile(path: String, value: Serializable) {
+ private def serializeIntoFile(path: String, value: AnyRef) {
val serializer = serialization.findSerializerFor(value)
val serialized = serializer.toBinary(value)
zk.create(path, serialized, CreateMode.PERSISTENT)
}
- def deserializeFromFile[T <: Serializable](filename: String)(implicit m: Manifest[T]): T = {
+ def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = {
val fileData = zk.getData("/spark/master_status/" + filename)
- val clazz = m.erasure.asInstanceOf[Class[T]]
+ val clazz = m.runtimeClass.asInstanceOf[Class[T]]
val serializer = serialization.serializerFor(clazz)
serializer.fromBinary(fileData).asInstanceOf[T]
}
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 216d9d44ac..87531b6719 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
@@ -17,23 +17,31 @@
package org.apache.spark.deploy.worker
+import java.io.File
import java.text.SimpleDateFormat
import java.util.Date
-import java.io.File
import scala.collection.mutable.HashMap
+import scala.concurrent.duration._
import akka.actor._
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
-import akka.util.duration._
+import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent}
-import org.apache.spark.Logging
+import org.apache.spark.{SparkException, Logging}
import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
import org.apache.spark.deploy.worker.ui.WorkerWebUI
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.deploy.DeployMessages.WorkerStateResponse
+import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed
+import org.apache.spark.deploy.DeployMessages.KillExecutor
+import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
+import org.apache.spark.deploy.DeployMessages.Heartbeat
+import org.apache.spark.deploy.DeployMessages.RegisteredWorker
+import org.apache.spark.deploy.DeployMessages.LaunchExecutor
+import org.apache.spark.deploy.DeployMessages.RegisterWorker
/**
* @param masterUrls Each url should look like spark://host:port.
@@ -47,6 +55,7 @@ private[spark] class Worker(
masterUrls: Array[String],
workDirPath: String = null)
extends Actor with Logging {
+ import context.dispatcher
Utils.checkHost(host, "Expected hostname")
assert (port > 0)
@@ -63,7 +72,8 @@ private[spark] class Worker(
var masterIndex = 0
val masterLock: Object = new Object()
- var master: ActorRef = null
+ var master: ActorSelection = null
+ var masterAddress: Address = null
var activeMasterUrl: String = ""
var activeMasterWebUiUrl : String = ""
@volatile var registered = false
@@ -114,7 +124,7 @@ private[spark] class Worker(
logInfo("Spark home: " + sparkHome)
createWorkDir()
webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
-
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi.start()
registerWithMaster()
@@ -126,9 +136,13 @@ private[spark] class Worker(
masterLock.synchronized {
activeMasterUrl = url
activeMasterWebUiUrl = uiUrl
- master = context.actorFor(Master.toAkkaUrl(activeMasterUrl))
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
- context.watch(master) // Doesn't work with remote actors, but useful for testing
+ master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+ masterAddress = activeMasterUrl match {
+ case Master.sparkUrlRegex(_host, _port) =>
+ Address("akka.tcp", Master.systemName, _host, _port.toInt)
+ case x =>
+ throw new SparkException("Invalid spark URL: " + x)
+ }
connected = true
}
}
@@ -136,7 +150,7 @@ private[spark] class Worker(
def tryRegisterAllMasters() {
for (masterUrl <- masterUrls) {
logInfo("Connecting to master " + masterUrl + "...")
- val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+ val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get,
publicAddress)
}
@@ -175,7 +189,6 @@ private[spark] class Worker(
case MasterChanged(masterUrl, masterWebUiUrl) =>
logInfo("Master has changed, new master is at " + masterUrl)
- context.unwatch(master)
changeMaster(masterUrl, masterWebUiUrl)
val execs = executors.values.
@@ -234,13 +247,8 @@ private[spark] class Worker(
}
}
- case Terminated(actor_) if actor_ == master =>
- masterDisconnected()
-
- case RemoteClientDisconnected(transport, address) if address == master.path.address =>
- masterDisconnected()
-
- case RemoteClientShutdown(transport, address) if address == master.path.address =>
+ case x: DisassociatedEvent if x.remoteAddress == masterAddress =>
+ logInfo(s"$x Disassociated !")
masterDisconnected()
case RequestWorkerState => {
@@ -280,8 +288,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,
- masterUrls, workDir)), name = "Worker")
+ actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
+ masterUrls, 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 800f1cafcc..6c18a3c245 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,20 +17,19 @@
package org.apache.spark.deploy.worker.ui
-import akka.util.{Duration, Timeout}
+import java.io.File
-import java.io.{FileInputStream, File}
+import scala.concurrent.duration._
+import akka.util.Timeout
import javax.servlet.http.HttpServletRequest
-import org.eclipse.jetty.server.{Handler, Server}
-
+import org.apache.spark.Logging
import org.apache.spark.deploy.worker.Worker
-import org.apache.spark.{Logging}
-import org.apache.spark.ui.JettyUtils
+import org.apache.spark.ui.{JettyUtils, UIUtils}
import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.ui.UIUtils
import org.apache.spark.util.Utils
+import org.eclipse.jetty.server.{Handler, Server}
/**
* Web UI server for the standalone worker.
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index 8332631838..debbdd4c44 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -19,15 +19,14 @@ 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
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
import org.apache.spark.util.{Utils, AkkaUtils}
-
private[spark] class CoarseGrainedExecutorBackend(
driverUrl: String,
executorId: String,
@@ -40,14 +39,13 @@ private[spark] class CoarseGrainedExecutorBackend(
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])
}
override def receive = {
@@ -77,8 +75,8 @@ private[spark] class CoarseGrainedExecutorBackend(
executor.killTask(taskId)
}
- case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
- logError("Driver terminated or disconnected! Shutting down.")
+ case x: DisassociatedEvent =>
+ logError(s"Driver $x disassociated! Shutting down.")
System.exit(1)
case StopExecutor =>
@@ -99,12 +97,13 @@ private[spark] object CoarseGrainedExecutorBackend {
// Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor
// before getting started with all our system properties, etc
- val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0)
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0,
+ indestructible = true)
// set it
val sparkHostPort = hostname + ":" + boundPort
System.setProperty("spark.hostPort", sparkHostPort)
- val actor = actorSystem.actorOf(
- Props(new CoarseGrainedExecutorBackend(driverUrl, executorId, sparkHostPort, cores)),
+ actorSystem.actorOf(
+ Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
name = "Executor")
actorSystem.awaitTermination()
}
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 5c9bb9db1c..0b0a60ee60 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -121,7 +121,7 @@ private[spark] class Executor(
// Akka's message frame size. If task result is bigger than this, we use the block manager
// to send the result back.
private val akkaFrameSize = {
- env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size")
+ env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size")
}
// Start worker thread pool
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 9c2fee4023..703bc6a9ca 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/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index faaf837be0..d1c74a5063 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.ExecutionContext.Implicits.global
+import scala.reflect.ClassTag
import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
@@ -28,7 +29,7 @@ import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
* A set of asynchronous RDD actions available through an implicit conversion.
* Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
*/
-class AsyncRDDActions[T: ClassManifest](self: RDD[T]) extends Serializable with Logging {
+class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging {
/**
* Returns a future for counting the number of elements in the RDD.
diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
index 44ea573a7c..424354ae16 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -17,6 +17,8 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
import org.apache.spark.storage.{BlockId, BlockManager}
@@ -25,7 +27,7 @@ private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends P
}
private[spark]
-class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[BlockId])
+class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId])
extends RDD[T](sc, Nil) {
@transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
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 0de22f0e06..87b950ba43 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 d3033ea4a6..a712ef1c27 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -17,15 +17,13 @@
package org.apache.spark.rdd
+import java.io.IOException
+
+import scala.reflect.ClassTag
+
+import org.apache.hadoop.fs.Path
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.{NullWritable, BytesWritable}
-import org.apache.hadoop.util.ReflectionUtils
-import org.apache.hadoop.fs.Path
-import java.io.{File, IOException, EOFException}
-import java.text.NumberFormat
private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
@@ -33,7 +31,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/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
index c5de6362a9..98da35763b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
@@ -22,6 +22,7 @@ import java.io.{ObjectOutputStream, IOException}
import scala.collection.mutable
import scala.Some
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
/**
* Class that captures a coalesced RDD by essentially keeping track of parent partitions
@@ -68,7 +69,7 @@ case class CoalescedRDDPartition(
* @param maxPartitions number of desired partitions in the coalesced RDD
* @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance
*/
-class CoalescedRDD[T: ClassManifest](
+class CoalescedRDD[T: ClassTag](
@transient var prev: RDD[T],
maxPartitions: Int,
balanceSlack: Double = 0.10)
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index 02d75eccc5..688c310ee9 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -90,12 +90,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = {
// Compute the minimum and the maxium
val (max: Double, min: Double) = self.mapPartitions { items =>
- Iterator(items.foldRight(-1/0.0, Double.NaN)((e: Double, x: Pair[Double, Double]) =>
+ Iterator(items.foldRight(Double.NegativeInfinity,
+ Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) =>
(x._1.max(e), x._2.min(e))))
}.reduce { (maxmin1, maxmin2) =>
(maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2))
}
- if (max.isNaN() || max.isInfinity || min.isInfinity ) {
+ if (min.isNaN || max.isNaN || max.isInfinity || min.isInfinity ) {
throw new UnsupportedOperationException(
"Histogram on either an empty RDD or RDD containing +/-infinity or NaN")
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
index c8900d1a93..a84e5f9fd8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
@@ -17,13 +17,14 @@
package org.apache.spark.rdd
-import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
+import scala.reflect.ClassTag
+import org.apache.spark.{Partition, SparkContext, TaskContext}
/**
* An RDD that is empty, i.e. has no element in it.
*/
-class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) {
+class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) {
override def getPartitions: Array[Partition] = Array.empty
diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
index 5312dc0b59..e74c83b90b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
@@ -18,8 +18,9 @@
package org.apache.spark.rdd
import org.apache.spark.{OneToOneDependency, Partition, TaskContext}
+import scala.reflect.ClassTag
-private[spark] class FilteredRDD[T: ClassManifest](
+private[spark] class FilteredRDD[T: ClassTag](
prev: RDD[T],
f: T => Boolean)
extends RDD[T](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
index cbdf6d84c0..4d1878fc14 100644
--- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
@@ -18,10 +18,11 @@
package org.apache.spark.rdd
import org.apache.spark.{Partition, TaskContext}
+import scala.reflect.ClassTag
private[spark]
-class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
+class FlatMappedRDD[U: ClassTag, T: ClassTag](
prev: RDD[T],
f: T => TraversableOnce[U])
extends RDD[U](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
index 829545d7b0..1a694475f6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
@@ -18,8 +18,9 @@
package org.apache.spark.rdd
import org.apache.spark.{Partition, TaskContext}
+import scala.reflect.ClassTag
-private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
+private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T])
extends RDD[Array[T]](prev) {
override def getPartitions: Array[Partition] = firstParent[T].partitions
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..8df8718f3b 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,8 @@ 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 +47,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/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
index ae70d55951..db15baf503 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
@@ -18,9 +18,9 @@
package org.apache.spark.rdd
import org.apache.spark.{Partition, TaskContext}
+import scala.reflect.ClassTag
-
-private[spark] class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
+private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag](
prev: RDD[T],
f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator)
preservesPartitioning: Boolean = false)
diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
index e8be1c4816..8d7c288593 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
@@ -17,10 +17,12 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.spark.{Partition, TaskContext}
private[spark]
-class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U)
+class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U)
extends RDD[U](prev) {
override def getPartitions: Array[Partition] = firstParent[T].partitions
diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index 697be8b997..d5691f2267 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -17,7 +17,9 @@
package org.apache.spark.rdd
-import org.apache.spark.{RangePartitioner, Logging}
+import scala.reflect.ClassTag
+
+import org.apache.spark.{Logging, RangePartitioner}
/**
* Extra functions available on RDDs of (key, value) pairs where the key is sortable through
@@ -25,9 +27,9 @@ import org.apache.spark.{RangePartitioner, Logging}
* use these functions. They will work with any key type that has a `scala.math.Ordered`
* implementation.
*/
-class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest,
- V: ClassManifest,
- P <: Product2[K, V] : ClassManifest](
+class OrderedRDDFunctions[K <% Ordered[K]: ClassTag,
+ V: ClassTag,
+ P <: Product2[K, V] : ClassTag](
self: RDD[P])
extends Logging with Serializable {
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 93b78e1232..48168e152e 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
@@ -50,7 +51,7 @@ import org.apache.spark.Partitioner.defaultPartitioner
* Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
* Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
*/
-class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
+class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
extends Logging
with SparkHadoopMapReduceUtil
with Serializable {
@@ -415,7 +416,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]])
}
@@ -431,7 +432,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]])
}
@@ -488,15 +489,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)
/**
@@ -525,8 +526,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]])
}
/**
@@ -535,16 +536,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]])
}
/**
@@ -698,11 +699,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 cd96250389..09d0a8189d 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,
@@ -109,7 +111,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 574dd4233f..ea8885b36e 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}
@@ -49,7 +51,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))) {
@@ -69,6 +71,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 893708f8f2..ea45566ad1 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))
/**
@@ -374,25 +377,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)
}
@@ -439,7 +442,7 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD by applying a function to each partition of this RDD.
*/
- def mapPartitions[U: ClassManifest](
+ def mapPartitions[U: ClassTag](
f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter)
new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
@@ -449,7 +452,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] = {
val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter)
new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
@@ -459,7 +462,7 @@ abstract class RDD[T: ClassManifest](
* Return a new RDD by applying a function to each partition of this RDD. This is a variant of
* mapPartitions that also passes the TaskContext into the closure.
*/
- def mapPartitionsWithContext[U: ClassManifest](
+ def mapPartitionsWithContext[U: ClassTag](
f: (TaskContext, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean = false): RDD[U] = {
val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter)
@@ -471,7 +474,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] = {
mapPartitionsWithIndex(f, preservesPartitioning)
}
@@ -481,7 +484,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]
+ def mapWith[A: ClassTag, U: ClassTag]
(constructA: Int => A, preservesPartitioning: Boolean = false)
(f: (T, A) => U): RDD[U] = {
mapPartitionsWithIndex((index, iter) => {
@@ -495,7 +498,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]
+ def flatMapWith[A: ClassTag, U: ClassTag]
(constructA: Int => A, preservesPartitioning: Boolean = false)
(f: (T, A) => Seq[U]): RDD[U] = {
mapPartitionsWithIndex((index, iter) => {
@@ -509,7 +512,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)(f: (T, A) => Unit) {
+ def foreachWith[A: ClassTag](constructA: Int => A)(f: (T, A) => Unit) {
mapPartitionsWithIndex { (index, iter) =>
val a = constructA(index)
iter.map(t => {f(t, a); t})
@@ -521,7 +524,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)(p: (T, A) => Boolean): RDD[T] = {
+ def filterWith[A: ClassTag](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
mapPartitionsWithIndex((index, iter) => {
val a = constructA(index)
iter.filter(t => p(t, a))
@@ -534,7 +537,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
@@ -542,32 +545,27 @@ 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]
- (rdd2: RDD[B], preservesPartitioning: Boolean)
- (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
- new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning)
-
- 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, false)
- def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean)
(f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning)
- 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, false)
- 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], preservesPartitioning: Boolean)
(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning)
- 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, false)
@@ -605,7 +603,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)
}
@@ -695,7 +693,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)
@@ -744,7 +742,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.
@@ -775,7 +773,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) =>
@@ -942,12 +940,12 @@ abstract class RDD[T: ClassManifest](
/** Record user function generating this RDD. */
@transient 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]]
}
@@ -1009,7 +1007,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/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index a5d751a7bd..3682c84598 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -17,8 +17,10 @@
package org.apache.spark.rdd
-import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext}
+import scala.reflect.ClassTag
+import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency,
+ SparkEnv, TaskContext}
private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
override val index = idx
@@ -32,7 +34,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
* @tparam K the key class.
* @tparam V the value class.
*/
-class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest](
+class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag](
@transient var prev: RDD[P],
part: Partitioner)
extends RDD[P](prev.context, Nil) {
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 7af4d803e7..aab30b1bb4 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 a97d2a01c8..83be3c6eb4 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,
@@ -38,7 +39,7 @@ private[spark] class ZippedPartitionsPartition(
}
}
-abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
+abstract class ZippedPartitionsBaseRDD[V: ClassTag](
sc: SparkContext,
var rdds: Seq[RDD[_]],
preservesPartitioning: Boolean = false)
@@ -71,7 +72,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],
@@ -117,7 +118,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 f9cd021dd3..963d15b76d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -21,9 +21,11 @@ import java.io.NotSerializableException
import java.util.Properties
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor._
-import akka.util.duration._
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import akka.actor._
import org.apache.spark._
import org.apache.spark.rdd.RDD
@@ -104,7 +106,7 @@ class DAGScheduler(
// The time, in millis, to wait for fetch failure events to stop coming in after one is detected;
// this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one
// as more failure events come in
- val RESUBMIT_TIMEOUT = 50L
+ val RESUBMIT_TIMEOUT = 50.milliseconds
// The time, in millis, to wake up between polls of the completion queue in order to potentially
// resubmit failed stages
@@ -177,13 +179,14 @@ class DAGScheduler(
var resubmissionTask: Cancellable = _
override def preStart() {
+ import context.dispatcher
/**
* A message is sent to the actor itself periodically to remind the actor to resubmit failed
* stages. In this way, stage resubmission can be done within the same thread context of
* other event processing logic to avoid unnecessary synchronization overhead.
*/
resubmissionTask = context.system.scheduler.schedule(
- RESUBMIT_TIMEOUT.millis, RESUBMIT_TIMEOUT.millis, self, ResubmitFailedStages)
+ RESUBMIT_TIMEOUT, RESUBMIT_TIMEOUT, self, ResubmitFailedStages)
}
/**
@@ -460,7 +463,7 @@ class DAGScheduler(
waiter
}
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
index 0a786deb16..3832ee7ff6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
@@ -22,7 +22,7 @@ package org.apache.spark.scheduler
* to order tasks amongst a Schedulable's sub-queues
* "NONE" is used when the a Schedulable has no sub-queues.
*/
-object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") {
+object SchedulingMode extends Enumeration {
type SchedulingMode = Value
val FAIR,FIFO,NONE = Value
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
index 47b0f387aa..35de13c385 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
@@ -18,9 +18,7 @@
package org.apache.spark.scheduler
-private[spark] object TaskLocality
- extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY")
-{
+private[spark] object TaskLocality extends Enumeration {
// process local is expected to be used ONLY within tasksetmanager for now.
val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
index 4d82430b97..66ab8ea4cd 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
@@ -24,8 +24,7 @@ import java.util.{TimerTask, Timer}
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet
-
-import akka.util.duration._
+import scala.concurrent.duration._
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
@@ -122,7 +121,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
if (System.getProperty("spark.speculation", "false").toBoolean) {
logInfo("Starting speculative execution thread")
-
+ import sc.env.actorSystem.dispatcher
sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
SPECULATION_INTERVAL milliseconds) {
checkSpeculatableTasks()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index d0ba5bf55d..f5e8766f6d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.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.{DisassociatedEvent, RemotingLifecycleEvent}
import org.apache.spark.{SparkException, Logging, TaskState}
import org.apache.spark.scheduler.TaskDescription
@@ -53,15 +52,15 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
private val executorAddress = new HashMap[String, Address]
private val executorHost = new HashMap[String, String]
private val freeCores = new HashMap[String, Int]
- private val actorToExecutorId = new HashMap[ActorRef, String]
private val addressToExecutorId = new HashMap[Address, String]
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)
}
@@ -73,12 +72,10 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
} else {
logInfo("Registered executor: " + sender + " with ID " + executorId)
sender ! RegisteredExecutor(sparkProperties)
- context.watch(sender)
executorActor(executorId) = sender
executorHost(executorId) = Utils.parseHostPort(hostPort)._1
freeCores(executorId) = cores
executorAddress(executorId) = sender.path.address
- actorToExecutorId(sender) = executorId
addressToExecutorId(sender.path.address) = executorId
totalCoreCount.addAndGet(cores)
makeOffers()
@@ -118,14 +115,9 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
removeExecutor(executorId, reason)
sender ! true
- case Terminated(actor) =>
- actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
+ case DisassociatedEvent(_, address, _) =>
+ addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated"))
- case RemoteClientDisconnected(transport, address) =>
- addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected"))
-
- case RemoteClientShutdown(transport, address) =>
- addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown"))
}
// Make fake resource offers on all executors
@@ -153,7 +145,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
if (executorActor.contains(executorId)) {
logInfo("Executor " + executorId + " disconnected, so removing it")
val numCores = freeCores(executorId)
- actorToExecutorId -= executorActor(executorId)
addressToExecutorId -= executorAddress(executorId)
executorActor -= executorId
executorHost -= executorId
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index e000531a26..e8fecec4a6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -36,7 +36,7 @@ private[spark] class SimrSchedulerBackend(
override def start() {
super.start()
- 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"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
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 cefa970bb9..7127a72d6d 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"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
index 2064d97b49..e68c527713 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
@@ -71,7 +71,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche
case cnf: ClassNotFoundException =>
val loader = Thread.currentThread.getContextClassLoader
taskSetManager.abort("ClassNotFound with classloader: " + loader)
- case ex =>
+ case ex: Throwable =>
taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex))
}
}
@@ -95,7 +95,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche
val loader = Thread.currentThread.getContextClassLoader
logError(
"Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
- case ex => {}
+ case ex: Throwable => {}
}
scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
}
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 cd521e0f2b..84fe3094cc 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
@@ -120,7 +120,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"),
CoarseGrainedSchedulerBackend.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 702aca8323..19a025a329 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -24,9 +24,9 @@ import scala.collection.mutable.{HashMap, ArrayBuffer}
import scala.util.Random
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.{FastBufferedOutputStream, FastByteArrayOutputStream}
@@ -924,4 +924,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 94038649b3..e05b842476 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -17,16 +17,17 @@
package org.apache.spark.storage
-import akka.actor.ActorRef
-import akka.dispatch.{Await, Future}
+import scala.concurrent.{Await, Future}
+import scala.concurrent.duration._
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import akka.actor._
import akka.pattern.ask
-import akka.util.Duration
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.storage.BlockManagerMessages._
-
-private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
+private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging {
val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt
@@ -156,7 +157,10 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
while (attempts < AKKA_RETRY_ATTEMPTS) {
attempts += 1
try {
- val future = driverActor.ask(message)(timeout)
+ val future = driverActor match {
+ case Left(a: ActorRef) => a.ask(message)(timeout)
+ case Right(b: ActorSelection) => b.ask(message)(timeout)
+ }
val result = Await.result(future, timeout)
if (result == null) {
throw new SparkException("BlockManagerMaster returned null")
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 f8cf14b503..154a3980e9 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/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
index 860e680576..a8db37ded1 100644
--- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -93,7 +93,7 @@ private[spark] object ThreadingTest {
val actorSystem = ActorSystem("test")
val serializer = new KryoSerializer
val blockManagerMaster = new BlockManagerMaster(
- actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+ Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))))
val blockManager = new BlockManager(
"<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024)
val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i))
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..74133cef6c 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -17,11 +17,8 @@
package org.apache.spark.util
-import akka.actor.{ActorSystem, ExtendedActorSystem}
+import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem}
import com.typesafe.config.ConfigFactory
-import akka.util.duration._
-import akka.remote.RemoteActorRefProvider
-
/**
* Various utility classes for working with Akka.
@@ -34,39 +31,57 @@ private[spark] object AkkaUtils {
*
* Note: the `name` parameter is important, as even if a client sends a message to right
* host + port, if the system name is incorrect, Akka will drop the message.
+ *
+ * If indestructible is set to true, the Actor System will continue running in the event
+ * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]].
*/
- def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = {
- val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt
+ def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false)
+ : (ActorSystem, Int) = {
+
+ 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 akkaTimeout = System.getProperty("spark.akka.timeout", "100").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.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.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))
+ val lifecycleEvents =
+ if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
+
+ val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt
+ val akkaFailureDetector =
+ System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble
+ val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt
- val actorSystem = ActorSystem(name, akkaConf)
+ val akkaConf = ConfigFactory.parseString(
+ s"""
+ |akka.daemonic = on
+ |akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
+ |akka.stdout-loglevel = "ERROR"
+ |akka.jvm-exit-on-fatal-error = off
+ |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s
+ |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s
+ |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector
+ |akka.actor.provider = "akka.remote.RemoteActorRefProvider"
+ |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
+ |akka.remote.netty.tcp.hostname = "$host"
+ |akka.remote.netty.tcp.port = $port
+ |akka.remote.netty.tcp.tcp-nodelay = on
+ |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s
+ |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB
+ |akka.remote.netty.tcp.execution-pool-size = $akkaThreads
+ |akka.actor.default-dispatcher.throughput = $akkaBatchSize
+ |akka.remote.log-remote-lifecycle-events = $lifecycleEvents
+ """.stripMargin)
+
+ val actorSystem = if (indestructible) {
+ IndestructibleActorSystem(name, akkaConf)
+ } else {
+ 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/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala
new file mode 100644
index 0000000000..bf71882ef7
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+
+// Must be in akka.actor package as ActorSystemImpl is protected[akka].
+package akka.actor
+
+import scala.util.control.{ControlThrowable, NonFatal}
+
+import com.typesafe.config.Config
+
+/**
+ * An [[akka.actor.ActorSystem]] which refuses to shut down in the event of a fatal exception.
+ * This is necessary as Spark Executors are allowed to recover from fatal exceptions
+ * (see [[org.apache.spark.executor.Executor]]).
+ */
+object IndestructibleActorSystem {
+ def apply(name: String, config: Config): ActorSystem =
+ apply(name, config, ActorSystem.findClassLoader())
+
+ def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem =
+ new IndestructibleActorSystemImpl(name, config, classLoader).start()
+}
+
+private[akka] class IndestructibleActorSystemImpl(
+ override val name: String,
+ applicationConfig: Config,
+ classLoader: ClassLoader)
+ extends ActorSystemImpl(name, applicationConfig, classLoader) {
+
+ protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = {
+ val fallbackHandler = super.uncaughtExceptionHandler
+
+ new Thread.UncaughtExceptionHandler() {
+ def uncaughtException(thread: Thread, cause: Throwable): Unit = {
+ if (isFatalError(cause) && !settings.JvmExitOnFatalError) {
+ log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " +
+ "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name)
+ //shutdown() //TODO make it configurable
+ } else {
+ fallbackHandler.uncaughtException(thread, cause)
+ }
+ }
+ }
+ }
+
+ def isFatalError(e: Throwable): Boolean = {
+ e match {
+ case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable =>
+ false
+ case _ =>
+ true
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index 67a7f87a5c..7b41ef89f1 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -55,8 +55,7 @@ class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, clea
}
}
-object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask",
- "ShuffleMapTask", "BlockManager", "DiskBlockManager", "BroadcastVars") {
+object MetadataCleanerType extends Enumeration {
val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK,
SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value
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 a79e64e810..3f7858d2de 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -22,10 +22,11 @@ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address}
import java.util.{Locale, Random, UUID}
import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor}
+import scala.collection.JavaConversions._
import scala.collection.Map
import scala.collection.mutable.ArrayBuffer
-import scala.collection.JavaConversions._
import scala.io.Source
+import scala.reflect.ClassTag
import com.google.common.io.Files
import com.google.common.util.concurrent.ThreadFactoryBuilder
@@ -319,7 +320,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/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
index 80545c9688..c26f23d500 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
@@ -17,6 +17,7 @@
package org.apache.spark.util.collection
+import scala.reflect.ClassTag
/**
* A fast hash map implementation for nullable keys. This hash map supports insertions and updates,
@@ -26,7 +27,7 @@ package org.apache.spark.util.collection
* Under the hood, it uses our OpenHashSet implementation.
*/
private[spark]
-class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: ClassManifest](
+class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag](
initialCapacity: Int)
extends Iterable[(K, V)]
with Serializable {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
index 40986e3731..87e009a4de 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
@@ -17,6 +17,7 @@
package org.apache.spark.util.collection
+import scala.reflect._
/**
* A simple, fast hash set optimized for non-null insertion-only use case, where keys are never
@@ -36,7 +37,7 @@ package org.apache.spark.util.collection
* to explore all spaces for each key (see http://en.wikipedia.org/wiki/Quadratic_probing).
*/
private[spark]
-class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
+class OpenHashSet[@specialized(Long, Int) T: ClassTag](
initialCapacity: Int,
loadFactor: Double)
extends Serializable {
@@ -62,14 +63,14 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
// throws:
// scala.tools.nsc.symtab.Types$TypeError: type mismatch;
// found : scala.reflect.AnyValManifest[Long]
- // required: scala.reflect.ClassManifest[Int]
+ // required: scala.reflect.ClassTag[Int]
// at scala.tools.nsc.typechecker.Contexts$Context.error(Contexts.scala:298)
// at scala.tools.nsc.typechecker.Infer$Inferencer.error(Infer.scala:207)
// ...
- val mt = classManifest[T]
- if (mt == ClassManifest.Long) {
+ val mt = classTag[T]
+ if (mt == ClassTag.Long) {
(new LongHasher).asInstanceOf[Hasher[T]]
- } else if (mt == ClassManifest.Int) {
+ } else if (mt == ClassTag.Int) {
(new IntHasher).asInstanceOf[Hasher[T]]
} else {
new Hasher[T]
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
index d76143e45a..2e1ef06cbc 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
@@ -17,6 +17,7 @@
package org.apache.spark.util.collection
+import scala.reflect._
/**
* A fast hash map implementation for primitive, non-null keys. This hash map supports
@@ -26,15 +27,15 @@ package org.apache.spark.util.collection
* Under the hood, it uses our OpenHashSet implementation.
*/
private[spark]
-class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest,
- @specialized(Long, Int, Double) V: ClassManifest](
+class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
+ @specialized(Long, Int, Double) V: ClassTag](
initialCapacity: Int)
extends Iterable[(K, V)]
with Serializable {
def this() = this(64)
- require(classManifest[K] == classManifest[Long] || classManifest[K] == classManifest[Int])
+ require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int])
// Init in constructor (instead of in declaration) to work around a Scala compiler specialization
// bug that would generate two arrays (one for Object and one for specialized T).
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
index 20554f0aab..b84eb65c62 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
@@ -17,11 +17,13 @@
package org.apache.spark.util.collection
+import scala.reflect.ClassTag
+
/**
* An append-only, non-threadsafe, array-backed vector that is optimized for primitive types.
*/
private[spark]
-class PrimitiveVector[@specialized(Long, Int, Double) V: ClassManifest](initialSize: Int = 64) {
+class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: Int = 64) {
private var _numElements = 0
private var _array: Array[V] = _
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 d2226aa5a5..f25d921d3f 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._
@@ -205,7 +206,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
@@ -274,7 +275,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 480bac84f3..d9cffb74de 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -303,12 +303,13 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
Thread.sleep(200)
}
} catch {
- case _ => { Thread.sleep(10) }
+ case _: Throwable => { Thread.sleep(10) }
// Do nothing. We might see exceptions because block manager
// is racing this thread to remove entries from the driver.
}
}
}
+
}
object DistributedSuite {
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 b7eb268bd5..271dc905bc 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -49,14 +49,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
test("master start and stop") {
val actorSystem = ActorSystem("test")
val tracker = new MapOutputTrackerMaster()
- tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
+ tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))))
tracker.stop()
}
test("master register and fetch") {
val actorSystem = ActorSystem("test")
val tracker = new MapOutputTrackerMaster()
- tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
+ tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))))
tracker.registerShuffle(10, 2)
val compressedSize1000 = MapOutputTracker.compressSize(1000L)
val compressedSize10000 = MapOutputTracker.compressSize(10000L)
@@ -75,7 +75,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
test("master register and unregister and fetch") {
val actorSystem = ActorSystem("test")
val tracker = new MapOutputTrackerMaster()
- tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
+ tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))))
tracker.registerShuffle(10, 2)
val compressedSize1000 = MapOutputTracker.compressSize(1000L)
val compressedSize10000 = MapOutputTracker.compressSize(10000L)
@@ -101,13 +101,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
System.setProperty("spark.hostPort", hostname + ":" + boundPort)
val masterTracker = new MapOutputTrackerMaster()
- masterTracker.trackerActor = actorSystem.actorOf(
- Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")
+ masterTracker.trackerActor = Left(actorSystem.actorOf(
+ Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker"))
val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
val slaveTracker = new MapOutputTracker()
- slaveTracker.trackerActor = slaveSystem.actorFor(
- "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker")
+ slaveTracker.trackerActor = Right(slaveSystem.actorSelection(
+ "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker"))
masterTracker.registerShuffle(10, 1)
masterTracker.incrementEpoch()
diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
index 46a2da1724..768ca3850e 100644
--- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
@@ -37,7 +37,7 @@ class UnpersistSuite extends FunSuite with LocalSparkContext {
Thread.sleep(200)
}
} catch {
- case _ => { Thread.sleep(10) }
+ case _: Throwable => { Thread.sleep(10) }
// Do nothing. We might see exceptions because block manager
// is racing this thread to remove entries from the driver.
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index 354ab8ae5d..d8dcd6d14c 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -244,8 +244,8 @@ class RDDSuite extends FunSuite with SharedSparkContext {
// test that you get over 90% locality in each group
val minLocality = coalesced2.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction)
- .foldLeft(1.)((perc, loc) => math.min(perc,loc))
- assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%")
+ .foldLeft(1.0)((perc, loc) => math.min(perc,loc))
+ assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%")
// test that the groups are load balanced with 100 +/- 20 elements in each
val maxImbalance = coalesced2.partitions
@@ -257,9 +257,9 @@ class RDDSuite extends FunSuite with SharedSparkContext {
val coalesced3 = data3.coalesce(numMachines*2)
val minLocality2 = coalesced3.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction)
- .foldLeft(1.)((perc, loc) => math.min(perc,loc))
+ .foldLeft(1.0)((perc, loc) => math.min(perc,loc))
assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " +
- (minLocality2*100.).toInt + "%")
+ (minLocality2*100.0).toInt + "%")
}
test("zipped RDDs") {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 1fd76420ea..2e41438a52 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -145,7 +145,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
// Make a task whose result is larger than the akka frame size
System.setProperty("spark.akka.frameSize", "1")
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.maximum-frame-size").toInt
val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x)
assert(result === 1.to(akkaFrameSize).toArray)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
index ee150a3107..27c2d53361 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
@@ -82,7 +82,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA
test("handling results larger than Akka frame size") {
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.maximum-frame-size").toInt
val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
assert(result === 1.to(akkaFrameSize).toArray)
@@ -103,7 +103,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA
}
scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler)
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.maximum-frame-size").toInt
val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
assert(result === 1.to(akkaFrameSize).toArray)
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
index cb76275e39..b647e8a672 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
@@ -39,7 +39,7 @@ class BlockIdSuite extends FunSuite {
fail()
} catch {
case e: IllegalStateException => // OK
- case _ => fail()
+ case _: Throwable => fail()
}
}
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 484a654108..5b4d63b954 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -56,7 +56,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
System.setProperty("spark.hostPort", "localhost:" + boundPort)
master = new BlockManagerMaster(
- actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+ Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))))
// Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
oldArch = System.setProperty("os.arch", "amd64")
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 8f0ec6683b..3764f4d1a0 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
@@ -34,7 +34,6 @@ class UISuite extends FunSuite {
}
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/docs/configuration.md b/docs/configuration.md
index 62c4457a3e..677d182e50 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -275,12 +275,33 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td>spark.akka.timeout</td>
- <td>20</td>
+ <td>100</td>
<td>
Communication timeout between Spark nodes, in seconds.
</td>
</tr>
<tr>
+ <td>spark.akka.heartbeat.pauses</td>
+ <td>600</td>
+ <td>
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if you need to.
+ </td>
+</tr>
+<tr>
+ <td>spark.akka.failure-detector.threshold</td>
+ <td>300.0</td>
+ <td>
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). This maps to akka's `akka.remote.transport-failure-detector.threshold`. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
+ </td>
+</tr>
+<tr>
+ <td>spark.akka.heartbeat.interval</td>
+ <td>1000</td>
+ <td>
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those.
+ </td>
+</tr>
+<tr>
<td>spark.driver.host</td>
<td>(local hostname)</td>
<td>
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 9e4c4e123f..aa75ca4324 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -32,14 +32,16 @@ If you want to test out the YARN deployment mode, you can use the current Spark
Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN.
Environment variables:
+
* `SPARK_YARN_USER_ENV`, to add environment variables to the Spark processes launched on YARN. This can be a comma separated list of environment variables, e.g. `SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar"`.
System Properties:
-* 'spark.yarn.applicationMaster.waitTries', property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10.
-* 'spark.yarn.submit.file.replication', the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives.
-* 'spark.yarn.preserve.staging.files', set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
-* 'spark.yarn.scheduler.heartbeat.interval-ms', the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds.
-* 'spark.yarn.max.worker.failures', the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3.
+
+* `spark.yarn.applicationMaster.waitTries`, property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10.
+* `spark.yarn.submit.file.replication`, the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives.
+* `spark.yarn.preserve.staging.files`, set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
+* `spark.yarn.scheduler.heartbeat.interval-ms`, the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds.
+* `spark.yarn.max.worker.failures`, the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3.
# Launching Spark on YARN
diff --git a/examples/pom.xml b/examples/pom.xml
index aee371fbc7..e16b1c80aa 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-examples_2.9.3</artifactId>
+ <artifactId>spark-examples_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Examples</name>
<url>http://spark.incubator.apache.org/</url>
@@ -49,25 +49,25 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_2.10</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_2.10</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_2.10</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
@@ -87,8 +87,8 @@
</exclusions>
</dependency>
<dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka_2.9.2</artifactId>
+ <groupId>com.sksamuel.kafka</groupId>
+ <artifactId>kafka_2.10</artifactId>
<version>0.8.0-beta1</version>
<exclusions>
<exclusion>
@@ -107,23 +107,23 @@
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>algebird-core_2.9.2</artifactId>
+ <artifactId>algebird-core_2.10</artifactId>
<version>0.1.11</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_2.10</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>
@@ -166,8 +166,8 @@
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
index 152f029213..407cd7ccfa 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
@@ -123,7 +123,7 @@ public class JavaLogQuery {
});
List<Tuple2<Tuple3<String, String, String>, Stats>> output = counts.collect();
- for (Tuple2 t : output) {
+ for (Tuple2<?,?> t : output) {
System.out.println(t._1 + "\t" + t._2);
}
System.exit(0);
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
index c5603a639b..89aed8f279 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
@@ -21,7 +21,6 @@ import scala.Tuple2;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFlatMapFunction;
@@ -106,7 +105,7 @@ public class JavaPageRank {
// Collects all URL ranks and dump them to console.
List<Tuple2<String, Double>> output = ranks.collect();
- for (Tuple2 tuple : output) {
+ for (Tuple2<?,?> tuple : output) {
System.out.println(tuple._1 + " has rank: " + tuple._2 + ".");
}
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
index 07d32ad659..bd6383e13d 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
@@ -58,7 +58,7 @@ public class JavaWordCount {
});
List<Tuple2<String, Integer>> output = counts.collect();
- for (Tuple2 tuple : output) {
+ for (Tuple2<?,?> tuple : output) {
System.out.println(tuple._1 + ": " + tuple._2);
}
System.exit(0);
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
index 628cb892b6..45a0d237da 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
@@ -25,7 +25,6 @@ import org.apache.spark.mllib.recommendation.ALS;
import org.apache.spark.mllib.recommendation.MatrixFactorizationModel;
import org.apache.spark.mllib.recommendation.Rating;
-import java.io.Serializable;
import java.util.Arrays;
import java.util.StringTokenizer;
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 af52b7e9a1..50e3f9639c 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,10 +83,10 @@ 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)
+ lazy private val remotePublisher = context.actorSelection(urlOfPublisher)
override def preStart = remotePublisher ! SubscribeReceiver(context.self)
@@ -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 f472082ad1..228f8c029b 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project ML Library</name>
<url>http://spark.incubator.apache.org/</url>
@@ -34,7 +34,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -48,12 +48,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</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-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/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/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
index 32d3934ac1..33b99f4bd3 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
@@ -77,7 +77,7 @@ public class JavaKMeansSuite implements Serializable {
@Test
public void runKMeansUsingStaticMethods() {
- List<double[]> points = new ArrayList();
+ List<double[]> points = new ArrayList<double[]>();
points.add(new double[]{1.0, 2.0, 6.0});
points.add(new double[]{1.0, 3.0, 0.0});
points.add(new double[]{1.0, 4.0, 6.0});
@@ -94,7 +94,7 @@ public class JavaKMeansSuite implements Serializable {
@Test
public void runKMeansUsingConstructor() {
- List<double[]> points = new ArrayList();
+ List<double[]> points = new ArrayList<double[]>();
points.add(new double[]{1.0, 2.0, 6.0});
points.add(new double[]{1.0, 3.0, 0.0});
points.add(new double[]{1.0, 4.0, 6.0});
diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
index eafee060cd..b40f552e0d 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
@@ -21,8 +21,6 @@ import java.io.Serializable;
import java.util.List;
import java.lang.Math;
-import scala.Tuple2;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
diff --git a/pom.xml b/pom.xml
index 9348c770fb..58fc02e029 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,11 +99,12 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
- <java.version>1.5</java.version>
- <scala.version>2.9.3</scala.version>
+ <java.version>1.6</java.version>
+
+ <scala.version>2.10.3</scala.version>
<mesos.version>0.13.0</mesos.version>
+ <akka.version>2.2.3</akka.version>
<akka.group>com.typesafe.akka</akka.group>
- <akka.version>2.0.5</akka.version>
<protobuf.version>2.4.1</protobuf.version>
<slf4j.version>1.7.2</slf4j.version>
<log4j.version>1.2.17</log4j.version>
@@ -116,10 +117,10 @@
</properties>
<repositories>
- <repository>
- <id>jboss-repo</id>
- <name>JBoss Repository</name>
- <url>http://repository.jboss.org/nexus/content/repositories/releases/</url>
+ <repository>
+ <id>maven-repo</id> <!-- This should be at top, it makes maven try the central repo first and then others and hence faster dep resolution -->
+ <name>Maven Repository</name>
+ <url>http://repo.maven.apache.org/maven2/</url>
<releases>
<enabled>true</enabled>
</releases>
@@ -128,20 +129,9 @@
</snapshots>
</repository>
<repository>
- <id>cloudera-repo</id>
- <name>Cloudera Repository</name>
- <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
- </repository>
- <repository>
- <id>akka-repo</id>
- <name>Akka Repository</name>
- <url>http://repo.akka.io/releases/</url>
+ <id>jboss-repo</id>
+ <name>JBoss Repository</name>
+ <url>http://repository.jboss.org/nexus/content/repositories/releases/</url>
<releases>
<enabled>true</enabled>
</releases>
@@ -161,41 +151,6 @@
</snapshots>
</repository>
</repositories>
- <pluginRepositories>
- <pluginRepository>
- <id>oss-sonatype-releases</id>
- <name>OSS Sonatype</name>
- <url>https://oss.sonatype.org/content/repositories/releases</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
- </pluginRepository>
- <pluginRepository>
- <id>oss-sonatype-snapshots</id>
- <name>OSS Sonatype</name>
- <url>https://oss.sonatype.org/content/repositories/snapshots</url>
- <releases>
- <enabled>false</enabled>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- </snapshots>
- </pluginRepository>
- <pluginRepository>
- <id>oss-sonatype</id>
- <name>OSS Sonatype</name>
- <url>https://oss.sonatype.org/content/groups/public</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- </snapshots>
- </pluginRepository>
- </pluginRepositories>
<dependencyManagement>
<dependencies>
@@ -251,7 +206,7 @@
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>chill_2.9.3</artifactId>
+ <artifactId>chill_2.10</artifactId>
<version>0.3.1</version>
</dependency>
<dependency>
@@ -261,7 +216,7 @@
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-actor</artifactId>
+ <artifactId>akka-actor_2.10</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -272,7 +227,7 @@
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-remote</artifactId>
+ <artifactId>akka-remote_2.10</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -283,7 +238,7 @@
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-slf4j</artifactId>
+ <artifactId>akka-slf4j_2.10</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -294,7 +249,7 @@
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-zeromq</artifactId>
+ <artifactId>akka-zeromq_2.10</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -321,7 +276,7 @@
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
- <version>4.0.0.Beta2</version>
+ <version>4.0.0.CR1</version>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
@@ -331,8 +286,14 @@
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
- <artifactId>lift-json_2.9.2</artifactId>
- <version>2.5</version>
+ <artifactId>lift-json_2.10</artifactId>
+ <version>2.5.1</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.scala-lang</groupId>
+ <artifactId>scalap</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>com.codahale.metrics</groupId>
@@ -375,24 +336,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_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</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>
@@ -406,7 +365,7 @@
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_2.10</artifactId>
<version>1.10.0</version>
<scope>test</scope>
</dependency>
@@ -539,6 +498,10 @@
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
</exclusions>
</dependency>
</dependencies>
@@ -779,39 +742,39 @@
</dependencyManagement>
</profile>
- <profile>
- <id>new-yarn</id>
- <properties>
- <akka.group>org.spark-project</akka.group>
- <akka.version>2.0.5-protobuf-2.5-java-1.5</akka.version>
- <hadoop.major.version>2</hadoop.major.version>
- <hadoop.version>2.2.0</hadoop.version>
- <protobuf.version>2.5.0</protobuf.version>
- </properties>
+ <!-- <profile> -->
+ <!-- <id>new-yarn</id> -->
+ <!-- <properties> -->
+ <!-- <akka.group>org.spark-project</akka.group> -->
+ <!-- <akka.version>2.0.5-protobuf-2.5-java-1.5</akka.version> -->
+ <!-- <hadoop.major.version>2</hadoop.major.version> -->
+ <!-- <hadoop.version>2.2.0</hadoop.version> -->
+ <!-- <protobuf.version>2.5.0</protobuf.version> -->
+ <!-- </properties> -->
- <modules>
- <module>new-yarn</module>
- </modules>
+ <!-- <modules> -->
+ <!-- <module>new-yarn</module> -->
+ <!-- </modules> -->
- <repositories>
- <repository>
- <id>maven-root</id>
- <name>Maven root repository</name>
- <url>http://repo1.maven.org/maven2/</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
- </repository>
- </repositories>
+ <!-- <repositories> -->
+ <!-- <repository> -->
+ <!-- <id>maven-root</id> -->
+ <!-- <name>Maven root repository</name> -->
+ <!-- <url>http://repo1.maven.org/maven2/</url> -->
+ <!-- <releases> -->
+ <!-- <enabled>true</enabled> -->
+ <!-- </releases> -->
+ <!-- <snapshots> -->
+ <!-- <enabled>false</enabled> -->
+ <!-- </snapshots> -->
+ <!-- </repository> -->
+ <!-- </repositories> -->
- <dependencyManagement>
- <dependencies>
- </dependencies>
- </dependencyManagement>
- </profile>
+ <!-- <dependencyManagement> -->
+ <!-- <dependencies> -->
+ <!-- </dependencies> -->
+ <!-- </dependencyManagement> -->
+ <!-- </profile> -->
<profile>
<id>repl-bin</id>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index ac87cffd9f..441dcc18fb 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -39,8 +39,8 @@ object SparkBuild extends Build {
val HBASE_VERSION = "0.94.6"
// Target JVM version
- val SCALAC_JVM_VERSION = "jvm-1.5"
- val JAVAC_JVM_VERSION = "1.5"
+ val SCALAC_JVM_VERSION = "jvm-1.6"
+ val JAVAC_JVM_VERSION = "1.6"
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects: _*)
@@ -78,18 +78,28 @@ object SparkBuild extends Build {
}
case Some(v) => v.toBoolean
}
+
lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match {
case None => DEFAULT_YARN
case Some(v) => v.toBoolean
}
+ if (isNewHadoop && isYarnEnabled) {
+ println( """Yarn with Hadoop version 2.2.x is not yet expected to work.
+ Please set env SPARK_HADOOP_VERSION to appropriate version or set SPARK_YARN to false.""")
+ throw new Exception("Yarn with Hadoop version 2.2.x is not yet expected to work.")
+ }
+
// Build against a protobuf-2.5 compatible Akka if Hadoop 2 is used.
- lazy val protobufVersion = if (isNewHadoop) "2.5.0" else "2.4.1"
- lazy val akkaVersion = if (isNewHadoop) "2.0.5-protobuf-2.5-java-1.5" else "2.0.5"
- lazy val akkaGroup = if (isNewHadoop) "org.spark-project" else "com.typesafe.akka"
+ // lazy val protobufVersion = if (isNewHadoop) "2.5.0" else "2.4.1"
+ // lazy val akkaVersion = if (isNewHadoop) "2.0.5-protobuf-2.5-java-1.5" else "2.0.5"
+ // lazy val akkaGroup = if (isNewHadoop) "org.spark-project" else "com.typesafe.akka"
// Conditionally include the yarn sub-project
- lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core)
+ //lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core)
+
+ lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
+
lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
@@ -99,9 +109,9 @@ object SparkBuild extends Build {
lazy val allProjects = packageProjects ++ Seq[ProjectReference](examples, tools, assemblyProj)
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.3",
scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation",
"-target:" + SCALAC_JVM_VERSION),
javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION),
@@ -121,12 +131,6 @@ object SparkBuild extends Build {
// also check the local Maven repository ~/.m2
resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))),
- // Shared between both core and streaming.
- resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"),
-
- // Shared between both examples and streaming.
- resolvers ++= Seq("Mqtt Repository" at "https://repo.eclipse.org/content/repositories/paho-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/"),
@@ -182,13 +186,17 @@ 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",
- "org.mockito" % "mockito-all" % "1.8.5" % "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",
+ "org.mockito" % "mockito-all" % "1.8.5" % "test",
+ "commons-io" % "commons-io" % "2.4" % "test"
),
+
+ parallelExecution := true,
/* 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) },
@@ -213,62 +221,62 @@ 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" % protobufVersion,
- akkaGroup % "akka-actor" % akkaVersion excludeAll(excludeNetty),
- akkaGroup % "akka-remote" % akkaVersion excludeAll(excludeNetty),
- akkaGroup % "akka-slf4j" % akkaVersion 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, excludeCglib),
- "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),
- "org.apache.zookeeper" % "zookeeper" % "3.4.5" 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.codahale.metrics" % "metrics-graphite" % "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,
+ "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-remote" % "2.2.3" excludeAll(excludeNetty),
+ "com.typesafe.akka" %% "akka-slf4j" % "2.2.3" 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, excludeCglib),
+ "org.apache.avro" % "avro" % "1.7.4",
+ "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty),
+ "org.apache.zookeeper" % "zookeeper" % "3.4.5" 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.codahale.metrics" % "metrics-graphite" % "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")
@@ -299,18 +307,21 @@ object SparkBuild extends Build {
def streamingSettings = sharedSettings ++ Seq(
name := "spark-streaming",
resolvers ++= Seq(
- "Akka Repository" at "http://repo.akka.io/releases/",
+ "Eclipse Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/",
"Apache repo" at "https://repository.apache.org/content/repositories/releases"
),
+
libraryDependencies ++= Seq(
- "org.eclipse.paho" % "mqtt-client" % "0.4.0",
- "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
- "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
- akkaGroup % "akka-zeromq" % akkaVersion excludeAll(excludeNetty),
- "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1"
+ "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
+ "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1"
exclude("com.sun.jdmk", "jmxtools")
exclude("com.sun.jmx", "jmxri")
exclude("net.sf.jopt-simple", "jopt-simple")
+ excludeAll(excludeNetty),
+ "org.eclipse.paho" % "mqtt-client" % "0.4.0",
+ "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty),
+ "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
+ "com.typesafe.akka" %% "akka-zeromq" % "2.2.3" excludeAll(excludeNetty)
)
)
diff --git a/pyspark b/pyspark
index 18f668ec97..8b34c98578 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 d8da02072c..61720dcf1a 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -978,7 +978,7 @@ 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,
@@ -986,7 +986,7 @@ class PipelinedRDD(RDD):
python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(),
bytearray(pickled_command), env, includes, self.preservesPartitioning,
self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator,
- class_manifest)
+ class_tag)
self._jrdd_val = python_rdd.asJavaRDD()
return self._jrdd_val
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index f6bf94be6b..c2a4efa36c 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl-bin_2.9.3</artifactId>
+ <artifactId>spark-repl-bin_2.10</artifactId>
<packaging>pom</packaging>
<name>Spark Project REPL binary packaging</name>
<url>http://spark.incubator.apache.org/</url>
@@ -40,18 +40,18 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_2.10</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_2.9.3</artifactId>
+ <artifactId>spark-repl_2.10</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
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 49d86621dd..bf06d730d4 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_2.9.3</artifactId>
+ <artifactId>spark-repl_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project REPL</name>
<url>http://spark.incubator.apache.org/</url>
@@ -39,18 +39,18 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_2.10</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_2.10</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
@@ -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_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_2.10</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-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/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..b2e1df173e
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
@@ -0,0 +1,109 @@
+/* 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
+
+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 0ced284da6..523fd1222d 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)
+ @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)
+ }
+ }
}
- */
-
- // 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
-
+ 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)) {
+
+ 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] = {
- // 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)
+ 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._
- 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
+ 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 = {
+ // 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 => {
@@ -657,23 +699,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() = {
- //replProps.power setValue true
- //power.unleash()
- //echo(power.banner)
+
+ def enablePowerMode(isDuringInit: Boolean) = {
+ // replProps.power setValue true
+ // unleashAndSetPhase()
+ // asyncEcho(isDuringInit, 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. */
@@ -688,11 +743,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"
@@ -700,23 +755,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))
}
}
@@ -731,7 +782,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 {
@@ -741,7 +792,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
@@ -755,10 +806,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.
@@ -773,28 +824,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
@@ -808,7 +843,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.
@@ -816,7 +851,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)
)
@@ -827,22 +862,71 @@ 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._")
+ 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 =>
+ // some post-initialization
+ chooseReader(settings) match {
+ case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x
+ case x => x
+ }
}
- echo("Type in expressions to have them evaluated.")
- echo("Type :help for more information.")
- }
+ 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
- var sparkContext: SparkContext = null
+ // 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")
@@ -856,85 +940,26 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val 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)
- try {
- sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
- } catch {
- case e: Exception =>
- e.printStackTrace()
- echo("Failed to create SparkContext, exiting...")
- sys.exit(1)
- }
+ 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
}
- 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(_))
-
- 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)
- }
-
- 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()
- }
- finally closeInterpreter()
- true
- }
-
/** 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)
}
@@ -943,15 +968,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 == '|')) ()
@@ -970,26 +997,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")
@@ -998,32 +1028,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 870e12de34..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,121 +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)
- }
- }
-
- // Get a copy of the local properties from SparkContext, and set it later in the thread
- // that triggers the execution. This is to make sure the caller of this function can pass
- // the right thread local (inheritable) properties down into Spark.
- val sc = org.apache.spark.repl.Main.interp.sparkContext
- val props = if (sc != null) sc.getLocalProperties() else null
-
- try {
- val execution = lineManager.set(originalLine) {
- // MATEI: set the right SparkEnv for our SparkContext, because
- // this execution will happen in a separate thread
- if (sc != null && sc.env != null) {
- SparkEnv.set(sc.env)
- sc.setLocalProperties(props)
- }
- // 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)
@@ -929,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),
@@ -1066,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
@@ -1109,7 +1263,7 @@ object SparkIMain {
code println postamble
}
}
-
+
trait StrippingWriter {
def isStripping: Boolean
def stripImpl(str: String): String
@@ -1119,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))
}
@@ -1139,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)) {
@@ -1163,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 6e4504d4d5..daaa2a0305 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -1,20 +1,3 @@
-/*
- * 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._
@@ -58,12 +41,12 @@ 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("propagation of local properties") {
@@ -95,96 +78,103 @@ class ReplSuite extends FunSuite {
System.clearProperty("spark.hostPort")
}
- 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
- """)
+ 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
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res1: Int = 55", output)
}
- 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(_+_)
- """)
+ 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(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
assertContains("res1: Int = 100", output)
}
- test ("external classes") {
- val output = runInterpreter("local", """
- class C {
- def foo = 5
- }
- sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
- """)
+ test("external classes") {
+ val output = runInterpreter("local",
+ """
+ |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") {
- val output = runInterpreter("local", """
- def double(x: Int) = x + x
- sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
- """)
+ test("external functions") {
+ val output = runInterpreter("local",
+ """
+ |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") {
- 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(_+_)
- """)
+ 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(_+_)
+ """.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
- """)
+ 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
+ """.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")
out.write("What's up?\n")
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))
+ val output = runInterpreter("local",
+ """
+ |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)
@@ -192,19 +182,20 @@ class ReplSuite extends FunSuite {
assertContains("res2: Long = 3", output)
}
- 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
- """)
+ 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
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
@@ -214,19 +205,20 @@ class ReplSuite extends FunSuite {
}
if (System.getenv("MESOS_NATIVE_LIBRARY") != null) {
- 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
- """)
+ 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
+ """.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 bbeca7f245..713404d077 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)"
@@ -135,3 +135,5 @@ 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 40892937b8..e27b437aae 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Streaming</name>
<url>http://spark.incubator.apache.org/</url>
@@ -48,7 +48,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -61,8 +61,8 @@
<version>1.9.11</version>
</dependency>
<dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka_2.9.2</artifactId>
+ <groupId>com.sksamuel.kafka</groupId>
+ <artifactId>kafka_2.10</artifactId>
<version>0.8.0-beta1</version>
<exclusions>
<exclusion>
@@ -111,16 +111,16 @@
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
- <artifactId>akka-zeromq</artifactId>
+ <artifactId>akka-zeromq_2.10</artifactId>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -134,14 +134,18 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.eclipse.paho</groupId>
<artifactId>mqtt-client</artifactId>
<version>0.4.0</version>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/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 9ceff754c4..329d2b5835 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}
@@ -37,7 +38,7 @@ import org.apache.hadoop.conf.Configuration
/**
* 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]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.rdd.RDD]]
* for more details on RDDs). DStreams can either be created from live data (such as, data from
* HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
* such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -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))
}
@@ -450,7 +451,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] = {
@@ -497,16 +498,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) => context.sparkContext.clean(transformFunc(r)))
}
@@ -514,7 +513,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))
val cleanedF = context.sparkContext.clean(transformFunc)
val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
@@ -528,7 +527,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 and 'other' DStream.
*/
- def transformWith[U: ClassManifest, V: ClassManifest](
+ def transformWith[U: ClassTag, V: ClassTag](
other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V]
): DStream[V] = {
val cleanedF = ssc.sparkContext.clean(transformFunc)
@@ -539,7 +538,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 and 'other' DStream.
*/
- def transformWith[U: ClassManifest, V: ClassManifest](
+ def transformWith[U: ClassTag, V: ClassTag](
other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V]
): DStream[V] = {
val cleanedF = ssc.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 b97fb7e6e3..6e9a781978 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
@@ -25,10 +25,10 @@ import org.apache.spark.SparkContext._
import scala.collection.mutable.HashMap
import scala.collection.mutable.Queue
+import scala.concurrent.duration._
import akka.actor._
import akka.pattern.ask
-import akka.util.duration._
import akka.dispatch._
import org.apache.spark.storage.BlockId
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 8c12fd11ef..80af96c060 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -18,16 +18,15 @@
package org.apache.spark.streaming
import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
-import org.apache.spark.streaming.dstream.{ShuffledDStream}
-import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
+import org.apache.spark.streaming.dstream._
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 +34,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 +104,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 +204,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 +335,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 +350,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 +366,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 +389,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
@@ -402,7 +401,7 @@ extends Serializable {
* Return a new DStream by applying a map function to the value of each key-value pairs in
* 'this' DStream without changing the key.
*/
- 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)
}
@@ -410,7 +409,7 @@ extends Serializable {
* Return a new DStream by applying a flatmap function to the value of each key-value pairs in
* 'this' DStream without changing the key.
*/
- def flatMapValues[U: ClassManifest](
+ def flatMapValues[U: ClassTag](
flatMapValuesFunc: V => TraversableOnce[U]
): DStream[(K, U)] = {
new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc)
@@ -421,7 +420,7 @@ extends Serializable {
* Hash partitioning is used to generate the RDDs with Spark's 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())
}
@@ -429,7 +428,7 @@ extends Serializable {
* Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
*/
- def cogroup[W: ClassManifest](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (Seq[V], Seq[W]))] = {
+ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner(numPartitions))
}
@@ -437,7 +436,7 @@ extends Serializable {
* Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
* The supplied [[org.apache.spark.Partitioner]] is used to partition the generated RDDs.
*/
- def cogroup[W: ClassManifest](
+ def cogroup[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (Seq[V], Seq[W]))] = {
@@ -451,7 +450,7 @@ extends Serializable {
* Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
* Hash partitioning is used to generate the RDDs with Spark's 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())
}
@@ -459,7 +458,7 @@ extends Serializable {
* Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
*/
- def join[W: ClassManifest](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (V, W))] = {
+ def join[W: ClassTag](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (V, W))] = {
join[W](other, defaultPartitioner(numPartitions))
}
@@ -467,7 +466,7 @@ extends Serializable {
* Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
* The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
*/
- def join[W: ClassManifest](
+ def join[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (V, W))] = {
@@ -482,7 +481,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
* number of partitions.
*/
- def leftOuterJoin[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, Option[W]))] = {
+ def leftOuterJoin[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (V, Option[W]))] = {
leftOuterJoin[W](other, defaultPartitioner())
}
@@ -491,7 +490,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
* partitions.
*/
- def leftOuterJoin[W: ClassManifest](
+ def leftOuterJoin[W: ClassTag](
other: DStream[(K, W)],
numPartitions: Int
): DStream[(K, (V, Option[W]))] = {
@@ -503,7 +502,7 @@ extends Serializable {
* `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
* the partitioning of each RDD.
*/
- def leftOuterJoin[W: ClassManifest](
+ def leftOuterJoin[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (V, Option[W]))] = {
@@ -518,7 +517,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
* number of partitions.
*/
- def rightOuterJoin[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Option[V], W))] = {
+ def rightOuterJoin[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Option[V], W))] = {
rightOuterJoin[W](other, defaultPartitioner())
}
@@ -527,7 +526,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
* partitions.
*/
- def rightOuterJoin[W: ClassManifest](
+ def rightOuterJoin[W: ClassTag](
other: DStream[(K, W)],
numPartitions: Int
): DStream[(K, (Option[V], W))] = {
@@ -539,7 +538,7 @@ extends Serializable {
* `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
* the partitioning of each RDD.
*/
- def rightOuterJoin[W: ClassManifest](
+ def rightOuterJoin[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (Option[V], W))] = {
@@ -556,8 +555,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]])
}
/**
@@ -586,8 +585,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]])
}
/**
@@ -609,9 +608,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 70bf902143..d2c4fdee65 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
/**
@@ -191,7 +193,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)
@@ -211,7 +213,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,
@@ -231,14 +233,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)
}
@@ -275,8 +277,8 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
*/
def kafkaStream[
- K: ClassManifest,
- V: ClassManifest,
+ K: ClassTag,
+ V: ClassTag,
U <: kafka.serializer.Decoder[_]: Manifest,
T <: kafka.serializer.Decoder[_]: Manifest](
kafkaParams: Map[String, String],
@@ -315,7 +317,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],
@@ -337,7 +339,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
}
@@ -352,7 +354,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
@@ -372,9 +374,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)
@@ -392,9 +394,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)
@@ -436,7 +438,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] = {
@@ -452,7 +454,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]
@@ -480,7 +482,7 @@ class StreamingContext private (
/**
* Create a unified DStream from multiple DStreams of the same type and same slide duration.
*/
- 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)
}
@@ -488,7 +490,7 @@ class StreamingContext private (
* Create a new DStream in which each RDD is generated by applying a function on RDDs of
* the DStreams.
*/
- def transform[T: ClassManifest](
+ def transform[T: ClassTag](
dstreams: Seq[DStream[_]],
transformFunc: (Seq[RDD[_]], Time) => RDD[T]
): DStream[T] = {
@@ -568,7 +570,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 1a2aeaa879..d29033df32 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,9 +23,11 @@ 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]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.rdd.RDD]]
* for more details on RDDs). DStreams can either be created from live data (such as, data from
* HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
* such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -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)
@@ -103,6 +105,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 09189eadd8..64f38ce1c0 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}
@@ -32,7 +33,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]
@@ -136,7 +137,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())
}
@@ -157,7 +158,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())
}
@@ -260,8 +261,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(_))
@@ -272,8 +273,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(_, _))
@@ -285,10 +286,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(_))
@@ -300,10 +301,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(_, _))
@@ -317,10 +318,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaDStream[U],
transformFunc: JFunction3[R, JavaRDD[U], Time, JavaRDD[W]]
): JavaDStream[W] = {
- implicit val cmu: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
- implicit val cmv: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cmu: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
+ implicit val cmv: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[W] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[U, W](other.dstream, scalaTransform(_, _, _))
@@ -334,12 +335,12 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaDStream[U],
transformFunc: JFunction3[R, JavaRDD[U], Time, JavaPairRDD[K2, V2]]
): JavaPairDStream[K2, V2] = {
- implicit val cmu: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
- implicit val cmk2: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv2: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+ implicit val cmu: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
+ implicit val cmk2: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv2: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[U, (K2, V2)](other.dstream, scalaTransform(_, _, _))
@@ -353,12 +354,12 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaPairDStream[K2, V2],
transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaRDD[W]]
): JavaDStream[W] = {
- implicit val cmk2: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv2: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
- implicit val cmw: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cmk2: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv2: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
+ implicit val cmw: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[(K2, V2), W](other.dstream, scalaTransform(_, _, _))
@@ -372,14 +373,14 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaPairDStream[K2, V2],
transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaPairRDD[K3, V3]]
): JavaPairDStream[K3, V3] = {
- implicit val cmk2: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv2: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
- implicit val cmk3: ClassManifest[K3] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K3]]
- implicit val cmv3: ClassManifest[V3] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V3]]
+ implicit val cmk2: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv2: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
+ implicit val cmk3: ClassTag[K3] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K3]]
+ implicit val cmv3: ClassTag[V3] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V3]]
def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[(K3, V3)] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[(K2, V2), (K3, V3)](other.dstream, 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 c6cd635afa..dfd6e27c3e 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 kManifest: ClassManifest[K],
- implicit val vManifest: ClassManifest[V])
+ implicit val kManifest: ClassTag[K],
+ implicit val vManifest: 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)
@@ -162,8 +163,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)
}
@@ -428,8 +429,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))
}
@@ -446,8 +447,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
numPartitions: Int)
: 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), numPartitions)
}
@@ -464,8 +465,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
partitioner: Partitioner
): 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), partitioner)
}
@@ -475,8 +476,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* 'this' DStream without changing the key.
*/
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)
}
@@ -487,8 +488,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
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)
}
@@ -498,8 +499,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))))
}
@@ -511,8 +512,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
numPartitions: Int
): 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, numPartitions)
.mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
}
@@ -525,8 +526,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
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))))
}
@@ -536,8 +537,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* Hash partitioning is used to generate the RDDs with Spark's 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)
}
@@ -546,8 +547,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
*/
def join[W](other: JavaPairDStream[K, W], numPartitions: Int): 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, numPartitions)
}
@@ -559,8 +560,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
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)
}
@@ -570,8 +571,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* number of partitions.
*/
def leftOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, Optional[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.leftOuterJoin(other.dstream)
joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
}
@@ -585,8 +586,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
numPartitions: Int
): JavaPairDStream[K, (V, Optional[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.leftOuterJoin(other.dstream, numPartitions)
joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
}
@@ -599,8 +600,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
partitioner: Partitioner
): JavaPairDStream[K, (V, Optional[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.leftOuterJoin(other.dstream, partitioner)
joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
}
@@ -611,8 +612,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* number of partitions.
*/
def rightOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (Optional[V], W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.rightOuterJoin(other.dstream)
joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
}
@@ -626,8 +627,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
numPartitions: Int
): JavaPairDStream[K, (Optional[V], W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.rightOuterJoin(other.dstream, numPartitions)
joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
}
@@ -641,8 +642,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
partitioner: Partitioner
): JavaPairDStream[K, (Optional[V], W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.rightOuterJoin(other.dstream, partitioner)
joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
}
@@ -722,24 +723,24 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
new JavaDStream[(K, V)](dstream)
}
- 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)]) = {
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 7f9dab0ef9..80dcf87491 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, List => JList}
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
@@ -141,10 +144,11 @@ class JavaStreamingContext(val ssc: StreamingContext) {
groupId: String,
topics: JMap[String, JInt])
: JavaPairDStream[String, 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)
+
}
/**
@@ -162,8 +166,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
topics: JMap[String, JInt],
storageLevel: StorageLevel)
: JavaPairDStream[String, 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)
}
@@ -189,10 +193,10 @@ class JavaStreamingContext(val ssc: StreamingContext) {
topics: JMap[String, JInt],
storageLevel: StorageLevel)
: JavaPairDStream[K, V] = {
- implicit val keyCmt: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val valueCmt: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val keyCmt: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val valueCmt: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
@@ -245,8 +249,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)
}
@@ -274,8 +278,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))
}
@@ -289,8 +293,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))
}
@@ -304,12 +308,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)
}
@@ -404,7 +408,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
@@ -422,8 +426,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)
}
@@ -443,8 +447,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)
}
@@ -462,8 +466,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)
}
@@ -480,12 +484,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)
}
@@ -505,9 +509,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)
}
@@ -525,9 +529,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)
}
@@ -547,8 +551,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)
@@ -564,8 +568,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)
@@ -585,8 +589,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)
@@ -597,7 +601,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
*/
def union[T](first: JavaDStream[T], rest: JList[JavaDStream[T]]): JavaDStream[T] = {
val dstreams: Seq[DStream[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream)
- implicit val cm: ClassManifest[T] = first.classManifest
+ implicit val cm: ClassTag[T] = first.classTag
ssc.union(dstreams)(cm)
}
@@ -609,9 +613,9 @@ class JavaStreamingContext(val ssc: StreamingContext) {
rest: JList[JavaPairDStream[K, V]]
): JavaPairDStream[K, V] = {
val dstreams: Seq[DStream[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream)
- 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.kManifest
+ implicit val vcm: ClassTag[V] = first.vManifest
new JavaPairDStream[K, V](ssc.union(dstreams)(cm))(kcm, vcm)
}
@@ -628,8 +632,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
dstreams: JList[JavaDStream[_]],
transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaRDD[T]]
): JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val scalaDStreams = dstreams.map(_.dstream).toSeq
val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList
@@ -651,10 +655,10 @@ class JavaStreamingContext(val ssc: StreamingContext) {
dstreams: JList[JavaDStream[_]],
transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaPairRDD[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]]
val scalaDStreams = dstreams.map(_.dstream).toSeq
val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
index a9a05c9981..f396c34758 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
@@ -19,11 +19,12 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Time, StreamingContext}
+import scala.reflect.ClassTag
/**
* An input stream that always returns the same RDD on each timestep. Useful for testing.
*/
-class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T])
+class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T])
extends InputDStream[T](ssc_) {
override def start() {}
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/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
index 91ee2c1a36..db2e0a4cee 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class FilteredDStream[T: ClassManifest](
+class FilteredDStream[T: ClassTag](
parent: DStream[T],
filterFunc: T => Boolean
) extends DStream[T](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
index ca7d7ca49e..244dc3ee4f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
@@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
+import scala.reflect.ClassTag
private[streaming]
-class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag](
parent: DStream[(K, V)],
flatMapValueFunc: V => TraversableOnce[U]
) extends DStream[(K, U)](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
index b37966f9a7..336c4b7a92 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class FlatMappedDStream[T: ClassManifest, U: ClassManifest](
+class FlatMappedDStream[T: ClassTag, U: ClassTag](
parent: DStream[T],
flatMapFunc: T => Traversable[U]
) extends DStream[U](parent.ssc) {
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 a0189eca04..60d79175f1 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/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
index e21bac4602..98b14cb224 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Duration, DStream, Job, Time}
+import scala.reflect.ClassTag
private[streaming]
-class ForEachDStream[T: ClassManifest] (
+class ForEachDStream[T: ClassTag] (
parent: DStream[T],
foreachFunc: (RDD[T], Time) => Unit
) extends DStream[Unit](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
index 4294b07d91..23136f44fa 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class GlommedDStream[T: ClassManifest](parent: DStream[T])
+class GlommedDStream[T: ClassTag](parent: DStream[T])
extends DStream[Array[T]](parent.ssc) {
override def dependencies = List(parent)
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 a5de5e1fb5..526f5564c7 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
@@ -31,11 +31,11 @@ import kafka.utils.ZKStringSerializer
import org.I0Itec.zkclient._
import scala.collection.Map
-
+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.
@@ -43,8 +43,8 @@ import scala.collection.Map
*/
private[streaming]
class KafkaInputDStream[
- K: ClassManifest,
- V: ClassManifest,
+ K: ClassTag,
+ V: ClassTag,
U <: Decoder[_]: Manifest,
T <: Decoder[_]: Manifest](
@transient ssc_ : StreamingContext,
@@ -61,8 +61,8 @@ class KafkaInputDStream[
private[streaming]
class KafkaReceiver[
- K: ClassManifest,
- V: ClassManifest,
+ K: ClassTag,
+ V: ClassTag,
U <: Decoder[_]: Manifest,
T <: Decoder[_]: Manifest](
kafkaParams: Map[String, String],
@@ -104,17 +104,18 @@ class KafkaReceiver[
tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id"))
}
- // Create Threads for each Topic/Message Stream we are listening
- val keyDecoder = manifest[U].erasure.getConstructor(classOf[VerifiableProperties])
+ val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
.newInstance(consumerConfig.props)
.asInstanceOf[Decoder[K]]
- val valueDecoder = manifest[T].erasure.getConstructor(classOf[VerifiableProperties])
+ val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
.newInstance(consumerConfig.props)
.asInstanceOf[Decoder[V]]
+ // Create Threads for each Topic/Message Stream we are listening
val topicMessageStreams = consumerConnector.createMessageStreams(
topics, keyDecoder, valueDecoder)
+
// Start the messages handler for each partition
topicMessageStreams.values.foreach { streams =>
streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) }
@@ -122,7 +123,7 @@ class KafkaReceiver[
}
// Handles Kafka Messages
- private class MessageHandler[K: ClassManifest, V: ClassManifest](stream: KafkaStream[K, V])
+ private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V])
extends Runnable {
def run() {
logInfo("Starting MessageHandler.")
@@ -146,7 +147,7 @@ class KafkaReceiver[
zk.deleteRecursive(dir)
zk.close()
} catch {
- case _ => // swallow
+ case _ : Throwable => // swallow
}
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
index ac0528213d..ef4a737568 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
@@ -37,6 +37,7 @@ import org.eclipse.paho.client.mqttv3.MqttTopic
import scala.collection.Map
import scala.collection.mutable.HashMap
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
/**
* Input stream that subscribe messages from a Mqtt Broker.
@@ -47,7 +48,7 @@ import scala.collection.JavaConversions._
*/
private[streaming]
-class MQTTInputDStream[T: ClassManifest](
+class MQTTInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
brokerUrl: String,
topic: String,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
index 5329601a6f..8a04060e5b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class MapPartitionedDStream[T: ClassManifest, U: ClassManifest](
+class MapPartitionedDStream[T: ClassTag, U: ClassTag](
parent: DStream[T],
mapPartFunc: Iterator[T] => Iterator[U],
preservePartitioning: Boolean
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
index 8290df90a2..0ce364fd46 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
@@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
+import scala.reflect.ClassTag
private[streaming]
-class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag](
parent: DStream[(K, V)],
mapValueFunc: V => U
) extends DStream[(K, U)](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
index b1682afea3..c0b7491d09 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class MappedDStream[T: ClassManifest, U: ClassManifest] (
+class MappedDStream[T: ClassTag, U: ClassTag] (
parent: DStream[T],
mapFunc: T => U
) extends DStream[U](parent.ssc) {
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 a82862c802..d5ae8aef92 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.{BlockId, StorageLevel, StreamBlockId}
* @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,8 +177,8 @@ 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 tracker = env.actorSystem.actorFor(url)
+ val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
+ val tracker = env.actorSystem.actorSelection(url)
val timeout = 5.seconds
override def preStart() {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
index 15782f5c11..6f9477020a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
@@ -18,9 +18,10 @@
package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.StreamingContext
+import scala.reflect.ClassTag
private[streaming]
-class PluggableInputDStream[T: ClassManifest](
+class PluggableInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
index 7d9f3521b1..97325f8ea3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
@@ -19,13 +19,13 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.UnionRDD
-
import scala.collection.mutable.Queue
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.streaming.{Time, StreamingContext}
+import scala.reflect.ClassTag
private[streaming]
-class QueueInputDStream[T: ClassManifest](
+class QueueInputDStream[T: ClassTag](
@transient ssc: StreamingContext,
val queue: Queue[RDD[T]],
oneAtATime: Boolean,
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 10ed4ef78d..dea0f26f90 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, StreamBlockId}
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/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
index a95e66d761..e6e0022097 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
@@ -21,9 +21,10 @@ import org.apache.spark.Partitioner
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
import org.apache.spark.streaming.{Duration, DStream, Time}
+import scala.reflect.ClassTag
private[streaming]
-class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
+class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
parent: DStream[(K,V)],
createCombiner: V => C,
mergeValue: (C, V) => C,
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/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
index 71bcb2b390..aeea060df7 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Duration, DStream, Time}
+import scala.reflect.ClassTag
private[streaming]
-class TransformedDStream[U: ClassManifest] (
+class TransformedDStream[U: ClassTag] (
parents: Seq[DStream[_]],
transformFunc: (Seq[RDD[_]], Time) => RDD[U]
) extends DStream[U](parents.head.ssc) {
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 ef0f85a717..fdf5371a89 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, StreamBlockId}
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..f164d516b0 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
@@ -17,7 +17,10 @@
package org.apache.spark.streaming.receivers
+import scala.reflect.ClassTag
+
import akka.actor.Actor
+import akka.util.ByteString
import akka.zeromq._
import org.apache.spark.Logging
@@ -25,12 +28,12 @@ import org.apache.spark.Logging
/**
* 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 ad4a8b9535..daeb99f5b7 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -21,28 +21,31 @@ import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
+
import kafka.serializer.StringDecoder;
+
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.spark.streaming.api.java.JavaDStreamLike;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+
import scala.Tuple2;
+import twitter4j.Status;
+
import org.apache.spark.HashPartitioner;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.*;
import org.apache.spark.storage.StorageLevel;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.dstream.SparkFlumeEvent;
import org.apache.spark.streaming.JavaTestUtils;
import org.apache.spark.streaming.JavaCheckpointTestUtils;
-import org.apache.spark.streaming.InputStreamsSuite;
import java.io.*;
import java.util.*;
@@ -51,7 +54,6 @@ import akka.actor.Props;
import akka.zeromq.Subscribe;
-
// The test suite itself is Serializable so that anonymous Function implementations can be
// serialized, as an alternative to converting these anonymous classes to static inner classes;
// see http://stackoverflow.com/questions/758570/.
@@ -86,8 +88,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(3L),
Arrays.asList(1L));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream count = stream.count();
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Long> count = stream.count();
JavaTestUtils.attachTestOutputStream(count);
List<List<Long>> result = JavaTestUtils.runStreams(ssc, 3, 3);
assertOrderInvariantEquals(expected, result);
@@ -103,8 +105,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(5,5),
Arrays.asList(9,4));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
@Override
public Integer call(String s) throws Exception {
return s.length();
@@ -129,8 +131,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(7,8,9,4,5,6),
Arrays.asList(7,8,9));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream windowed = stream.window(new Duration(2000));
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> windowed = stream.window(new Duration(2000));
JavaTestUtils.attachTestOutputStream(windowed);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
@@ -153,8 +155,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18),
Arrays.asList(13,14,15,16,17,18));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000));
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> windowed = stream.window(new Duration(4000), new Duration(2000));
JavaTestUtils.attachTestOutputStream(windowed);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 8, 4);
@@ -171,8 +173,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("giants"),
Arrays.asList("yankees"));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream filtered = stream.filter(new Function<String, Boolean>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> filtered = stream.filter(new Function<String, Boolean>() {
@Override
public Boolean call(String s) throws Exception {
return s.contains("a");
@@ -227,8 +229,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(Arrays.asList("giants", "dodgers")),
Arrays.asList(Arrays.asList("yankees", "red socks")));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream glommed = stream.glom();
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<List<String>> glommed = stream.glom();
JavaTestUtils.attachTestOutputStream(glommed);
List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
@@ -245,8 +247,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("GIANTSDODGERS"),
Arrays.asList("YANKEESRED SOCKS"));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream mapped = stream.mapPartitions(new FlatMapFunction<Iterator<String>, String>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> mapped = stream.mapPartitions(new FlatMapFunction<Iterator<String>, String>() {
@Override
public Iterable<String> call(Iterator<String> in) {
String out = "";
@@ -288,8 +290,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(15),
Arrays.asList(24));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream reduced = stream.reduce(new IntegerSum());
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> reduced = stream.reduce(new IntegerSum());
JavaTestUtils.attachTestOutputStream(reduced);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
@@ -309,8 +311,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(39),
Arrays.asList(24));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(),
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> reducedWindowed = stream.reduceByWindow(new IntegerSum(),
new IntegerDifference(), new Duration(2000), new Duration(1000));
JavaTestUtils.attachTestOutputStream(reducedWindowed);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
@@ -695,8 +697,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"),
Arrays.asList("a","t","h","l","e","t","i","c","s"));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
return Lists.newArrayList(x.split("(?!^)"));
@@ -742,8 +744,8 @@ public class JavaAPISuite implements Serializable {
new Tuple2<Integer, String>(9, "c"),
new Tuple2<Integer, String>(9, "s")));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction<String, Integer, String>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<Integer,String> flatMapped = stream.flatMap(new PairFlatMapFunction<String, Integer, String>() {
@Override
public Iterable<Tuple2<Integer, String>> call(String in) throws Exception {
List<Tuple2<Integer, String>> out = Lists.newArrayList();
@@ -776,10 +778,10 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(2,2,5,5),
Arrays.asList(3,3,6,6));
- JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
- JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
+ JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
+ JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
- JavaDStream unioned = stream1.union(stream2);
+ JavaDStream<Integer> unioned = stream1.union(stream2);
JavaTestUtils.attachTestOutputStream(unioned);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
@@ -790,7 +792,7 @@ public class JavaAPISuite implements Serializable {
* Performs an order-invariant comparison of lists representing two RDD streams. This allows
* us to account for ordering variation within individual RDD's which occurs during windowing.
*/
- public static <T extends Comparable> void assertOrderInvariantEquals(
+ public static <T extends Comparable<T>> void assertOrderInvariantEquals(
List<List<T>> expected, List<List<T>> actual) {
for (List<T> list: expected) {
Collections.sort(list);
@@ -813,11 +815,11 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(new Tuple2<String, Integer>("giants", 6)),
Arrays.asList(new Tuple2<String, Integer>("yankees", 7)));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaPairDStream<String, Integer> pairStream = stream.map(
new PairFunction<String, String, Integer>() {
@Override
- public Tuple2 call(String in) throws Exception {
+ public Tuple2<String, Integer> call(String in) throws Exception {
return new Tuple2<String, Integer>(in, in.length());
}
});
@@ -1540,8 +1542,8 @@ public class JavaAPISuite implements Serializable {
File tempDir = Files.createTempDir();
ssc.checkpoint(tempDir.getAbsolutePath());
- JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ JavaDStream<String> stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
@Override
public Integer call(String s) throws Exception {
return s.length();
@@ -1616,7 +1618,7 @@ public class JavaAPISuite implements Serializable {
@Test
public void testSocketTextStream() {
- JavaDStream test = ssc.socketTextStream("localhost", 12345);
+ JavaDStream<String> test = ssc.socketTextStream("localhost", 12345);
}
@Test
@@ -1636,7 +1638,7 @@ public class JavaAPISuite implements Serializable {
}
}
- JavaDStream test = ssc.socketStream(
+ JavaDStream<String> test = ssc.socketStream(
"localhost",
12345,
new Converter(),
@@ -1645,39 +1647,39 @@ public class JavaAPISuite implements Serializable {
@Test
public void testTextFileStream() {
- JavaDStream test = ssc.textFileStream("/tmp/foo");
+ JavaDStream<String> test = ssc.textFileStream("/tmp/foo");
}
@Test
public void testRawSocketStream() {
- JavaDStream test = ssc.rawSocketStream("localhost", 12345);
+ JavaDStream<String> test = ssc.rawSocketStream("localhost", 12345);
}
@Test
public void testFlumeStream() {
- JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
+ JavaDStream<SparkFlumeEvent> test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
}
@Test
public void testFileStream() {
JavaPairDStream<String, String> foo =
- ssc.<String, String, SequenceFileInputFormat>fileStream("/tmp/foo");
+ ssc.<String, String, SequenceFileInputFormat<String,String>>fileStream("/tmp/foo");
}
@Test
public void testTwitterStream() {
String[] filters = new String[] { "good", "bad", "ugly" };
- JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY());
+ JavaDStream<Status> test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY());
}
@Test
public void testActorStream() {
- JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
+ JavaDStream<String> test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
}
@Test
public void testZeroMQStream() {
- JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
+ JavaDStream<String> test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
@Override
public Iterable<String> call(byte[][] b) throws Exception {
return null;
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 5e384eeee4..42ab9590d6 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 TestOutputStreamWithPartitions(dstream.dstream)
dstream.dstream.ssc.registerOutputStream(ostream)
}
@@ -67,8 +69,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)))
@@ -85,8 +87,8 @@ trait JavaTestBase extends TestSuiteBase {
*/
def runStreamsWithPartitions[V](ssc: JavaStreamingContext, numBatches: Int,
numExpectedOutput: Int): JList[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 = runStreamsWithPartitions[V](ssc.ssc, numBatches, numExpectedOutput)
val out = new ArrayList[JList[JList[V]]]()
res.map{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 beb20831bd..e81287b44e 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 8c8c359e6e..2f34e812a1 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() {}
@@ -63,7 +64,7 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[
*
* The buffer contains a sequence of RDD's, each containing a sequence of items
*/
-class TestOutputStream[T: ClassManifest](parent: DStream[T],
+class TestOutputStream[T: ClassTag](parent: DStream[T],
val output: ArrayBuffer[Seq[T]] = ArrayBuffer[Seq[T]]())
extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.collect()
@@ -85,7 +86,7 @@ class TestOutputStream[T: ClassManifest](parent: DStream[T],
* The buffer contains a sequence of RDD's, each containing a sequence of partitions, each
* containing a sequence of items.
*/
-class TestOutputStreamWithPartitions[T: ClassManifest](parent: DStream[T],
+class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T],
val output: ArrayBuffer[Seq[Seq[T]]] = ArrayBuffer[Seq[Seq[T]]]())
extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.glom().collect().map(_.toSeq)
@@ -133,7 +134,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],
numPartitions: Int = numInputPartitions
@@ -159,7 +160,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]
@@ -190,7 +191,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
*
* Returns a sequence of items for each RDD.
*/
- def runStreams[V: ClassManifest](
+ def runStreams[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int
@@ -207,7 +208,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Returns a sequence of RDD's. Each RDD is represented as several sequences of items, each
* representing one partition.
*/
- def runStreamsWithPartitions[V: ClassManifest](
+ def runStreamsWithPartitions[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int
@@ -263,7 +264,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
@@ -293,7 +294,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]],
@@ -311,7 +312,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]],
@@ -328,7 +329,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],
@@ -348,7 +349,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 f1c489beea..2bad494fc5 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -25,7 +25,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-tools_2.9.3</artifactId>
+ <artifactId>spark-tools_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Tools</name>
<url>http://spark.incubator.apache.org/</url>
@@ -33,24 +33,24 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</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-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
index f824c472ae..f670f65bf5 100644
--- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
@@ -199,7 +199,7 @@ object JavaAPICompletenessChecker {
private def toJavaMethod(method: SparkMethod): SparkMethod = {
val params = method.parameters
- .filterNot(_.name == "scala.reflect.ClassManifest")
+ .filterNot(_.name == "scala.reflect.ClassTag")
.map(toJavaType(_, isReturnType = false))
SparkMethod(method.name, toJavaType(method.returnType, isReturnType = true), params)
}
@@ -212,7 +212,7 @@ object JavaAPICompletenessChecker {
// internal Spark components.
val excludedNames = Seq(
"org.apache.spark.rdd.RDD.origin",
- "org.apache.spark.rdd.RDD.elementClassManifest",
+ "org.apache.spark.rdd.RDD.elementClassTag",
"org.apache.spark.rdd.RDD.checkpointData",
"org.apache.spark.rdd.RDD.partitioner",
"org.apache.spark.rdd.RDD.partitions",
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 8a065c6d7d..d18ac3736d 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -25,7 +25,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn_2.9.3</artifactId>
+ <artifactId>spark-yarn_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project YARN Support</name>
<url>http://spark.incubator.apache.org/</url>
@@ -33,7 +33,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_2.10</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -63,7 +63,7 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_2.10</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -74,8 +74,8 @@
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-2.10/classes</outputDirectory>
+ <testOutputDirectory>target/scala-2.10/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
index 421a83c87a..69038844bb 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
@@ -27,10 +27,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
import akka.actor._
-import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
-import akka.remote.RemoteClientShutdown
+import akka.remote._
import akka.actor.Terminated
-import akka.remote.RemoteClientDisconnected
import org.apache.spark.{SparkContext, Logging}
import org.apache.spark.util.{Utils, AkkaUtils}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
@@ -55,19 +53,18 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
// This actor just working as a monitor to watch on Driver Actor.
class MonitorActor(driverUrl: String) extends Actor {
- var driver: ActorRef = null
+ var driver: ActorSelection = null
override def preStart() {
logInfo("Listen to driver: " + driverUrl)
- driver = context.actorFor(driverUrl)
+ driver = context.actorSelection(driverUrl)
driver ! "hello"
- 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])
}
override def receive = {
- case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
- logInfo("Driver terminated or disconnected! Shutting down.")
+ case x: DisassociatedEvent =>
+ logInfo(s"Driver terminated or disconnected! Shutting down. $x")
driverClosed = true
}
}
@@ -168,7 +165,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte
System.setProperty("spark.driver.host", driverHost)
System.setProperty("spark.driver.port", driverPort.toString)
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
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 f15f3c7c11..9ab2073529 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
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResp
import org.apache.hadoop.yarn.util.{RackResolver, Records}
-object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+object AllocationType extends Enumeration {
type AllocationType = Value
val HOST, RACK, ANY = Value
}
@@ -238,7 +238,7 @@ private[yarn] class YarnAllocationHandler(
// 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"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)