From ad88f083a627ba38e99b1b135a82a1fcfd107444 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 24 Apr 2013 18:08:26 +0530 Subject: scala 2.10 and master merge --- .../scala/spark/streaming/NetworkInputTracker.scala | 4 ++-- .../spark/streaming/dstream/NetworkInputDStream.scala | 6 ++++-- .../spark/streaming/receivers/ActorReceiver.scala | 19 +++++++++---------- .../spark/streaming/receivers/ZeroMQReceiver.scala | 2 +- 4 files changed, 16 insertions(+), 15 deletions(-) (limited to 'streaming') diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index b159d26c02..e5bb654578 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -11,8 +11,8 @@ import scala.collection.mutable.Queue import akka.actor._ import akka.pattern.ask -import akka.util.duration._ -import akka.dispatch._ +import scala.concurrent.duration._ +// import akka.dispatch._ private[streaming] sealed trait NetworkInputTrackerMessage private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 7385474963..5347374730 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -7,13 +7,15 @@ import spark.rdd.BlockRDD import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ import java.nio.ByteBuffer import akka.actor.{Props, Actor} import akka.pattern.ask -import akka.dispatch.Await -import akka.util.duration._ +import scala.concurrent.Await +import akka.util.Timeout + import spark.streaming.util.{RecurringTimer, SystemClock} import java.util.concurrent.ArrayBlockingQueue diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index b3201d0b28..6c9e373de3 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -3,6 +3,8 @@ package 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 spark.storage.StorageLevel import spark.streaming.dstream.NetworkReceiver @@ -12,9 +14,6 @@ import java.util.concurrent.atomic.AtomicInteger /** 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 @@ -27,10 +26,10 @@ object ReceiverSupervisorStrategy { * pushBlock API. * * @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") @@ -74,12 +73,12 @@ 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") * }}} diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index 5533c3cf1e..e7608f08ae 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -13,7 +13,7 @@ private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String, bytesToObjects: Seq[Seq[Byte]] ⇒ 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 = { -- cgit v1.2.3 From bb4102b0eefd7321d1fadf9df6db79c8dd9880fb Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 25 Apr 2013 14:38:01 +0530 Subject: Fixed breaking tests in streaming checkpoint suite. Changed RichInt to Int as it is final and not serializable --- .../scala/spark/streaming/CheckpointSuite.scala | 35 ++++++++++++---------- 1 file changed, 19 insertions(+), 16 deletions(-) (limited to 'streaming') diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index cac86deeaf..f9285b19e2 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -1,16 +1,19 @@ package spark.streaming -import dstream.FileInputDStream -import spark.streaming.StreamingContext._ import java.io.File -import runtime.RichInt -import org.scalatest.BeforeAndAfter + +import scala.collection.mutable.ArrayBuffer + 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 spark.streaming.StreamingContext.toPairDStreamFunctions +import spark.streaming.dstream.FileInputDStream +import spark.streaming.util.ManualClock + + /** * This test suites tests the checkpointing functionality of DStreams - @@ -56,13 +59,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 @@ -162,13 +165,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) } @@ -350,4 +353,4 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]] outputStream.output } -} \ No newline at end of file +} -- cgit v1.2.3 From 4b4a36ea7d7f9e1d9c9ee1d6738deea579dc1b4e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 12:55:43 +0530 Subject: Fixed pom.xml with updated dependencies. --- README.md | 5 ++--- examples/pom.xml | 2 +- project/SparkBuild.scala | 4 ++-- streaming/pom.xml | 12 +++++++++++- 4 files changed, 16 insertions(+), 7 deletions(-) (limited to 'streaming') diff --git a/README.md b/README.md index ba24ab43b1..60f3a7576c 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.2 (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 package diff --git a/examples/pom.xml b/examples/pom.xml index 270777e29c..f521e85027 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ com.twitter - algebird-core_2.9.2 + algebird-core_2.10 0.1.11 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7b61e2ba3e..49c8870019 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -170,7 +170,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") + libraryDependencies ++= Seq("com.twitter" %% "algebird-core" % "0.1.11") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") @@ -178,7 +178,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", "org.twitter4j" % "twitter4j-stream" % "3.0.3", "com.typesafe.akka" % "akka-zeromq" % "2.1-M1" excludeAll(ExclusionRule(name = "akka-actor"), ExclusionRule(organization = "org.scala-lang")) diff --git a/streaming/pom.xml b/streaming/pom.xml index b0d0cd0ff3..26f0ae951c 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -55,7 +55,17 @@ com.typesafe.akka akka-zeromq - 2.0.3 + 2.1-M1 + + + org.scala-lang + scala-library + + + com.typesafe.akka + akka-actor + + org.scalatest -- cgit v1.2.3 From 8f3ac240cbdd678c0c76155b080dcc461355452e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 16:39:13 +0530 Subject: Fixed Warning: ClassManifest -> ClassTag --- core/src/main/scala/spark/PairRDDFunctions.scala | 31 ++++----- core/src/main/scala/spark/Partitioner.scala | 8 ++- core/src/main/scala/spark/RDD.scala | 43 ++++++------- core/src/main/scala/spark/RDDCheckpointData.scala | 6 +- .../scala/spark/SequenceFileRDDFunctions.scala | 9 +-- core/src/main/scala/spark/SparkContext.scala | 61 +++++++++--------- core/src/main/scala/spark/Utils.scala | 9 ++- .../main/scala/spark/api/java/JavaDoubleRDD.scala | 9 +-- .../main/scala/spark/api/java/JavaPairRDD.scala | 59 ++++++++--------- core/src/main/scala/spark/api/java/JavaRDD.scala | 14 ++--- .../main/scala/spark/api/java/JavaRDDLike.scala | 31 ++++----- .../scala/spark/api/java/JavaSparkContext.scala | 65 +++++++++---------- .../spark/api/java/function/FlatMapFunction.scala | 4 +- .../main/scala/spark/api/python/PythonRDD.scala | 7 ++- core/src/main/scala/spark/rdd/BlockRDD.scala | 4 +- core/src/main/scala/spark/rdd/CartesianRDD.scala | 5 +- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 3 +- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 3 +- core/src/main/scala/spark/rdd/FilteredRDD.scala | 3 +- core/src/main/scala/spark/rdd/FlatMappedRDD.scala | 3 +- core/src/main/scala/spark/rdd/GlommedRDD.scala | 3 +- .../main/scala/spark/rdd/MapPartitionsRDD.scala | 3 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 3 +- core/src/main/scala/spark/rdd/MappedRDD.scala | 3 +- .../scala/spark/rdd/ParallelCollectionRDD.scala | 8 ++- .../main/scala/spark/rdd/PartitionPruningRDD.scala | 5 +- core/src/main/scala/spark/rdd/PipedRDD.scala | 3 +- core/src/main/scala/spark/rdd/SampledRDD.scala | 5 +- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 5 +- core/src/main/scala/spark/rdd/UnionRDD.scala | 5 +- core/src/main/scala/spark/rdd/ZippedRDD.scala | 8 ++- .../main/scala/spark/scheduler/DAGScheduler.scala | 9 +-- core/src/test/scala/spark/CheckpointSuite.scala | 5 +- .../spark/streaming/examples/ActorWordCount.scala | 3 +- .../src/main/scala/spark/streaming/DStream.scala | 27 ++++---- .../spark/streaming/DStreamCheckpointData.scala | 7 ++- .../spark/streaming/PairDStreamFunctions.scala | 43 +++++++------ .../scala/spark/streaming/StreamingContext.scala | 35 ++++++----- .../spark/streaming/api/java/JavaDStream.scala | 8 ++- .../spark/streaming/api/java/JavaDStreamLike.scala | 33 +++++----- .../spark/streaming/api/java/JavaPairDStream.scala | 57 ++++++++--------- .../streaming/api/java/JavaStreamingContext.scala | 73 +++++++++++----------- .../spark/streaming/dstream/CoGroupedDStream.scala | 4 +- .../streaming/dstream/ConstantInputDStream.scala | 6 +- .../spark/streaming/dstream/FileInputDStream.scala | 12 ++-- .../spark/streaming/dstream/FilteredDStream.scala | 6 +- .../streaming/dstream/FlatMapValuedDStream.scala | 4 +- .../streaming/dstream/FlatMappedDStream.scala | 5 +- .../streaming/dstream/FlumeInputDStream.scala | 3 +- .../spark/streaming/dstream/ForEachDStream.scala | 3 +- .../spark/streaming/dstream/GlommedDStream.scala | 4 +- .../spark/streaming/dstream/InputDStream.scala | 4 +- .../streaming/dstream/KafkaInputDStream.scala | 8 +-- .../streaming/dstream/MapPartitionedDStream.scala | 5 +- .../spark/streaming/dstream/MapValuedDStream.scala | 5 +- .../spark/streaming/dstream/MappedDStream.scala | 5 +- .../streaming/dstream/NetworkInputDStream.scala | 5 +- .../streaming/dstream/PluggableInputDStream.scala | 4 +- .../streaming/dstream/QueueInputDStream.scala | 12 ++-- .../spark/streaming/dstream/RawInputDStream.scala | 4 +- .../streaming/dstream/ReducedWindowedDStream.scala | 10 +-- .../spark/streaming/dstream/ShuffledDStream.scala | 4 +- .../streaming/dstream/SocketInputDStream.scala | 6 +- .../spark/streaming/dstream/StateDStream.scala | 4 +- .../streaming/dstream/TransformedDStream.scala | 4 +- .../spark/streaming/dstream/UnionDStream.scala | 6 +- .../spark/streaming/dstream/WindowedDStream.scala | 7 +-- .../spark/streaming/receivers/ActorReceiver.scala | 14 +++-- .../spark/streaming/receivers/ZeroMQReceiver.scala | 4 +- .../spark/streaming/util/MasterFailureTest.scala | 31 +++++---- .../test/java/spark/streaming/JavaTestUtils.scala | 22 ++++--- .../scala/spark/streaming/CheckpointSuite.scala | 5 +- .../test/scala/spark/streaming/TestSuiteBase.scala | 25 ++++---- 73 files changed, 531 insertions(+), 438 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 07efba9e8d..0453e98e7d 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -7,6 +7,7 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ +import scala.reflect.{ ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -29,7 +30,7 @@ import spark.Partitioner._ * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( +class PairRDDFunctions[K: ClassTag, V: ClassTag]( self: RDD[(K, V)]) extends Logging with HadoopMapReduceUtil @@ -394,7 +395,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val cg = new CoGroupedRDD[K]( Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]), 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]]) @@ -415,7 +416,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( other1.asInstanceOf[RDD[(K, _)]], other2.asInstanceOf[RDD[(K, _)]]), 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]]) @@ -469,19 +470,19 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Return an RDD with the pairs from `this` whose keys are not in `other`. - * + * * 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) /** @@ -510,7 +511,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * 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]) { + def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -518,7 +519,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * 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]) { + def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -644,15 +645,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Return an RDD with the keys of each tuple. */ def keys: RDD[K] = self.map(_._1) - + /** * Return an RDD with the values of each tuple. */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure + private[spark] def getKeyClass() = implicitly[ClassTag[K]].erasure - private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure + private[spark] def getValueClass() = implicitly[ClassTag[V]].erasure } /** @@ -660,7 +661,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * an implicit conversion. Import `spark.SparkContext._` at the top of your program to 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]( +class OrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( self: RDD[(K, V)]) extends Logging with Serializable { @@ -704,6 +705,6 @@ class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U] } } -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/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 6f8cd17c88..fc811d85e1 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -1,5 +1,7 @@ package spark +import scala.reflect.ClassTag + /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. @@ -60,7 +62,7 @@ class HashPartitioner(partitions: Int) extends Partitioner { } } } - + override def equals(other: Any): Boolean = other match { case h: HashPartitioner => h.numPartitions == numPartitions @@ -73,10 +75,10 @@ class HashPartitioner(partitions: Int) extends Partitioner { * A [[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[(K,V)], - private val ascending: Boolean = true) + private val ascending: Boolean = true) extends Partitioner { // An array of upper bounds for the first (partitions - 1) partitions diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index ccd9d0364a..e6e0997a59 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -8,6 +8,7 @@ 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.NullWritable @@ -65,7 +66,7 @@ import SparkContext._ * [[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 { @@ -213,13 +214,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)) /** @@ -307,25 +308,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) } @@ -349,7 +350,7 @@ abstract class RDD[T: ClassManifest]( /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U], + def mapPartitions[U: ClassTag](f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) @@ -357,7 +358,7 @@ abstract class RDD[T: ClassManifest]( * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[U: ClassManifest]( + def mapPartitionsWithIndex[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) @@ -367,7 +368,7 @@ abstract class RDD[T: ClassManifest]( * of the original partition. */ @deprecated("use mapPartitionsWithIndex", "0.7.0") - def mapPartitionsWithSplit[U: ClassManifest]( + def mapPartitionsWithSplit[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) @@ -377,7 +378,7 @@ abstract class RDD[T: ClassManifest]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def mapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) + def mapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false) (f:(T, A) => U): RDD[U] = { def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { val a = constructA(index) @@ -391,7 +392,7 @@ abstract class RDD[T: ClassManifest]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def flatMapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) + def flatMapWith[A: ClassTag, U: ClassTag](constructA: Int => A, preservesPartitioning: Boolean = false) (f:(T, A) => Seq[U]): RDD[U] = { def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { val a = constructA(index) @@ -405,7 +406,7 @@ abstract class RDD[T: ClassManifest]( * This additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def foreachWith[A: ClassManifest](constructA: Int => A) + def foreachWith[A: ClassTag](constructA: Int => A) (f:(T, A) => Unit) { def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { val a = constructA(index) @@ -419,7 +420,7 @@ abstract class RDD[T: ClassManifest]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def filterWith[A: ClassManifest](constructA: Int => A) + def filterWith[A: ClassTag](constructA: Int => A) (p:(T, A) => Boolean): RDD[T] = { def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { val a = constructA(index) @@ -434,7 +435,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) // Actions (launch a job to return a value to the user program) @@ -470,7 +471,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) } @@ -560,7 +561,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) @@ -607,7 +608,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.erasure.isArray) { throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. @@ -638,7 +639,7 @@ abstract class RDD[T: ClassManifest]( timeout: Long, confidence: Double = 0.95 ): PartialResult[Map[T, BoundedDouble]] = { - if (elementClassManifest.erasure.isArray) { + if (elementClassTag.erasure.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => @@ -751,12 +752,12 @@ abstract class RDD[T: ClassManifest]( /** Record user function generating this RDD. */ private[spark] val origin = Utils.getSparkCallSite - 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]] } diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index d00092e984..083ba9b8fa 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -1,7 +1,11 @@ package spark +import scala.reflect.ClassTag + import org.apache.hadoop.fs.Path + import rdd.{CheckpointRDD, CoalescedRDD} + import scheduler.{ResultTask, ShuffleMapTask} /** @@ -19,7 +23,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/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 518034e07b..900d73bf42 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -11,6 +11,7 @@ import java.util.Date import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map import scala.collection.mutable.HashMap +import scala.reflect.{ classTag, ClassTag} import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat @@ -32,15 +33,15 @@ import spark.SparkContext._ * * Users should import `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].erasure)) { + classTag[T].erasure } 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/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 4957a54c1b..6bd87bf3ec 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -8,6 +8,7 @@ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ +import scala.reflect.{ ClassTag, classTag} import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration @@ -210,19 +211,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) } @@ -265,7 +266,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, * {{{ @@ -273,7 +274,7 @@ 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]], @@ -283,7 +284,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, * {{{ @@ -291,12 +292,12 @@ 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]], @@ -359,11 +360,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() @@ -382,7 +383,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] = { @@ -391,17 +392,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 @@ -569,7 +570,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], @@ -589,7 +590,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], @@ -604,7 +605,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], @@ -616,21 +617,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) @@ -641,7 +642,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) @@ -745,14 +746,14 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings - implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = + implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = new PairRDDFunctions(rdd) - implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( + implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = new SequenceFileRDDFunctions(rdd) - implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( + implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = new OrderedRDDFunctions(rdd) @@ -777,16 +778,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].erasure.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].erasure.asInstanceOf[Class[W]] new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } @@ -834,11 +835,11 @@ 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/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 81daacf958..cdccb8b336 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -4,14 +4,19 @@ import java.io._ import java.net._ import java.util.{Locale, Random, UUID} import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} + import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.io.Source +import scala.reflect.ClassTag +import scala.Some + import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import scala.Some + import spark.serializer.SerializerInstance /** @@ -207,7 +212,7 @@ private 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/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 16692c0440..3e88a9ccc7 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -1,5 +1,6 @@ package spark.api.java +import scala.reflect.ClassTag import spark.RDD import spark.SparkContext.doubleRDDToDoubleRDDFunctions import spark.api.java.function.{Function => JFunction} @@ -11,7 +12,7 @@ import spark.Partitioner class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { - override val classManifest: ClassManifest[Double] = implicitly[ClassManifest[Double]] + override val classTag: ClassTag[Double] = implicitly[ClassTag[Double]] override val rdd: RDD[Double] = srdd.map(x => Double.valueOf(x)) @@ -25,7 +26,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. */ @@ -65,7 +66,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. */ @@ -123,7 +124,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav /** Return the approximate sum of the elements in this RDD. */ def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - + /** Return the approximate sum of the elements in this RDD. */ def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) } diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 30084df4e2..89c6d05383 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -5,6 +5,7 @@ import java.util.Comparator import scala.Tuple2 import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat @@ -23,13 +24,13 @@ import spark.Partitioner._ import spark.RDD import spark.SparkContext.rddToPairRDDFunctions -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._ @@ -38,7 +39,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. */ @@ -94,14 +95,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. @@ -113,8 +114,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val cm: ClassManifest[C] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]] + implicit val cm: ClassTag[C] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(rdd.combineByKey( createCombiner, mergeValue, @@ -151,14 +152,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. */ @@ -214,7 +215,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. */ @@ -271,15 +272,15 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif : JavaPairRDD[K, (Option[V], W)] = fromRDD(rdd.rightOuterJoin(other, partitioner)) - /** + /** * 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))) } @@ -362,8 +363,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * this also retains the original RDD's partitioning. */ def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = { - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.mapValues(f)) } @@ -374,8 +375,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.flatMapValues(fn)) } @@ -541,22 +542,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif } object JavaPairRDD { - def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassManifest[K], - vcm: ClassManifest[T]): RDD[(K, JList[T])] = + def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassTag[K], + vcm: ClassTag[T]): RDD[(K, JList[T])] = rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _) - def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassManifest[K], - vcm: ClassManifest[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V], + def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K], + vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V], Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1], - Seq[W2]))])(implicit kcm: ClassManifest[K]) : RDD[(K, (JList[V], JList[W1], + Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1], JList[W2]))] = rddToPairRDDFunctions(rdd).mapValues( (x: (Seq[V], Seq[W1], Seq[W2])) => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) - def fromRDD[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = + def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd) implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index e29f1e5899..032506383c 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -1,10 +1,11 @@ package spark.api.java +import scala.reflect.ClassTag import spark._ import spark.api.java.function.{Function => JFunction} import 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) @@ -14,7 +15,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): JavaRDD[T] = wrapRDD(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. */ @@ -31,7 +32,7 @@ JavaRDDLike[T, JavaRDD[T]] { * Return a new RDD containing the distinct elements in this RDD. */ def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions)) - + /** * Return a new RDD containing only the elements that satisfy a predicate. */ @@ -54,7 +55,7 @@ JavaRDDLike[T, JavaRDD[T]] { */ def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) - + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -63,7 +64,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** * 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. */ @@ -85,8 +86,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/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d884529d7a..a6555081b3 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -3,6 +3,7 @@ package spark.api.java import java.util.{List => JList} import scala.Tuple2 import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import spark.{SparkContext, Partition, RDD, TaskContext} import spark.api.java.JavaPairRDD._ @@ -15,7 +16,7 @@ import com.google.common.base.Optional 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] @@ -57,7 +58,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()) } @@ -88,7 +89,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()) } @@ -128,18 +129,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * elements (a, b) where a is in `this` and b is in `other`. */ def cartesian[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = - JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest, - other.classManifest) + JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, + other.classTag) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val vcm: ClassManifest[JList[T]] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]] + implicit val kcm: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[JList[T]] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm) } @@ -148,10 +149,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * mapping to that key. */ def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val vcm: ClassManifest[JList[T]] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]] + implicit val kcm: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[JList[T]] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm) } @@ -179,7 +180,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) } // Actions (launch a job to return a value to the user program) @@ -304,7 +305,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/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 5f18b1e15b..b633273eaf 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -4,6 +4,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 @@ -63,8 +64,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } @@ -75,10 +76,10 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int) : JavaPairRDD[K, V] = { - implicit val kcm: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val vcm: ClassManifest[V] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] + implicit val kcm: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)) } @@ -113,16 +114,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)) } @@ -134,8 +135,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path, minSplits)(cm) } @@ -147,8 +148,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path)(cm) } @@ -164,8 +165,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)) } @@ -180,8 +181,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)) } @@ -193,8 +194,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)) } @@ -205,8 +206,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)) } @@ -221,8 +222,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)) } @@ -235,15 +236,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) } @@ -251,9 +252,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) } @@ -386,8 +387,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/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala index e027cdacd3..e69480607e 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala @@ -1,5 +1,7 @@ package spark.api.java.function +import scala.reflect.ClassTag + /** * A function that returns zero or more output records from each input record. */ @@ -7,5 +9,5 @@ abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] @throws(classOf[Exception]) def call(x: T) : java.lang.Iterable[R] - def elementType() : ClassManifest[R] = ClassManifest.Any.asInstanceOf[ClassManifest[R]] + def elementType() : ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 9b4d54ab4e..82959a33eb 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -6,6 +6,7 @@ import java.util.{List => JList, ArrayList => JArrayList, Collections} import scala.collection.JavaConversions._ import scala.io.Source +import scala.reflect.ClassTag import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import spark.broadcast.Broadcast @@ -13,7 +14,7 @@ import spark._ import spark.rdd.PipedRDD -private[spark] class PythonRDD[T: ClassManifest]( +private[spark] class PythonRDD[T: ClassTag]( parent: RDD[T], command: Seq[String], envVars: java.util.Map[String, String], @@ -251,7 +252,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 } } @@ -277,7 +278,7 @@ private class BytesToString extends spark.api.java.function.Function[Array[Byte] */ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { - + override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]]) diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 7348c4f15b..f44d37a91f 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,6 +1,7 @@ package spark.rdd import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { @@ -8,7 +9,7 @@ private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Pa } private[spark] -class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) +class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = { @@ -37,4 +38,3 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St locations_(split.asInstanceOf[BlockRDDPartition].blockId) } - diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 38600b8be4..20a7487c48 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,6 +1,9 @@ package spark.rdd import java.io.{ObjectOutputStream, IOException} + +import scala.reflect.ClassTag + import spark._ @@ -26,7 +29,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/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 5db77eb142..700a4160c8 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark._ import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter} import org.apache.hadoop.conf.Configuration @@ -15,7 +16,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/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 6d862c0c28..0ca678e4af 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} @@ -26,7 +27,7 @@ private[spark] case class CoalescedRDDPartition( * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, * or to avoid having a large number of small tasks when processing a directory with many files. */ -class CoalescedRDD[T: ClassManifest]( +class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index c84ec39d21..eee72520d9 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,8 +1,9 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{OneToOneDependency, RDD, Partition, TaskContext} -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/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 8ebc778925..ab39ed6b5d 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,10 +1,11 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} 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/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index e16c7ba881..8783aa2e86 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,8 +1,9 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} -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/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index d283c5b2bb..685eefa09e 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,10 +1,11 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false) diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala index afb7504ba1..2a043c994b 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, Partition, TaskContext} @@ -9,7 +10,7 @@ import spark.{RDD, Partition, TaskContext} * information such as the number of tuples in a partition. */ private[spark] -class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsWithIndexRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index af07311b6d..79762e46a2 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,9 +1,10 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{RDD, 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/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 07585a88ce..81dbcde227 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -3,9 +3,11 @@ package spark.rdd import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer import scala.collection.Map +import scala.reflect.ClassTag + import spark.{RDD, TaskContext, SparkContext, Partition} -private[spark] class ParallelCollectionPartition[T: ClassManifest]( +private[spark] class ParallelCollectionPartition[T: ClassTag]( val rddId: Long, val slice: Int, values: Seq[T]) @@ -23,7 +25,7 @@ private[spark] class ParallelCollectionPartition[T: ClassManifest]( override val index: Int = slice } -private[spark] class ParallelCollectionRDD[T: ClassManifest]( +private[spark] class ParallelCollectionRDD[T: ClassTag]( @transient sc: SparkContext, @transient data: Seq[T], numSlices: Int, @@ -53,7 +55,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/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..0afea33f8e 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} @@ -29,7 +30,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))) { @@ -49,6 +50,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/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 962a1b21ad..34d32eb85a 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -7,6 +7,7 @@ import scala.collection.Map import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source +import scala.reflect.ClassTag import spark.{RDD, SparkEnv, Partition, TaskContext} @@ -15,7 +16,7 @@ import spark.{RDD, SparkEnv, Partition, TaskContext} * 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/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 243673f151..a503088f61 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -1,5 +1,6 @@ package spark.rdd +import scala.reflect.ClassTag import java.util.Random import cern.jet.random.Poisson @@ -12,9 +13,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/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 481e03b349..5e56900b18 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -1,8 +1,11 @@ package spark.rdd import java.util.{HashMap => JHashMap} + import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + import spark.RDD import spark.Partitioner import spark.Dependency @@ -28,7 +31,7 @@ import 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[(K, V)], @transient var rdd2: RDD[(K, W)], part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 2c52a67e22..b4b21c0aac 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,10 +1,11 @@ package spark.rdd import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag import spark.{Dependency, RangeDependency, RDD, 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) @@ -23,7 +24,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/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 35b0e06785..1b438cd505 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,10 +1,12 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} +import scala.reflect.ClassTag + +import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} -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] @@ -25,7 +27,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/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c54dce51d7..b838cf84a8 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -6,6 +6,7 @@ import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.reflect.ClassTag import spark._ import spark.executor.TaskMetrics @@ -215,7 +216,7 @@ class DAGScheduler( * The job is assumed to have at least one partition; zero partition jobs should be handled * without a JobSubmitted event. */ - private[scheduler] def prepareJob[T, U: ClassManifest]( + private[scheduler] def prepareJob[T, U: ClassTag]( finalRdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -231,7 +232,7 @@ class DAGScheduler( return (toSubmit, waiter) } - def runJob[T, U: ClassManifest]( + def runJob[T, U: ClassTag]( finalRdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -326,7 +327,7 @@ class DAGScheduler( submitStage(stage) } } - + /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. @@ -712,7 +713,7 @@ class DAGScheduler( sizeBefore = shuffleToMapStage.size shuffleToMapStage.clearOldValues(cleanupTime) logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size) - + sizeBefore = pendingTasks.size pendingTasks.clearOldValues(cleanupTime) logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index ca385972fb..8836c68ae6 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -1,5 +1,6 @@ package spark +import scala.reflect.ClassTag import org.scalatest.FunSuite import java.io.File import spark.rdd._ @@ -179,7 +180,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 @@ -248,7 +249,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/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 3b847fe603..deb1c8511a 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -2,6 +2,7 @@ package spark.streaming.examples import scala.collection.mutable.LinkedList import scala.util.Random +import scala.reflect.ClassTag import akka.actor.Actor import akka.actor.ActorRef @@ -65,7 +66,7 @@ class FeederActor extends Actor { * * @see [[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) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e1be5ef51c..c307c69611 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -9,6 +9,7 @@ import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import java.io.{ObjectInputStream, IOException, ObjectOutputStream} @@ -36,7 +37,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 { @@ -62,7 +63,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 @@ -254,16 +255,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 => { @@ -280,7 +281,7 @@ abstract class DStream[T: ClassManifest] ( } generatedRDDs.put(time, newRDD) Some(newRDD) - case None => + case None => None } } else { @@ -324,7 +325,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) { @@ -396,7 +397,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)) } @@ -404,7 +405,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)) } @@ -423,7 +424,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] = { @@ -474,7 +475,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] => RDD[U]): DStream[U] = { + def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { transform((r: RDD[T], t: Time) => transformFunc(r)) } @@ -482,7 +483,7 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD is generated by applying a function * on each RDD of this DStream. */ - def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { + def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { new TransformedDStream(this, context.sparkContext.clean(transformFunc)) } diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index 6b0fade7c6..82e5bb6e49 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -3,13 +3,15 @@ package 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 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]() @@ -90,4 +92,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/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 3ec922957d..962ba6619d 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -5,18 +5,19 @@ import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} -import spark.{Manifests, RDD, Partitioner, HashPartitioner} +import spark.{ClassTags, RDD, Partitioner, HashPartitioner} import spark.SparkContext._ import 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} import org.apache.hadoop.mapred.OutputFormat 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 @@ -86,7 +87,7 @@ extends Serializable { * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more * information. */ - def combineByKey[C: ClassManifest]( + def combineByKey[C: ClassTag]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiner: (C, C) => C, @@ -186,7 +187,7 @@ extends Serializable { * DStream's batching interval */ def reduceByKeyAndWindow( - reduceFunc: (V, V) => V, + reduceFunc: (V, V) => V, windowDuration: Duration, slideDuration: Duration ): DStream[(K, V)] = { @@ -317,7 +318,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()) @@ -332,7 +333,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)] = { @@ -348,7 +349,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)] = { @@ -371,7 +372,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 @@ -380,11 +381,11 @@ extends Serializable { } - def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = { + def mapValues[U: ClassTag](mapValuesFunc: V => U): DStream[(K, U)] = { new MapValuedDStream[K, V, U](self, mapValuesFunc) } - def flatMapValues[U: ClassManifest]( + def flatMapValues[U: ClassTag]( flatMapValuesFunc: V => TraversableOnce[U] ): DStream[(K, U)] = { new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc) @@ -396,7 +397,7 @@ extends Serializable { * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number * of partitions. */ - def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner()) } @@ -405,7 +406,7 @@ extends Serializable { * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that * key in both RDDs. Partitioner is used to partition each generated RDD. */ - def cogroup[W: ClassManifest]( + def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner ): DStream[(K, (Seq[V], Seq[W]))] = { @@ -415,8 +416,8 @@ extends Serializable { partitioner ) val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)( - classManifest[K], - Manifests.seqSeqManifest + classTag[K], + ClassTags.seqSeqClassTag ) pdfs.mapValues { case Seq(vs, ws) => @@ -428,7 +429,7 @@ extends Serializable { * Join `this` DStream with `other` DStream. HashPartitioner is used * to partition each generated RDD into default number of partitions. */ - def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = { + def join[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (V, W))] = { join[W](other, defaultPartitioner()) } @@ -437,7 +438,7 @@ extends Serializable { * be generated by joining RDDs from `this` and other DStream. Uses the given * Partitioner to partition each generated RDD. */ - def join[W: ClassManifest]( + def join[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner ): DStream[(K, (V, W))] = { @@ -455,7 +456,7 @@ extends Serializable { def saveAsHadoopFiles[F <: OutputFormat[K, V]]( prefix: String, suffix: String - )(implicit fm: ClassManifest[F]) { + )(implicit fm: ClassTag[F]) { saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -485,7 +486,7 @@ extends Serializable { def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]]( prefix: String, suffix: String - )(implicit fm: ClassManifest[F]) { + )(implicit fm: ClassTag[F]) { saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } @@ -508,9 +509,7 @@ extends Serializable { self.foreach(saveFunc) } - private def getKeyClass() = implicitly[ClassManifest[K]].erasure + private def getKeyClass() = implicitly[ClassTag[K]].erasure - private def getValueClass() = implicitly[ClassManifest[V]].erasure + private def getValueClass() = implicitly[ClassTag[V]].erasure } - - diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index b8b60aab43..7646e15521 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -16,6 +16,7 @@ import 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 @@ -166,7 +167,7 @@ class StreamingContext private ( * Create an input stream with any arbitrary user implemented network receiver. * @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) @@ -185,7 +186,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, @@ -203,7 +204,7 @@ 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], @@ -225,7 +226,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def kafkaStream[T: ClassManifest]( + def kafkaStream[T: ClassTag]( zkQuorum: String, groupId: String, topics: Map[String, Int], @@ -264,7 +265,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], @@ -286,7 +287,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 } @@ -301,7 +302,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 @@ -321,9 +322,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) @@ -341,9 +342,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) @@ -385,7 +386,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] = { @@ -400,7 +401,7 @@ class StreamingContext private ( * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. 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] @@ -413,7 +414,7 @@ class StreamingContext private ( /** * Create a unified DStream from multiple DStreams of the same type and same interval */ - def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = { + def union[T: ClassTag](streams: Seq[DStream[T]]): DStream[T] = { new UnionDStream[T](streams.toArray) } @@ -490,7 +491,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/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 4d93f0a5f7..535a302d60 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -6,6 +6,8 @@ import spark.api.java.JavaRDD import spark.storage.StorageLevel import spark.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 [[spark.RDD]] @@ -24,7 +26,7 @@ import spark.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) @@ -80,6 +82,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) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 548809a359..d67dd34760 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -4,6 +4,7 @@ import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import spark.streaming._ import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} @@ -14,7 +15,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] @@ -116,7 +117,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()) } @@ -137,7 +138,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()) } @@ -240,8 +241,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(_)) @@ -252,8 +253,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(_, _)) @@ -265,10 +266,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(_)) @@ -280,10 +281,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(_, _)) @@ -296,4 +297,4 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def checkpoint(interval: Duration) = { dstream.checkpoint(interval) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 30240cad98..4c93a7e440 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -4,6 +4,7 @@ import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import spark.streaming._ import spark.streaming.StreamingContext._ @@ -18,8 +19,8 @@ import com.google.common.base.Optional import spark.RDD class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( - implicit val kManifiest: ClassManifest[K], - implicit val vManifest: ClassManifest[V]) + implicit val kTag: ClassTag[K], + implicit val vTag: ClassTag[V]) extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) @@ -138,8 +139,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) } @@ -407,8 +408,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)) } @@ -421,7 +422,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], numPartitions: Int) : JavaPairDStream[K, S] = { @@ -437,7 +438,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. * @tparam S State type */ - def updateStateByKey[S: ClassManifest]( + def updateStateByKey[S: ClassTag]( updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], partitioner: Partitioner ): JavaPairDStream[K, S] = { @@ -445,16 +446,16 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = { - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.mapValues(f) } def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassManifest[U] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]] + implicit val cm: ClassTag[U] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.flatMapValues(fn) } @@ -465,8 +466,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)))) } @@ -477,8 +478,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def cogroup[W](other: JavaPairDStream[K, W], partitioner: Partitioner) : JavaPairDStream[K, (JList[V], JList[W])] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.cogroup(other.dstream, partitioner) .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) } @@ -488,8 +489,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * to partition each generated RDD into default number of partitions. */ def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.join(other.dstream) } @@ -500,8 +501,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def join[W](other: JavaPairDStream[K, W], partitioner: Partitioner) : JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + implicit val cm: ClassTag[W] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] dstream.join(other.dstream, partitioner) } @@ -575,24 +576,24 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf) } - override val classManifest: ClassManifest[(K, V)] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]] + override val classTag: ClassTag[(K, V)] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]] } object JavaPairDStream { - implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)]) + implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) :JavaPairDStream[K, V] = new JavaPairDStream[K, V](dstream) def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = { - implicit val cmk: ClassManifest[K] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] - implicit val cmv: ClassManifest[V] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] + implicit val cmk: ClassTag[K] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val cmv: ClassTag[V] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] new JavaPairDStream[K, V](dstream.dstream) } - def scalaToJavaLong[K: ClassManifest](dstream: JavaPairDStream[K, Long]) + def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long]) : JavaPairDStream[K, JLong] = { StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 3d149a742c..00e5aa0603 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,6 +17,7 @@ import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream @@ -126,8 +127,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { groupId: String, topics: JMap[String, JInt]) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } @@ -146,8 +147,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { topics: JMap[String, JInt], initialOffsets: JMap[KafkaPartitionKey, JLong]) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.kafkaStream[T]( zkQuorum, groupId, @@ -172,8 +173,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { initialOffsets: JMap[KafkaPartitionKey, JLong], 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]] ssc.kafkaStream[T]( zkQuorum, groupId, @@ -224,8 +225,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) } @@ -253,8 +254,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)) } @@ -268,8 +269,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)) } @@ -283,12 +284,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); } @@ -372,8 +373,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) } @@ -393,8 +394,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) } @@ -412,8 +413,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) } @@ -434,8 +435,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.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy) } @@ -455,8 +456,8 @@ 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]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) } @@ -475,8 +476,8 @@ 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]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn) } @@ -497,8 +498,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[spark.RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue) @@ -514,8 +515,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[spark.RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime) @@ -535,8 +536,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[spark.RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala index 4ef4bb7de1..3e974b2731 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala @@ -4,8 +4,10 @@ import spark.{RDD, Partitioner} import spark.rdd.CoGroupedRDD import spark.streaming.{Time, DStream, Duration} +import scala.reflect.ClassTag + private[streaming] -class CoGroupedDStream[K : ClassManifest]( +class CoGroupedDStream[K : ClassTag]( parents: Seq[DStream[(K, _)]], partitioner: Partitioner ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala index 41c3af4694..780f361869 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala @@ -3,10 +3,12 @@ package spark.streaming.dstream import spark.RDD import 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() {} @@ -16,4 +18,4 @@ class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T override def compute(validTime: Time): Option[RDD[T]] = { Some(rdd) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 41b9bd9461..78e62ca1d2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -9,14 +9,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 @@ -37,7 +39,7 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K } logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly) } - + override def stop() { } /** @@ -83,7 +85,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) @@ -178,5 +180,3 @@ private[streaming] object FileInputDStream { def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") } - - diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala index e993164f99..3f6054994e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.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) { @@ -17,5 +19,3 @@ class FilteredDStream[T: ClassManifest]( parent.getOrCompute(validTime).map(_.filter(filterFunc)) } } - - diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala index cabd34f5f2..66d28b9841 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -4,8 +4,10 @@ import spark.streaming.{Duration, DStream, Time} import spark.RDD import 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/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala index a69af60589..b946d9c689 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.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) { @@ -17,4 +19,3 @@ class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( parent.getOrCompute(validTime).map(_.flatMap(flatMapFunc)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index c9644b3a83..640033c86c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -12,13 +12,14 @@ import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import java.net.InetSocketAddress import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.nio.ByteBuffer private[streaming] -class FlumeInputDStream[T: ClassManifest]( +class FlumeInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, host: String, port: Int, diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala index ee69ea5177..2494273ce6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala @@ -2,9 +2,10 @@ package spark.streaming.dstream import spark.RDD import 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/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala index b589cbd4d5..658c49b61b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.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/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index 3c5d43a609..f9451281d6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -2,6 +2,8 @@ package spark.streaming.dstream import 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/ @@ -13,7 +15,7 @@ import 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/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index ddd9becf32..e093edb05b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -16,14 +16,14 @@ import kafka.utils.ZkUtils._ import scala.collection.Map import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ - +import scala.reflect.ClassTag // Key for a specific Kafka Partition: (broker, topic, group, part) case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int) /** * Input stream that pulls messages from a Kafka Broker. - * + * * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed @@ -33,7 +33,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part * @param storageLevel RDD storage level. */ private[streaming] -class KafkaInputDStream[T: ClassManifest]( +class KafkaInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, zkQuorum: String, groupId: String, @@ -51,7 +51,7 @@ class KafkaInputDStream[T: ClassManifest]( private[streaming] class KafkaReceiver(zkQuorum: String, groupId: String, - topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], + topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel) extends NetworkReceiver[Any] { // Timeout for establishing a connection to Zookeper in ms. diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala index 848afecfad..382739932b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.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 @@ -18,4 +20,3 @@ class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( parent.getOrCompute(validTime).map(_.mapPartitions[U](mapPartFunc, preservePartitioning)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala index 6055aa6a05..9c508eb854 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala @@ -4,8 +4,10 @@ import spark.streaming.{Duration, DStream, Time} import spark.RDD import 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) { @@ -18,4 +20,3 @@ class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( parent.getOrCompute(validTime).map(_.mapValues[U](mapValueFunc)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala index 20818a0cab..c7cbf6cc46 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.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) { @@ -17,4 +19,3 @@ class MappedDStream[T: ClassManifest, U: ClassManifest] ( parent.getOrCompute(validTime).map(_.map[U](mapFunc)) } } - diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 5347374730..52b9968f6e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -8,6 +8,7 @@ import spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ +import scala.reflect.ClassTag import java.nio.ByteBuffer @@ -28,7 +29,7 @@ import java.util.concurrent.ArrayBlockingQueue * @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 @@ -70,7 +71,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 * [[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() diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala index 3c2a81947b..70deec6ec9 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -2,8 +2,10 @@ package spark.streaming.dstream import 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/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index da224ad6f7..8b03a49fd6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -7,18 +7,20 @@ import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer import 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, defaultRDD: RDD[T] ) extends InputDStream[T](ssc) { - + override def start() { } - + override def stop() { } - + override def compute(validTime: Time): Option[RDD[T]] = { val buffer = new ArrayBuffer[RDD[T]]() if (oneAtATime && queue.size > 0) { @@ -38,5 +40,5 @@ class QueueInputDStream[T: ClassManifest]( None } } - + } diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 1b2fa56779..7d20ac8e95 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -4,6 +4,8 @@ import spark.Logging import spark.storage.StorageLevel import spark.streaming.StreamingContext +import scala.reflect.ClassTag + import java.net.InetSocketAddress import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, SocketChannel} @@ -18,7 +20,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/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 343b6915e7..c9ada5b306 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -8,11 +8,13 @@ import spark.Partitioner import spark.SparkContext._ import spark.storage.StorageLevel -import scala.collection.mutable.ArrayBuffer import 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, @@ -32,7 +34,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) @@ -153,5 +155,3 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( } } } - - diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala index 1f9548bfb8..930e6c2044 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala @@ -4,8 +4,10 @@ import spark.{RDD, Partitioner} import spark.SparkContext._ import 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/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index 1408af0afa..c9e701e07f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -4,11 +4,13 @@ import spark.streaming.StreamingContext import spark.storage.StorageLevel import 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, @@ -22,7 +24,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/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala index db62955036..f6fb2ca941 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala @@ -6,8 +6,10 @@ import spark.SparkContext._ import spark.storage.StorageLevel import 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/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala index 99660d9dee..23ab87d82f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala @@ -3,8 +3,10 @@ package spark.streaming.dstream import spark.RDD import spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag + private[streaming] -class TransformedDStream[T: ClassManifest, U: ClassManifest] ( +class TransformedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], transformFunc: (RDD[T], Time) => RDD[U] ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala index 00bad5da34..3a5bf35249 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala @@ -2,11 +2,13 @@ package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} import spark.RDD -import collection.mutable.ArrayBuffer import 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/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala index cbf0c88108..2382210b0e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala @@ -5,8 +5,10 @@ import spark.rdd.UnionRDD import spark.storage.StorageLevel import 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) @@ -35,6 +37,3 @@ class WindowedDStream[T: ClassManifest]( Some(new UnionRDD(ssc.sc, parent.slice(currentWindow))) } } - - - diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index 6c9e373de3..42e885af5c 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -4,14 +4,16 @@ 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 spark.storage.StorageLevel import spark.streaming.dstream.NetworkReceiver import java.util.concurrent.atomic.AtomicInteger -/** A helper with set of defaults for supervisor strategy **/ +/** A helper with set of defaults for supervisor strategy */ object ReceiverSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = @@ -43,11 +45,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) } @@ -61,8 +63,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. @@ -85,7 +87,7 @@ private[streaming] case class Data[T: ClassManifest](data: T) * * */ -private[streaming] class ActorReceiver[T: ClassManifest]( +private[streaming] class ActorReceiver[T: ClassTag]( props: Props, name: String, storageLevel: StorageLevel, diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index e7608f08ae..45e2596707 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -5,10 +5,12 @@ import akka.zeromq._ import spark.Logging +import scala.reflect.ClassTag + /** * A receiver to subscribe to ZeroMQ stream. */ -private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String, +private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, subscribe: Subscribe, bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T]) extends Actor with Receiver with Logging { diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index f673e5be15..3db1eaa834 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -7,6 +7,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 @@ -102,7 +103,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], @@ -140,7 +141,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] @@ -173,7 +174,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 @@ -254,7 +255,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), @@ -285,7 +286,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]( @@ -359,22 +360,22 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val hadoopFile = new Path(testDir, (i+1).toString) FileUtils.writeStringToFile(localFile, input(i).toString + "\n") var tries = 0 - var done = false + var done = false while (!done && tries < maxTries) { tries += 1 try { fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) - done = true - } catch { - case ioe: IOException => { - fs = testDir.getFileSystem(new Configuration()) + 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() @@ -388,5 +389,3 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) } } } - - diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 64a7e7cbf9..8a7c48bde6 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -1,6 +1,8 @@ package spark.streaming -import collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.reflect.ClassTag + import java.util.{List => JList} import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} import spark.streaming._ @@ -13,15 +15,15 @@ trait JavaTestBase extends TestSuiteBase { /** * Create a [[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) @@ -30,12 +32,12 @@ trait JavaTestBase extends TestSuiteBase { /** * Attach a provided stream to it's associated StreamingContext as a * [[spark.streaming.TestOutputStream]]. - **/ + */ def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R], R <: spark.api.java.JavaRDDLike[T, R]]( dstream: JavaDStreamLike[T, This, R]) = { - implicit val cm: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val ostream = new TestOutputStream(dstream.dstream, new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]]) dstream.dstream.ssc.registerOutputStream(ostream) @@ -48,8 +50,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))) @@ -64,4 +66,4 @@ object JavaTestUtils extends JavaTestBase { object JavaCheckpointTestUtils extends JavaTestBase { override def actuallyWait = true -} \ No newline at end of file +} diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index f9285b19e2..143a26d911 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -3,6 +3,7 @@ package spark.streaming import java.io.File import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter @@ -297,7 +298,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]], @@ -340,7 +341,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/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index ad6aa79d10..dc280b09c9 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -5,8 +5,9 @@ import spark.streaming.util.ManualClock import spark.{RDD, Logging} -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} @@ -17,7 +18,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} * 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() {} @@ -43,7 +44,7 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[ * This is a output stream just for the testsuites. All the output is collected into a * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. */ -class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBuffer[Seq[T]]) +class TestOutputStream[T: ClassTag](parent: DStream[T], val output: ArrayBuffer[Seq[T]]) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.collect() output += collected @@ -88,7 +89,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * Set up required DStreams to test the DStream operation using the two sequences * of input collections. */ - def setupStreams[U: ClassManifest, V: ClassManifest]( + def setupStreams[U: ClassTag, V: ClassTag]( input: Seq[Seq[U]], operation: DStream[U] => DStream[V] ): StreamingContext = { @@ -112,7 +113,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] @@ -140,7 +141,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { * returns the collected output. It will wait until `numExpectedOutput` number of * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached. */ - def runStreams[V: ClassManifest]( + def runStreams[V: ClassTag]( ssc: StreamingContext, numBatches: Int, numExpectedOutput: Int @@ -196,7 +197,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 @@ -226,7 +227,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]], @@ -244,7 +245,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]], @@ -261,7 +262,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], @@ -281,7 +282,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], -- cgit v1.2.3 From d3518f57cd833f6297860c6344e685b9bf0210f5 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 18:14:25 +0530 Subject: Fixed warning: erasure -> runtimeClass --- core/src/main/scala/spark/PairRDDFunctions.scala | 8 ++++---- core/src/main/scala/spark/RDD.scala | 4 ++-- .../main/scala/spark/SequenceFileRDDFunctions.scala | 4 ++-- core/src/main/scala/spark/SparkContext.scala | 18 +++++++++--------- .../scala/spark/streaming/PairDStreamFunctions.scala | 8 ++++---- 5 files changed, 21 insertions(+), 21 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 0453e98e7d..2052d05788 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -512,7 +512,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( * supporting the key and value types K and V in this RDD. */ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -520,7 +520,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( * (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: ClassTag[F]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -651,9 +651,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassTag[K]].erasure + private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass - private[spark] def getValueClass() = implicitly[ClassTag[V]].erasure + private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass } /** diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6e0997a59..6ee075315a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -608,7 +608,7 @@ abstract class RDD[T: ClassTag]( * combine step happens locally on the master, equivalent to running a single reduce task. */ def countByValue(): Map[T, Long] = { - if (elementClassTag.erasure.isArray) { + if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. @@ -639,7 +639,7 @@ abstract class RDD[T: ClassTag]( timeout: Long, confidence: Double = 0.95 ): PartialResult[Map[T, BoundedDouble]] = { - if (elementClassTag.erasure.isArray) { + if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 900d73bf42..883a0152bb 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -40,8 +40,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = { val c = { - if (classOf[Writable].isAssignableFrom(classTag[T].erasure)) { - classTag[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/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 6bd87bf3ec..7272a592a5 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -277,9 +277,9 @@ class SparkContext( (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) } @@ -300,9 +300,9 @@ class SparkContext( (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]]) } /** @@ -781,13 +781,13 @@ object SparkContext { private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = { def anyToWritable[U <% Writable](u: U): Writable = u - new ArrayWritable(classTag[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: ClassTag](convert: W => T) = { - val wClass = classTag[W].erasure.asInstanceOf[Class[W]] + val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } @@ -806,7 +806,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 diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 962ba6619d..aa1a8b6ba2 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -457,7 +457,7 @@ extends Serializable { prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -487,7 +487,7 @@ extends Serializable { prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -509,7 +509,7 @@ extends Serializable { self.foreach(saveFunc) } - private def getKeyClass() = implicitly[ClassTag[K]].erasure + private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass - private def getValueClass() = implicitly[ClassTag[V]].erasure + private def getValueClass() = implicitly[ClassTag[V]].runtimeClass } -- cgit v1.2.3 From 24bbf318b3e8e657f911204b84f2d032fb1ff6e2 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 29 Apr 2013 19:56:28 +0530 Subject: Fixied other warnings --- core/src/main/scala/spark/TaskState.scala | 3 +-- core/src/main/scala/spark/api/python/PythonRDD.scala | 4 ++-- core/src/main/scala/spark/deploy/ExecutorState.scala | 3 +-- core/src/main/scala/spark/deploy/master/ApplicationState.scala | 3 +-- core/src/main/scala/spark/deploy/master/WorkerState.scala | 2 +- core/src/main/scala/spark/util/AkkaUtils.scala | 4 ++-- repl/src/main/scala/spark/repl/SparkIMain.scala | 2 +- streaming/src/main/scala/spark/streaming/DStream.scala | 4 +--- 8 files changed, 10 insertions(+), 15 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 78eb33a628..44893ef089 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -2,8 +2,7 @@ package 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/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 82959a33eb..220047c360 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -134,7 +134,7 @@ private[spark] class PythonRDD[T: ClassTag]( } new Array[Byte](0) } - case e => throw e + case e : Throwable => throw e } } @@ -233,7 +233,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)) } diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala index 5dc0c54552..a487e11f2c 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/spark/deploy/ExecutorState.scala @@ -1,7 +1,6 @@ package 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/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala index 15016b388d..4f359711c5 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala @@ -1,7 +1,6 @@ package spark.deploy.master -private[spark] object ApplicationState - extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { +private[spark] object ApplicationState extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala index 0bf35014c8..1e347bee20 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala @@ -1,6 +1,6 @@ package spark.deploy.master -private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { +private[spark] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED = Value diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 70338ec4dc..e16915c8e9 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -61,7 +61,7 @@ private[spark] object AkkaUtils { * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to * handle requests. Returns the bound port or throws a SparkException on failure. */ - def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) { + def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) = { val ioWorker = IOExtension(actorSystem).ioBridge() val httpService = actorSystem.actorOf(Props(HttpServiceActor(route))) val server = actorSystem.actorOf( @@ -72,7 +72,7 @@ private[spark] object AkkaUtils { try { Await.result(future, timeout) match { case bound: HttpServer.Bound => - return server + server case other: Any => throw new SparkException("Failed to bind web UI to port " + port + ": " + other) } diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/spark/repl/SparkIMain.scala index 9894429ec1..b1977d6788 100644 --- a/repl/src/main/scala/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/spark/repl/SparkIMain.scala @@ -809,7 +809,7 @@ import spark.Logging // 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 = definitions.getClass(newTypeName(readPath)) // the outermost wrapper + val readRoot = rootMirror.getClassByName(newTypeName(readPath)) // the outermost wrapper (accessPath split '.').foldLeft(readRoot: Symbol) { case (sym, "") => sym case (sym, name) => afterTyper(termMember(sym, name)) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index c307c69611..6ad43dd9b5 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -466,9 +466,7 @@ abstract class DStream[T: ClassTag] ( * 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))) } /** -- cgit v1.2.3 From 4041a2689e9f66ecf9550c9f0d0ae577b2f904c2 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 1 May 2013 11:35:35 +0530 Subject: Updated to latest stable scala 2.10.1 and akka 2.1.2 --- pom.xml | 2 +- project/SparkBuild.scala | 19 ++++++++----------- streaming/pom.xml | 14 ++------------ 3 files changed, 11 insertions(+), 24 deletions(-) (limited to 'streaming') diff --git a/pom.xml b/pom.xml index 52f9228896..6e2f36dee3 100644 --- a/pom.xml +++ b/pom.xml @@ -53,7 +53,7 @@ 1.5 2.10 0.9.0-incubating - 2.1.0 + 2.1.2 1.1-M7 1.2.3 1.6.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 49c8870019..1500aee672 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -36,7 +36,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", version := "0.8.0-SNAPSHOT", - scalaVersion := "2.10.0", + scalaVersion := "2.10.1", scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, @@ -51,9 +51,6 @@ object SparkBuild extends Build { // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), - // Shared between both core and streaming. - resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), - // For Sonatype publishing resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), @@ -141,8 +138,8 @@ object SparkBuild extends Build { "asm" % "asm-all" % "3.3.1", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.20", - "com.typesafe.akka" %% "akka-remote" % "2.1.0", - "com.typesafe.akka" %% "akka-slf4j" % "2.1.0", + "com.typesafe.akka" %% "akka-remote" % "2.1.2", + "com.typesafe.akka" %% "akka-slf4j" % "2.1.2", "it.unimi.dsi" % "fastutil" % "6.4.4", "io.spray" % "spray-can" % "1.1-M7", "io.spray" % "spray-io" % "1.1-M7", @@ -150,9 +147,9 @@ object SparkBuild extends Build { "io.spray" %% "spray-json" % "1.2.3", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.9.0-incubating", - "org.scala-lang" % "scala-actors" % "2.10.0", - "org.scala-lang" % "jline" % "2.10.0", - "org.scala-lang" % "scala-reflect" % "2.10.0" + "org.scala-lang" % "scala-actors" % "2.10.1", + "org.scala-lang" % "jline" % "2.10.1", + "org.scala-lang" % "scala-reflect" % "2.10.1" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } @@ -165,7 +162,7 @@ object SparkBuild extends Build { def replSettings = sharedSettings ++ Seq( name := "spark-repl", // libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - libraryDependencies ++= Seq("org.scala-lang" % "scala-compiler" % "2.10.0") + libraryDependencies ++= Seq("org.scala-lang" % "scala-compiler" % "2.10.1") ) def examplesSettings = sharedSettings ++ Seq( @@ -181,7 +178,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", "org.twitter4j" % "twitter4j-stream" % "3.0.3", - "com.typesafe.akka" % "akka-zeromq" % "2.1-M1" excludeAll(ExclusionRule(name = "akka-actor"), ExclusionRule(organization = "org.scala-lang")) + "com.typesafe.akka" %% "akka-zeromq" % "2.1.2" ) ) ++ assemblySettings ++ extraAssemblySettings diff --git a/streaming/pom.xml b/streaming/pom.xml index 26f0ae951c..fe869ba66e 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -54,18 +54,8 @@ com.typesafe.akka - akka-zeromq - 2.1-M1 - - - org.scala-lang - scala-library - - - com.typesafe.akka - akka-actor - - + akka-zeromq_${scala.version} + ${akka.version} org.scalatest -- cgit v1.2.3 From 63addd93a8337f5181c0e7d64c86393cb519c661 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:09:52 +0530 Subject: Fixed warning ClassManifest -> ClassTag --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/main/scala/spark/RDD.scala | 6 +++--- .../main/scala/spark/api/java/function/FlatMapFunction2.scala | 4 +++- core/src/main/scala/spark/rdd/EmptyRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/JdbcRDD.scala | 3 ++- core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala | 9 +++++---- .../main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 7 files changed, 17 insertions(+), 13 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 0095b868a8..5cf162f23e 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -536,7 +536,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( * supplied codec. */ def saveAsHadoopFile[F <: OutputFormat[K, V]]( - path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) { + path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 3a454df10d..2ebfaadc46 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -500,18 +500,18 @@ abstract class RDD[T: ClassTag]( * *same number of partitions*, but does *not* require them to have the same number * of elements in each partition. */ - def zipPartitions[B: ClassManifest, V: ClassManifest]( + def zipPartitions[B: ClassTag, V: ClassTag]( f: (Iterator[T], Iterator[B]) => Iterator[V], rdd2: RDD[B]): RDD[V] = new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) - def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( + def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]( f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], rdd2: RDD[B], rdd3: RDD[C]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) - def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( + def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]( f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], rdd2: RDD[B], rdd3: RDD[C], diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala index 6044043add..4562884eb3 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -1,5 +1,7 @@ package spark.api.java.function +import scala.reflect.ClassTag + /** * A function that takes two inputs and returns zero or more output records. */ @@ -7,5 +9,5 @@ abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Itera @throws(classOf[Exception]) def call(a: A, b:B) : java.lang.Iterable[C] - def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]] + def elementType() : ClassTag[C] = ClassTag.Any.asInstanceOf[ClassTag[C]] } diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala index e4dd3a7fa7..3b9899238c 100644 --- a/core/src/main/scala/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -1,12 +1,12 @@ package spark.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} - +import scala.reflect.ClassTag /** * 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/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index a50f407737..f570dd6d8b 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -4,6 +4,7 @@ import java.sql.{Connection, ResultSet} import spark.{Logging, Partition, RDD, SparkContext, TaskContext} import spark.util.NextIterator +import scala.reflect.ClassTag private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx @@ -28,7 +29,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/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index b234428ab2..d3c2370885 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -2,6 +2,7 @@ package spark.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} +import scala.reflect.ClassTag private[spark] class ZippedPartitionsPartition( idx: Int, @@ -20,7 +21,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( +abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]]) extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) { @@ -67,7 +68,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], @@ -87,7 +88,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], @@ -111,7 +112,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/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index e0f6351ef7..ba1312cbe8 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -95,7 +95,7 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest]( } // Handles Kafka Messages - private class MessageHandler[T: ClassManifest](stream: KafkaStream[T]) extends Runnable { + private class MessageHandler[T: ClassTag](stream: KafkaStream[T]) extends Runnable { def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { -- cgit v1.2.3 From ff14f38f3dce314eb1afb876e79cbab6b1c252b8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:34:56 +0530 Subject: Fixed warning Throwables --- core/src/main/scala/spark/api/python/PythonWorkerFactory.scala | 4 ++-- .../main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 85d1dfeac8..7163f01b24 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -28,7 +28,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String startDaemon() new Socket(daemonHost, daemonPort) } - case e => throw e + case e : Throwable => throw e } } } @@ -88,7 +88,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/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 327d6797ae..56374be35a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -524,7 +524,7 @@ private[spark] class ClusterTaskSetManager( case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader throw new SparkException("ClassNotFound with classloader: " + loader, cnf) - case ex => throw ex + case ex : Throwable => throw ex } // Mark finished and stop if we've finished all the tasks finished(index) = true diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index ba1312cbe8..8b23642e4a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -118,7 +118,7 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest]( zk.deleteRecursive(dir) zk.close() } catch { - case _ => // swallow + case _ : Throwable => // swallow } } } -- cgit v1.2.3 From 55da6e9504c533b6eaced215c9d4a6199062f799 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:36:36 +0530 Subject: Fixed warning erasure -> runtimeClass --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- .../src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 5cf162f23e..ee09c2085b 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -537,7 +537,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( */ def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]], codec) } /** diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 8b23642e4a..47274f41a5 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -85,7 +85,7 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest]( } // Create Threads for each Topic/Message Stream we are listening - val decoder = manifest[D].erasure.newInstance.asInstanceOf[Decoder[T]] + val decoder = manifest[D].runtimeClass.newInstance.asInstanceOf[Decoder[T]] val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder) // Start the messages handler for each partition -- cgit v1.2.3 From 119c98c1becf9f0038d6bb946545cd65006bd367 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 15:01:33 +0530 Subject: code formatting, The warning related to scope exit and enter is not worth fixing as it only affects debugging scopes and nothing else. --- .../spark/streaming/util/MasterFailureTest.scala | 28 +++++++++++----------- 1 file changed, 14 insertions(+), 14 deletions(-) (limited to 'streaming') diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index b3daa5a91b..ca654c45c9 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -363,20 +363,20 @@ 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) logError("Could not generate file " + hadoopFile) -- cgit v1.2.3 From 6fcfefcb278ffb9d54452d3336cb74b97480fbf5 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 10 Sep 2013 10:57:47 +0530 Subject: Few more fixes to tests broken during merge --- .../org/apache/spark/repl/ReplSuiteMixin.scala | 50 ---------------------- .../apache/spark/streaming/CheckpointSuite.scala | 6 +-- 2 files changed, 3 insertions(+), 53 deletions(-) delete mode 100644 repl/src/test/scala/org/apache/spark/repl/ReplSuiteMixin.scala (limited to 'streaming') diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuiteMixin.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuiteMixin.scala deleted file mode 100644 index ccfbf5193a..0000000000 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuiteMixin.scala +++ /dev/null @@ -1,50 +0,0 @@ -package org.apache.spark.repl - -import java.io.BufferedReader -import java.io.PrintWriter -import java.io.StringReader -import java.io.StringWriter -import java.net.URLClassLoader - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.future - - -trait ReplSuiteMixin { - def runInterpreter(master: String, input: String): String = { - val in = new BufferedReader(new StringReader(input + "\n")) - val out = new StringWriter() - val cl = getClass.getClassLoader - var paths = new ArrayBuffer[String] - if (cl.isInstanceOf[URLClassLoader]) { - val urlLoader = cl.asInstanceOf[URLClassLoader] - for (url <- urlLoader.getURLs) { - if (url.getProtocol == "file") { - paths += url.getFile - } - } - } - - val interp = new SparkILoop(in, new PrintWriter(out), master) - spark.repl.Main.interp = interp - val separator = System.getProperty("path.separator") - interp.process(Array("-classpath", paths.mkString(separator))) - if (interp != null) { - interp.closeInterpreter() - } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.hostPort") - return out.toString - } - - def assertContains(message: String, output: String) { - assert(output contains message, - "Interpreter output did not contain '" + message + "':\n" + output) - } - - def assertDoesNotContain(message: String, output: String) { - assert(!(output contains message), - "Interpreter output contained '" + message + "':\n" + output) - } -} 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 b8337ed423..07de51bebb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -29,9 +29,9 @@ import org.scalatest.BeforeAndAfter import com.google.common.io.Files -import spark.streaming.StreamingContext.toPairDStreamFunctions -import spark.streaming.dstream.FileInputDStream -import spark.streaming.util.ManualClock +import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions +import org.apache.spark.streaming.dstream.FileInputDStream +import org.apache.spark.streaming.util.ManualClock -- cgit v1.2.3 From 276c37a51c9a6188dbbe02754935540ace338dd1 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 22 Sep 2013 08:20:12 +0530 Subject: Akka 2.2 migration --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/deploy/client/Client.scala | 10 +++---- .../org/apache/spark/deploy/master/Master.scala | 31 ++++++++++++++++------ .../org/apache/spark/deploy/worker/Worker.scala | 12 ++++----- .../scala/org/apache/spark/executor/Executor.scala | 2 +- .../spark/executor/StandaloneExecutorBackend.scala | 30 ++++++++++++++------- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/StandaloneSchedulerBackend.scala | 12 ++++----- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 23 ++++++++-------- .../scala/org/apache/spark/DistributedSuite.scala | 2 +- .../test/scala/org/apache/spark/DriverSuite.scala | 2 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 8 +++--- project/SparkBuild.scala | 18 ++++++------- .../apache/spark/streaming/StreamingContext.scala | 5 ++-- .../streaming/api/java/JavaStreamingContext.scala | 7 ++--- .../streaming/dstream/NetworkInputDStream.scala | 2 +- .../spark/streaming/receivers/ActorReceiver.scala | 4 +-- .../spark/streaming/receivers/ZeroMQReceiver.scala | 7 ++--- .../org/apache/spark/streaming/JavaAPISuite.java | 2 +- .../spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- 23 files changed, 109 insertions(+), 80 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 1e63b54b7a..a267407c67 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -169,7 +169,7 @@ object SparkEnv extends Logging { val driverHost: String = System.getProperty("spark.driver.host", "localhost") val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") - val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name) + val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) actorSystem.actorFor(url) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 14a90934f6..164386782c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -26,9 +26,7 @@ import akka.actor._ import akka.actor.Terminated import akka.pattern.AskTimeoutException import akka.pattern.ask -import akka.remote.RemoteClientDisconnected -import akka.remote.RemoteClientLifeCycleEvent -import akka.remote.RemoteClientShutdown +import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} import org.apache.spark.Logging import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -61,7 +59,7 @@ private[spark] class Client( master = context.actorFor(Master.toAkkaUrl(masterUrl)) masterAddress = master.path.address master ! RegisterApplication(appDescription) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => @@ -99,12 +97,12 @@ private[spark] class Client( markDisconnected() context.stop(self) - case RemoteClientDisconnected(transport, address) if address == masterAddress => + case DisassociatedEvent(_, address, _) if address == masterAddress => logError("Connection to master failed; stopping client") markDisconnected() context.stop(self) - case RemoteClientShutdown(transport, address) if address == masterAddress => + case AssociationErrorEvent(_, _, address, _) if address == masterAddress => logError("Connection to master failed; stopping client") markDisconnected() context.stop(self) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2efd16bca0..cb0fe6a850 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -25,9 +25,8 @@ import scala.concurrent.Await import scala.concurrent.duration._ import akka.actor._ -import akka.actor.Terminated import akka.pattern.ask -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.remote._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -36,6 +35,22 @@ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} import akka.util.Timeout +import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed +import org.apache.spark.deploy.DeployMessages.KillExecutor +import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import scala.Some +import org.apache.spark.deploy.DeployMessages.WebUIPortResponse +import org.apache.spark.deploy.DeployMessages.LaunchExecutor +import org.apache.spark.deploy.DeployMessages.RegisteredApplication +import org.apache.spark.deploy.DeployMessages.RegisterWorker +import org.apache.spark.deploy.DeployMessages.ExecutorUpdated +import org.apache.spark.deploy.DeployMessages.MasterStateResponse +import org.apache.spark.deploy.DeployMessages.ExecutorAdded +import org.apache.spark.deploy.DeployMessages.RegisterApplication +import org.apache.spark.deploy.DeployMessages.ApplicationRemoved +import org.apache.spark.deploy.DeployMessages.Heartbeat +import org.apache.spark.deploy.DeployMessages.RegisteredWorker +import akka.actor.Terminated private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -81,7 +96,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def preStart() { logInfo("Starting Spark master at spark://" + host + ":" + port) // Listen for remote client disconnection events, since they don't go through Akka's watch() - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() import context.dispatcher context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -165,13 +180,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act actorToApp.get(actor).foreach(finishApplication) } - case RemoteClientDisconnected(transport, address) => { + case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) addressToApp.get(address).foreach(finishApplication) } - case RemoteClientShutdown(transport, address) => { + case AssociationErrorEvent(_, _, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) addressToApp.get(address).foreach(finishApplication) @@ -376,11 +391,11 @@ private[spark] object Master { actorSystem.awaitTermination() } - /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ + /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ def toAkkaUrl(sparkUrl: String): String = { sparkUrl match { case sparkUrlRegex(host, port) => - "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) case _ => throw new SparkException("Invalid master URL: " + sparkUrl) } @@ -388,7 +403,7 @@ private[spark] object Master { def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) - val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName) + val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName) val timeoutDuration = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") implicit val timeout = Timeout(timeoutDuration) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index a0a9d1040a..1f04c1eea5 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 @@ -25,9 +25,9 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} -import org.apache.spark.{Logging} +import org.apache.spark.Logging import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -113,7 +113,7 @@ private[spark] class Worker( logInfo("Connecting to master " + masterUrl) master = context.actorFor(Master.toAkkaUrl(masterUrl)) master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } @@ -165,7 +165,7 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + case _: Terminated | DisassociatedEvent | AssociationErrorEvent => masterDisconnected() case RequestWorkerState => { @@ -207,8 +207,8 @@ private[spark] object Worker { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) - val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory, - masterUrl, workDir)), name = "Worker") + actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, + masterUrl, workDir), name = "Worker") (actorSystem, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index ceae3b8289..99a4a95e82 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -105,7 +105,7 @@ private[spark] class Executor( SparkEnv.set(env) env.metricsSystem.registerSource(executorSource) - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index 7839023868..46f0ef2cc6 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -19,13 +19,25 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import akka.actor.{ActorRef, Actor, Props, Terminated} -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.actor._ +import akka.remote._ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask +import akka.remote.DisassociatedEvent +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask +import akka.remote.AssociationErrorEvent +import akka.remote.DisassociatedEvent +import akka.actor.Terminated +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed private[spark] class StandaloneExecutorBackend( @@ -40,14 +52,14 @@ private[spark] class StandaloneExecutorBackend( Utils.checkHostPort(hostPort, "Expected hostport") var executor: Executor = null - var driver: ActorRef = null + var driver: ActorSelection = null override def preStart() { logInfo("Connecting to driver: " + driverUrl) - driver = context.actorFor(driverUrl) + driver = context.actorSelection(driverUrl) driver ! RegisterExecutor(executorId, hostPort, cores) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(driver) // Doesn't work with remote actors, but useful for testing + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + // context.watch(driver) // Doesn't work with remote actors, but useful for testing } override def receive = { @@ -69,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + case _: Terminated | DisassociatedEvent | AssociationErrorEvent => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } @@ -90,8 +102,8 @@ private[spark] object StandaloneExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) - val actor = actorSystem.actorOf( - Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)), + actorSystem.actorOf( + Props(classOf[StandaloneExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") actorSystem.awaitTermination() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 9c49768c0c..fa83ae19d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -42,7 +42,7 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index addfa077c1..49f668eb32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -25,7 +25,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ @@ -53,7 +53,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong @@ -101,11 +101,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case RemoteClientDisconnected(transport, address) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected")) + case DisassociatedEvent(_, remoteAddress, _) => + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected")) - case RemoteClientShutdown(transport, address) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown")) + case AssociationErrorEvent(_, _, remoteAddress, _) => + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown")) } // Make fake resource offers on all executors diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 3dbe61d706..babe875fa1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -119,7 +119,7 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index e674d120ea..af1c36b34d 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -48,28 +48,27 @@ private[spark] object AkkaUtils { val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on - akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] + akka.loggers = [""akka.event.slf4j.Slf4jLogger""] akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" - akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.netty.hostname = "%s" - akka.remote.netty.port = %d - akka.remote.netty.connection-timeout = %ds - akka.remote.netty.message-frame-size = %d MiB - akka.remote.netty.execution-pool-size = %d + akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" + akka.remote.netty.tcp.hostname = "%s" + akka.remote.netty.tcp.port = %d + akka.remote.netty.tcp.connection-timeout = %ds + akka.remote.netty.tcp.message-frame-size = %d MiB + akka.remote.netty.tcp.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d akka.remote.log-remote-lifecycle-events = %s - akka.remote.netty.write-timeout = %ds - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, - lifecycleEvents, akkaWriteTimeout)) + """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, + lifecycleEvents)) val actorSystem = ActorSystem(name, akkaConf) // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider - val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get - return (actorSystem, boundPort) + val boundPort = provider.getDefaultAddress.port.get + (actorSystem, boundPort) } } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 7a856d4081..c719a54a61 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -325,7 +325,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter // when running under LocalScheduler: sc = new SparkContext("local-cluster[1,1,512]", "test") val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} val exception = intercept[SparkException] { rdd.reduce((x, y) => x) diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 01a72d8401..6d1695eae7 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts { // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => - failAfter(30 seconds) { + failAfter(60 seconds) { Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 6013320eaa..18fb1bf590 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -109,7 +109,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() slaveTracker.trackerActor = slaveSystem.actorFor( - "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") + "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker") masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 13aa24fa1a..08e399f9ee 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -165,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/project/SparkBuild.scala b/project/SparkBuild.scala index 5e7ed81c1e..f18ebf1400 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -81,7 +81,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := "0.8.0-SNAPSHOT", scalaVersion := "2.10.2", - scalacOptions := Seq("-unchecked", "-optimize", "-deprecation", "-target:" + SCALAC_JVM_VERSION), +// scalacOptions := Seq("-unchecked", "-optimize", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, @@ -150,7 +150,7 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.0.Beta2", + "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", @@ -183,9 +183,9 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Spray Repository" at "http://repo.spray.cc/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" + // "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", + // "Spray Repository" at "http://repo.spray.cc/", + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), libraryDependencies ++= Seq( @@ -200,9 +200,9 @@ object SparkBuild extends Build { "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", - "com.typesafe.akka" %% "akka-remote" % "2.1.4" excludeAll(excludeNetty), - "com.typesafe.akka" %% "akka-slf4j" % "2.1.4" excludeAll(excludeNetty), - "net.liftweb" %% "lift-json" % "2.5.1", + "com.typesafe.akka" %% "akka-remote" % "2.2.1" excludeAll(excludeNetty), + "com.typesafe.akka" %% "akka-slf4j" % "2.2.1" excludeAll(excludeNetty), + "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.12.1", @@ -271,7 +271,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" %% "akka-zeromq" % "2.1.4" excludeAll(excludeNetty) + "com.typesafe.akka" %% "akka-zeromq" % "2.2.1" excludeAll(excludeNetty) ) ) 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 9e14c8ace7..c722aa15ab 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -47,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 /** @@ -231,11 +232,11 @@ class StreamingContext private ( 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) } 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 8135d2499e..8242af6d5f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -29,6 +29,7 @@ import twitter4j.Status import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe +import akka.util.ByteString import twitter4j.auth.Authorization @@ -475,7 +476,7 @@ 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] = { @@ -502,7 +503,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel) } @@ -522,7 +523,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator ssc.zeroMQStream[T](publisherUrl, subscribe, fn) } 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 a61a1780f1..394a39fbb0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -177,7 +177,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging logInfo("Attempting to register with tracker") val ip = System.getProperty("spark.driver.host", "localhost") val port = System.getProperty("spark.driver.port", "7077").toInt - val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port) + val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) val tracker = env.actorSystem.actorFor(url) val timeout = 5.seconds diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index c220127c00..ee087a1cf0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -51,7 +51,7 @@ object ReceiverSupervisorStrategy { * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { - * case anything :String ⇒ pushBlock(anything) + * case anything :String => pushBlock(anything) * } * } * //Can be plugged in actorStream as follows @@ -121,7 +121,7 @@ private[streaming] class ActorReceiver[T: ClassTag]( 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 e009325b67..ce8c56fa8a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.receivers import akka.actor.Actor +import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging @@ -29,7 +30,7 @@ import scala.reflect.ClassTag */ 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() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self), @@ -40,10 +41,10 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](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/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index c0d729ff87..783b8dea31 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -48,7 +48,7 @@ import java.util.*; import akka.actor.Props; import akka.zeromq.Subscribe; - +import akka.util.ByteString; // The test suite itself is Serializable so that anonymous Function implementations can be diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 6d6ef149cc..d222f412a0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -209,7 +209,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM else { // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) -- cgit v1.2.3 From 7ff4c2d399e1497966689cbe13edf2cd2a9a29b1 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 26 Sep 2013 10:48:24 +0530 Subject: fixed maven build for scala 2.10 --- assembly/pom.xml | 2 +- bagel/pom.xml | 8 ++++---- core/pom.xml | 31 ++++++++++++++----------------- examples/pom.xml | 10 +++++----- mllib/pom.xml | 8 ++++---- pom.xml | 42 ++++++++++++++++++------------------------ project/SparkBuild.scala | 3 +-- repl/pom.xml | 12 ++++++------ streaming/pom.xml | 14 +++++++++----- tools/pom.xml | 6 +++--- yarn/pom.xml | 4 ++-- 11 files changed, 67 insertions(+), 73 deletions(-) (limited to 'streaming') diff --git a/assembly/pom.xml b/assembly/pom.xml index 808a829e19..47a110ca6c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -79,7 +79,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + ${project.build.directory}/scala-${scala-short.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar *:* diff --git a/bagel/pom.xml b/bagel/pom.xml index 51173c32b2..feaed6d2b0 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -43,18 +43,18 @@ org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} test org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala-short.version} test - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.scalatest diff --git a/core/pom.xml b/core/pom.xml index d42c2ee2fd..8d9f0e386f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -83,7 +83,7 @@ com.twitter - chill_2.10 + chill_${scala-short.version} 0.3.1 @@ -91,21 +91,13 @@ chill-java 0.3.1 - - org.scala-lang - scala-actors - com.typesafe.akka - akka-remote_${scala.version} + akka-remote_${scala-short.version} com.typesafe.akka - akka-slf4j_${scala.version} - - - org.scala-lang - scalap + akka-slf4j_${scala-short.version} org.scala-lang @@ -113,7 +105,7 @@ net.liftweb - lift-json_2.10 + lift-json_${scala-short.version} it.unimi.dsi @@ -125,7 +117,7 @@ com.github.scala-incubator.io - scala-io-file_2.10 + scala-io-file_${scala-short.version} org.apache.mesos @@ -160,14 +152,19 @@ derby test + + commons-io + commons-io + test + org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} test org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala-short.version} test @@ -187,8 +184,8 @@ - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.apache.maven.plugins diff --git a/examples/pom.xml b/examples/pom.xml index 9eb6c8a77f..b44a126546 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -77,17 +77,17 @@ com.twitter - algebird-core_2.10 + algebird-core_${scala-short.version} 0.1.11 org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} test org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala-short.version} test @@ -128,8 +128,8 @@ - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.apache.maven.plugins diff --git a/mllib/pom.xml b/mllib/pom.xml index 966caf6835..f4190148b1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -48,12 +48,12 @@ org.scalatest - scalatest_${scala.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala.version} + scalacheck_2.10 test @@ -63,8 +63,8 @@ - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.scalatest diff --git a/pom.xml b/pom.xml index cd4593388f..4033956968 100644 --- a/pom.xml +++ b/pom.xml @@ -76,9 +76,10 @@ UTF-8 1.5 - 2.10 - 0.9.0-incubating - 2.1.4 + 2.10 + 2.10.2 + 0.12.1 + 2.2.1 1.7.2 1.2.17 1.0.4 @@ -214,7 +215,7 @@ com.twitter - chill_2.9.3 + chill_${scala-short.version} 0.3.1 @@ -222,19 +223,14 @@ chill-java 0.3.1 - - org.scala-lang - scala-actors - 2.10.0 - com.typesafe.akka - akka-remote_${scala.version} + akka-remote_${scala-short.version} ${akka.version} com.typesafe.akka - akka-slf4j_${scala.version} + akka-slf4j_${scala-short.version} ${akka.version} @@ -249,7 +245,7 @@ com.github.scala-incubator.io - scala-io-file_2.10 + scala-io-file_${scala-short.version} 0.4.1 @@ -270,7 +266,7 @@ net.liftweb - lift-json_2.10 + lift-json_${scala-short.version} 2.5.1 @@ -296,36 +292,34 @@ org.scala-lang scala-compiler - 2.10.0 + ${scala.version} org.scala-lang jline - 2.10.0 - - - org.scala-lang - scala-library ${scala.version} org.scala-lang - scalap + scala-library ${scala.version} - log4j log4j ${log4j.version} - org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} 1.9.1 test + + commons-io + commons-io + 2.4 + org.easymock easymock @@ -334,7 +328,7 @@ org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala-short.version} 1.10.0 test diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f18ebf1400..ffa9c93925 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -183,8 +183,7 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - // "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - // "Spray Repository" at "http://repo.spray.cc/", + "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), diff --git a/repl/pom.xml b/repl/pom.xml index b1ca5220bc..a7b5e1f3c7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -61,12 +61,12 @@ org.scala-lang scala-compiler - 2.10.0 + ${scala.version} org.scala-lang jline - 2.10.0 + ${scala.version} org.slf4j @@ -78,18 +78,18 @@ org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} test org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala-short.version} test - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.apache.maven.plugins diff --git a/streaming/pom.xml b/streaming/pom.xml index 3c9baf5f69..8367256004 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -86,17 +86,17 @@ com.typesafe.akka - akka-zeromq_${scala.version} + akka-zeromq_${scala-short.version} ${akka.version} org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} test org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala-short.version} test @@ -109,10 +109,14 @@ slf4j-log4j12 test + + commons-io + commons-io + - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.scalatest diff --git a/tools/pom.xml b/tools/pom.xml index 77646a6816..0933c75a7f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -43,14 +43,14 @@ org.scalatest - scalatest_${scala.version} + scalatest_${scala-short.version} test - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.apache.maven.plugins diff --git a/yarn/pom.xml b/yarn/pom.xml index 21b650d1ea..47e27ee41c 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -64,8 +64,8 @@ - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala-short.version}/classes + target/scala-${scala-short.version}/test-classes org.apache.maven.plugins -- cgit v1.2.3 From e09f4a9601b18921c309903737d309eab5c6d891 Mon Sep 17 00:00:00 2001 From: Martin Weindel Date: Sat, 5 Oct 2013 23:08:23 +0200 Subject: fixed some warnings --- .../org/apache/spark/network/netty/FileClient.java | 2 - .../org/apache/spark/network/netty/FileServer.java | 1 - .../java/JavaSparkContextVarargsWorkaround.java | 1 - .../spark/api/java/function/DoubleFunction.java | 2 - .../apache/spark/api/java/function/Function.java | 1 - .../apache/spark/api/java/function/Function2.java | 1 - .../api/java/function/PairFlatMapFunction.java | 1 - .../spark/api/java/function/PairFunction.java | 1 - .../spark/api/python/PythonWorkerFactory.scala | 4 +- .../org/apache/spark/deploy/ExecutorState.scala | 2 +- .../spark/deploy/master/ApplicationState.scala | 2 +- .../apache/spark/deploy/master/WorkerState.scala | 2 +- .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 3 +- .../scala/org/apache/spark/rdd/CoalescedRDD.scala | 3 +- .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 3 +- .../scala/org/apache/spark/rdd/FilteredRDD.scala | 3 +- .../scala/org/apache/spark/rdd/FlatMappedRDD.scala | 3 +- .../scala/org/apache/spark/rdd/GlommedRDD.scala | 3 +- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 3 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 3 +- .../scala/org/apache/spark/rdd/MappedRDD.scala | 3 +- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 7 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/cluster/SchedulingMode.scala | 2 +- .../spark/scheduler/cluster/TaskLocality.scala | 2 +- .../scala/org/apache/spark/DistributedSuite.scala | 2 +- .../scala/org/apache/spark/UnpersistSuite.scala | 2 +- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 8 +- .../org/apache/spark/examples/JavaLogQuery.java | 2 +- .../org/apache/spark/examples/JavaPageRank.java | 3 +- .../org/apache/spark/examples/JavaWordCount.java | 2 +- .../org/apache/spark/mllib/examples/JavaALS.java | 1 - .../spark/mllib/clustering/JavaKMeansSuite.java | 4 +- .../spark/mllib/recommendation/JavaALSSuite.java | 2 - .../spark/streaming/dstream/CoGroupedDStream.scala | 3 +- .../streaming/dstream/ConstantInputDStream.scala | 3 +- .../spark/streaming/dstream/FilteredDStream.scala | 3 +- .../streaming/dstream/FlatMapValuedDStream.scala | 3 +- .../streaming/dstream/FlatMappedDStream.scala | 3 +- .../spark/streaming/dstream/ForEachDStream.scala | 3 +- .../spark/streaming/dstream/GlommedDStream.scala | 3 +- .../streaming/dstream/MapPartitionedDStream.scala | 3 +- .../spark/streaming/dstream/MapValuedDStream.scala | 3 +- .../spark/streaming/dstream/MappedDStream.scala | 3 +- .../streaming/dstream/PluggableInputDStream.scala | 3 +- .../streaming/dstream/QueueInputDStream.scala | 4 +- .../spark/streaming/dstream/ShuffledDStream.scala | 3 +- .../streaming/dstream/TransformedDStream.scala | 3 +- .../org/apache/spark/streaming/JavaAPISuite.java | 94 +++++++++++----------- 51 files changed, 120 insertions(+), 108 deletions(-) (limited to 'streaming') 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/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/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java index db34cd190a..ed92d31af5 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java @@ -18,8 +18,6 @@ package org.apache.spark.api.java.function; -import scala.runtime.AbstractFunction1; - import java.io.Serializable; /** 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 f9dae6ed34..e97116986f 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 @@ -19,7 +19,6 @@ package org.apache.spark.api.java.function; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; 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 1659bfc552..cf77bb6b73 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 @@ -19,7 +19,6 @@ package org.apache.spark.api.java.function; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction2; import java.io.Serializable; 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 5a5c9b6296..fbd0cdabe0 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 @@ -20,7 +20,6 @@ package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; 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 4c39f483e5..f09559627d 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 @@ -20,7 +20,6 @@ package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; 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..72f84c99fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy private[spark] object ExecutorState - extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") { + extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value 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 7e804223cf..39ef090ddf 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.master private[spark] object ApplicationState - extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { + extends Enumeration { type ApplicationState = 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 b5ee6dca79..fb3fe88d92 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,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { +private[spark] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED = Value 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 bca6956a18..fe2946bcbe 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,13 +19,14 @@ package org.apache.spark.rdd import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.BlockManager +import scala.reflect.ClassTag private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) +class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[String]) 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/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/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index c8900d1a93..99c34c6cc5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -18,12 +18,13 @@ package org.apache.spark.rdd import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} +import scala.reflect.ClassTag /** * 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/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 203179c4ea..cdb5946b49 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -18,10 +18,11 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala index 3ed8339010..3cf22851dd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag /** @@ -26,7 +27,7 @@ import org.apache.spark.{Partition, TaskContext} * information such as the number of tuples in a partition. */ private[spark] -class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsWithIndexRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean 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..eb3b19907d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -18,9 +18,10 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag 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..4a465840c6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{RangePartitioner, Logging} +import scala.reflect.ClassTag /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -25,9 +26,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 aed585e6a1..c8e623081a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -51,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 { 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 9537152335..b7205865cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} +import scala.reflect.ClassTag private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -32,7 +33,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/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index 0ac3d7bcfd..f61fde6957 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -461,7 +461,7 @@ private[spark] class ClusterTaskSetManager( case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader throw new SparkException("ClassNotFound with classloader: " + loader, cnf) - case ex => throw ex + case ex: Throwable => throw ex } // Mark finished and stop if we've finished all the tasks finished(index) = true diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala index 34811389a0..16013b3208 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala @@ -22,7 +22,7 @@ package org.apache.spark.scheduler.cluster * 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/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala index 5d4130e14a..8d8d708612 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster private[spark] object TaskLocality - extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") + 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/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index c719a54a61..adc6ca94ff 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -313,7 +313,7 @@ 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. } 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 c1df5e151e..096023f476 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -225,8 +225,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 @@ -238,9 +238,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/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, 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> 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> 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/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 points = new ArrayList(); + List points = new ArrayList(); 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 points = new ArrayList(); + List points = new ArrayList(); 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 3323f6cee2..c474e01188 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 @@ -20,8 +20,6 @@ package org.apache.spark.mllib.recommendation; import java.io.Serializable; import java.util.List; -import scala.Tuple2; - import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala index 4eddc755b9..16c1567355 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala @@ -21,9 +21,10 @@ import org.apache.spark.Partitioner import org.apache.spark.rdd.RDD import org.apache.spark.rdd.CoGroupedRDD import org.apache.spark.streaming.{Time, DStream, Duration} +import scala.reflect.ClassTag private[streaming] -class CoGroupedDStream[K : ClassManifest]( +class CoGroupedDStream[K : ClassTag]( parents: Seq[DStream[(K, _)]], partitioner: Partitioner ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { 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/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/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/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/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/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/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 60485adef9..73e1ddf7a4 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[T: ClassManifest, U: ClassManifest] ( +class TransformedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], transformFunc: (RDD[T], Time) => RDD[U] ) extends DStream[U](parent.ssc) { 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 783b8dea31..076fb53fa1 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -21,34 +21,36 @@ 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.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.*; import akka.actor.Props; import akka.zeromq.Subscribe; -import akka.util.ByteString; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -85,8 +87,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(3L), Arrays.asList(1L)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream count = stream.count(); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream count = stream.count(); JavaTestUtils.attachTestOutputStream(count); List> result = JavaTestUtils.runStreams(ssc, 3, 3); assertOrderInvariantEquals(expected, result); @@ -102,8 +104,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() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { @Override public Integer call(String s) throws Exception { return s.length(); @@ -128,8 +130,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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(2000)); JavaTestUtils.attachTestOutputStream(windowed); List> result = JavaTestUtils.runStreams(ssc, 4, 4); @@ -152,8 +154,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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); JavaTestUtils.attachTestOutputStream(windowed); List> result = JavaTestUtils.runStreams(ssc, 8, 4); @@ -170,8 +172,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() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream filtered = stream.filter(new Function() { @Override public Boolean call(String s) throws Exception { return s.contains("a"); @@ -193,8 +195,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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream> glommed = stream.glom(); JavaTestUtils.attachTestOutputStream(glommed); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -211,8 +213,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, String>() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { @Override public Iterable call(Iterator in) { String out = ""; @@ -254,8 +256,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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reduced = stream.reduce(new IntegerSum()); JavaTestUtils.attachTestOutputStream(reduced); List> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -275,8 +277,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 stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(), new IntegerDifference(), new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(reducedWindowed); List> result = JavaTestUtils.runStreams(ssc, 4, 4); @@ -349,8 +351,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() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream flatMapped = stream.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { return Lists.newArrayList(x.split("(?!^)")); @@ -396,8 +398,8 @@ public class JavaAPISuite implements Serializable { new Tuple2(9, "c"), new Tuple2(9, "s"))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { @Override public Iterable> call(String in) throws Exception { List> out = Lists.newArrayList(); @@ -430,10 +432,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 stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); - JavaDStream unioned = stream1.union(stream2); + JavaDStream unioned = stream1.union(stream2); JavaTestUtils.attachTestOutputStream(unioned); List> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -444,7 +446,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 void assertOrderInvariantEquals( + public static > void assertOrderInvariantEquals( List> expected, List> actual) { for (List list: expected) { Collections.sort(list); @@ -467,11 +469,11 @@ public class JavaAPISuite implements Serializable { Arrays.asList(new Tuple2("giants", 6)), Arrays.asList(new Tuple2("yankees", 7))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = stream.map( new PairFunction() { @Override - public Tuple2 call(String in) throws Exception { + public Tuple2 call(String in) throws Exception { return new Tuple2(in, in.length()); } }); @@ -1163,8 +1165,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() { + JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { @Override public Integer call(String s) throws Exception { return s.length(); @@ -1220,20 +1222,20 @@ public class JavaAPISuite implements Serializable { @Test public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, + JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zk.connect","localhost:12345"); kafkaParams.put("groupid","consumer-group"); - JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, + JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK()); } @Test public void testSocketTextStream() { - JavaDStream test = ssc.socketTextStream("localhost", 12345); + JavaDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1253,7 +1255,7 @@ public class JavaAPISuite implements Serializable { } } - JavaDStream test = ssc.socketStream( + JavaDStream test = ssc.socketStream( "localhost", 12345, new Converter(), @@ -1262,39 +1264,39 @@ public class JavaAPISuite implements Serializable { @Test public void testTextFileStream() { - JavaDStream test = ssc.textFileStream("/tmp/foo"); + JavaDStream test = ssc.textFileStream("/tmp/foo"); } @Test public void testRawSocketStream() { - JavaDStream test = ssc.rawSocketStream("localhost", 12345); + JavaDStream test = ssc.rawSocketStream("localhost", 12345); } @Test public void testFlumeStream() { - JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); } @Test public void testFileStream() { JavaPairDStream foo = - ssc.fileStream("/tmp/foo"); + ssc.>fileStream("/tmp/foo"); } @Test public void testTwitterStream() { String[] filters = new String[] { "good", "bad", "ugly" }; - JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); } @Test public void testActorStream() { - JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); } @Test public void testZeroMQStream() { - JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { + JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { @Override public Iterable call(byte[][] b) throws Exception { return null; -- cgit v1.2.3 From 6860b79f6e4cc0d38b08848f19127c259d9b5069 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 12 Nov 2013 12:43:53 +0530 Subject: Remove deprecated actorFor and use actorSelection everywhere. --- .../scala/org/apache/spark/MapOutputTracker.scala | 8 +++++-- .../src/main/scala/org/apache/spark/SparkEnv.scala | 8 +++---- .../org/apache/spark/deploy/client/Client.scala | 23 +++++--------------- .../org/apache/spark/deploy/worker/Worker.scala | 23 ++++++++++++++------ .../apache/spark/storage/BlockManagerMaster.scala | 25 +++++++++++++--------- .../org/apache/spark/storage/ThreadingTest.scala | 2 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 14 ++++++------ .../apache/spark/storage/BlockManagerSuite.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 2 +- 10 files changed, 58 insertions(+), 51 deletions(-) (limited to 'streaming') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1afb1870f1..6590e9779e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -60,7 +60,7 @@ 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] = _ private var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -79,7 +79,11 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. def askTracker(message: Any): Any = { try { - val future = trackerActor.ask(message)(timeout) + val future = if (trackerActor.isLeft ) { + trackerActor.left.get.ask(message)(timeout) + } else { + trackerActor.right.get.ask(message)(timeout) + } return Await.result(future, timeout) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index a267407c67..0d9bd500e4 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 @@ -161,17 +161,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.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/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 164386782c..000d1ee9f8 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 @@ -49,18 +49,14 @@ private[spark] class Client( var appId: 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 override def preStart() { logInfo("Connecting to master " + masterUrl) try { - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - masterAddress = master.path.address + master = context.actorSelection(Master.toAkkaUrl(masterUrl)) master ! RegisterApplication(appDescription) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => logError("Failed to connect to master", e) @@ -71,6 +67,7 @@ private[spark] class Client( override def receive = { case RegisteredApplication(appId_) => + context.watch(sender) appId = appId_ listener.connected(appId) @@ -92,18 +89,8 @@ private[spark] class Client( listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) } - case Terminated(actor_) if actor_ == master => - logError("Connection to master failed; stopping client") - markDisconnected() - context.stop(self) - - case DisassociatedEvent(_, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") - markDisconnected() - context.stop(self) - - case AssociationErrorEvent(_, _, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + case Terminated(actor_) => + logError(s"Connection to $actor_ dropped, stopping client") markDisconnected() context.stop(self) 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 3904b701b2..400d6f26ea 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 @@ -24,7 +24,7 @@ import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ -import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.actor._ import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} import org.apache.spark.Logging @@ -34,6 +34,16 @@ 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 scala.Some +import org.apache.spark.deploy.DeployMessages.Heartbeat +import org.apache.spark.deploy.DeployMessages.RegisteredWorker +import akka.remote.DisassociatedEvent +import org.apache.spark.deploy.DeployMessages.LaunchExecutor +import org.apache.spark.deploy.DeployMessages.RegisterWorker private[spark] class Worker( @@ -54,7 +64,7 @@ private[spark] class Worker( // Send a heartbeat every (heartbeat timeout) / 4 milliseconds val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 - var master: ActorRef = null + var master: ActorSelection = null var masterWebUiUrl : String = "" val workerId = generateWorkerId() var sparkHome: File = null @@ -111,10 +121,8 @@ private[spark] class Worker( def connectToMaster() { logInfo("Connecting to master " + masterUrl) - master = context.actorFor(Master.toAkkaUrl(masterUrl)) + master = context.actorSelection(Master.toAkkaUrl(masterUrl)) master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing } import context.dispatcher @@ -123,6 +131,8 @@ private[spark] class Worker( case RegisteredWorker(url) => masterWebUiUrl = url logInfo("Successfully registered with master") + context.watch(sender) // remote death watch for master + //TODO: Is heartbeat really necessary akka does it anyway ! context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { master ! Heartbeat(workerId) } @@ -165,7 +175,8 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case DisassociatedEvent(_, _, _) => + case Terminated(actor_) => + logInfo(s"$actor_ terminated !") masterDisconnected() case RequestWorkerState => { 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 0c977f05d1..c1aa43d59c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -17,14 +17,7 @@ package org.apache.spark.storage -import java.io._ -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import akka.actor._ import scala.concurrent.Await import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global @@ -34,8 +27,16 @@ import scala.concurrent.duration._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.storage.BlockManagerMessages.GetLocations +import org.apache.spark.storage.BlockManagerMessages.GetLocationsMultipleBlockIds +import org.apache.spark.storage.BlockManagerMessages.RegisterBlockManager +import org.apache.spark.storage.BlockManagerMessages.HeartBeat +import org.apache.spark.storage.BlockManagerMessages.RemoveExecutor +import org.apache.spark.storage.BlockManagerMessages.GetPeers +import org.apache.spark.storage.BlockManagerMessages.RemoveBlock +import org.apache.spark.storage.BlockManagerMessages.RemoveRdd -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 @@ -165,7 +166,11 @@ 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 = if (driverActor.isLeft ) { + driverActor.left.get.ask(message)(timeout) + } else { + driverActor.right.get.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/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index f2ae8dd97d..1e6da269f2 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( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 18fb1bf590..955f6cdadc 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 MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(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 MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -103,13 +103,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTracker() - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") + masterTracker.trackerActor = Left(actorSystem.actorOf( + Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker")) val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() - slaveTracker.trackerActor = slaveSystem.actorFor( - "akka.tcp://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/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 038a9acb85..4fdc43cc22 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -53,7 +53,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/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 08e399f9ee..128711aacd 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 @@ -86,7 +86,7 @@ class FeederActor extends Actor { 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) 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 394a39fbb0..b2f9f8b224 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 @@ -178,7 +178,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging val ip = System.getProperty("spark.driver.host", "localhost") val port = System.getProperty("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - val tracker = env.actorSystem.actorFor(url) + val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds override def preStart() { -- cgit v1.2.3 From f629ba95b6a1a3508463bfdcb03efcfaa3327cb5 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 14 Nov 2013 22:13:09 -0800 Subject: Various merge corrections I've diff'd this patch against my own -- since they were both created independently, this means that two sets of eyes have gone over all the merge conflicts that were created, so I'm feeling significantly more confident in the resulting PR. @rxin has looked at the changes to the repl and is resoundingly confident that they are correct. --- .../apache/spark/api/java/function/Function.java | 2 - .../apache/spark/api/java/function/Function2.java | 2 - .../org/apache/spark/deploy/client/Client.scala | 4 +- .../spark/deploy/master/ApplicationState.scala | 3 +- .../org/apache/spark/deploy/master/Master.scala | 14 +---- .../org/apache/spark/deploy/worker/Worker.scala | 17 +------ .../executor/CoarseGrainedExecutorBackend.scala | 1 - .../scala/org/apache/spark/executor/Executor.scala | 6 ++- .../org/apache/spark/rdd/AsyncRDDActions.scala | 2 +- .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 3 +- .../scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 7 +-- .../spark/scheduler/cluster/ClusterScheduler.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 1 + .../main/scala/org/apache/spark/util/Utils.scala | 5 +- .../apache/spark/util/collection/OpenHashMap.scala | 1 - docs/hadoop-third-party-distributions.md | 4 +- project/SparkBuild.scala | 11 +--- .../scala/org/apache/spark/repl/SparkILoop.scala | 14 ++--- .../scala/org/apache/spark/repl/ReplSuite.scala | 2 +- .../spark/streaming/NetworkInputTracker.scala | 2 +- .../spark/streaming/PairDStreamFunctions.scala | 1 - .../spark/streaming/api/java/JavaPairDStream.scala | 2 +- .../spark/streaming/dstream/CoGroupedDStream.scala | 59 ---------------------- .../streaming/dstream/KafkaInputDStream.scala | 1 - 25 files changed, 33 insertions(+), 135 deletions(-) delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala (limited to 'streaming') 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 49e661a376..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 @@ -29,8 +29,6 @@ import java.io.Serializable; * when mapping RDDs of other types. */ public abstract class Function extends WrappedFunction1 implements Serializable { - public abstract R call(T t) throws Exception; - public ClassTag 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 cf77bb6b73..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 @@ -28,8 +28,6 @@ import java.io.Serializable; public abstract class Function2 extends WrappedFunction2 implements Serializable { - public abstract R call(T1 t1, T2 t2) throws Exception; - public ClassTag returnType() { return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } 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 3953a3e178..572fc347df 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 @@ -145,11 +145,11 @@ private[spark] class Client( markDisconnected() case DisassociatedEvent(_, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case AssociationErrorEvent(_, _, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + logWarning("Connection to master 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 a74d7be4c9..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 { +private[spark] object ApplicationState extends Enumeration { type ApplicationState = Value 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 0545ad185f..7db5097c2d 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 @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.concurrent.duration.{ Duration, FiniteDuration } +import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask @@ -41,16 +41,6 @@ import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import scala.Some -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisteredApplication -import org.apache.spark.deploy.DeployMessages.RegisterWorker -import org.apache.spark.deploy.DeployMessages.ExecutorUpdated -import org.apache.spark.deploy.DeployMessages.MasterStateResponse -import org.apache.spark.deploy.DeployMessages.ExecutorAdded -import org.apache.spark.deploy.DeployMessages.RegisterApplication -import org.apache.spark.deploy.DeployMessages.ApplicationRemoved -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker import akka.actor.Terminated import akka.serialization.SerializationExtension import java.util.concurrent.TimeUnit @@ -571,7 +561,7 @@ private[spark] object Master { def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName) - val timeoutDuration : FiniteDuration = Duration.create( + 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 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 98c57ca0b0..07189ac850 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 @@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ -import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.Logging import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} @@ -34,19 +34,6 @@ 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 scala.Some -import akka.remote.DisassociatedEvent -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisterWorker -import org.apache.spark.deploy.DeployMessages.WorkerSchedulerStateResponse -import org.apache.spark.deploy.DeployMessages.MasterChanged -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import akka.actor.Terminated /** * @param masterUrls Each url should look like spark://host:port. @@ -248,7 +235,7 @@ private[spark] class Worker( } } - case DisassociatedEvent(_, _, _) => + case DisassociatedEvent(_, address, _) if address == master.path.address => masterDisconnected() case RequestWorkerState => { 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 73fa7d6b6a..50302fcca4 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -107,7 +107,6 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) - actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") 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 de4540493a..0b0a60ee60 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -118,7 +118,11 @@ private[spark] class Executor( } } - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") + // 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.tcp.maximum-frame-size") + } // Start worker thread pool val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") 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 44c5078621..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,9 +21,9 @@ 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} -import scala.reflect.ClassTag /** * A set of asynchronous RDD actions available through an implicit conversion. 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 63b9fe1478..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,9 +17,10 @@ package org.apache.spark.rdd +import scala.reflect.ClassTag + import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.{BlockId, BlockManager} -import scala.reflect.ClassTag private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 47e958b5e6..53f77a38f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -52,7 +52,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * sources in HBase, or S3). * * @param sc The SparkContext to associate the RDD with. - * @param broadCastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed + * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. * Otherwise, a new JobConf will be created on each slave using the enclosed Configuration. * @param initLocalJobConfFuncOpt Optional closure used to initialize any JobConf that HadoopRDD 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 7b4fc6b9be..fdea3f6f88 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -20,13 +20,14 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.concurrent.duration._ -import scala.concurrent.ExecutionContext.Implicits.global -import akka.actor._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ import scala.reflect.ClassTag +import akka.actor._ + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics 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 2d8a0a62c9..9975ec1ab6 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 @@ -25,8 +25,8 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState 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 8de9b72b2f..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 @@ -181,6 +181,7 @@ private[spark] class CoarseMesosSchedulerBackend( !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) + totalCoresAcquired += cpusToUse val taskId = newMesosTaskId() taskIdToSlaveId(taskId) = slaveId slaveIdsWithExecutors += slaveId 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 7557ddab19..02adcb41c6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,14 +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.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.collection.Map +import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.Some - import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder 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 45849b3380..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 @@ -19,7 +19,6 @@ 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, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index f706625fe9..b33af2cf24 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -25,8 +25,8 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.

CDH Releases

- - + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 26e6a8326c..476e7c5800 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -105,12 +105,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/"), @@ -292,11 +286,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), - "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1" + "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") - exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty), + exclude("net.sf.jopt-simple", "jopt-simple"), "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), 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 43e504c290..523fd1222d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -940,17 +940,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, 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 } 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 418c31e24b..c230a03298 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -78,7 +78,7 @@ class ReplSuite extends FunSuite { System.clearProperty("spark.hostPort") } - test ("simple foreach with accumulator") { + test("simple foreach with accumulator") { val output = runInterpreter("local", """ |val accum = sc.accumulator(0) |sc.parallelize(1 to 10).foreach(x => accum += x) 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 66fe6e7870..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 scala.concurrent.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 ea5c165691..80af96c060 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -33,7 +33,6 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.conf.Configuration -import scala.Some class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) extends Serializable { 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 3ba37bed4d..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 @@ -728,7 +728,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } object JavaPairDStream { - implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) : JavaPairDStream[K, V] = { + implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) = { new JavaPairDStream[K, V](dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala deleted file mode 100644 index 16c1567355..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.streaming.dstream - -import org.apache.spark.Partitioner -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.CoGroupedRDD -import org.apache.spark.streaming.{Time, DStream, Duration} -import scala.reflect.ClassTag - -private[streaming] -class CoGroupedDStream[K : ClassTag]( - parents: Seq[DStream[(K, _)]], - partitioner: Partitioner - ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { - - if (parents.length == 0) { - throw new IllegalArgumentException("Empty array of parents") - } - - if (parents.map(_.ssc).distinct.size > 1) { - throw new IllegalArgumentException("Array of parents have different StreamingContexts") - } - - if (parents.map(_.slideDuration).distinct.size > 1) { - throw new IllegalArgumentException("Array of parents have different slide times") - } - - override def dependencies = parents.toList - - override def slideDuration: Duration = parents.head.slideDuration - - override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = { - val part = partitioner - val rdds = parents.flatMap(_.getOrCompute(validTime)) - if (rdds.size > 0) { - val q = new CoGroupedRDD[K](rdds, part) - Some(q) - } else { - None - } - } - -} 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 ec0096c85f..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 @@ -33,7 +33,6 @@ import org.I0Itec.zkclient._ import scala.collection.Map import scala.reflect.ClassTag - /** * Input stream that pulls messages from a Kafka Broker. * -- cgit v1.2.3 From 7ad6921ae0657ca806704f859d5b8c9ff26633e4 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 7 Dec 2013 12:45:57 +0530 Subject: Incorporated Patrick's feedback comment on #211 and made maven build/dep-resolution atleast a bit faster. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 56 +++++-------------------------------------------------- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 15 insertions(+), 61 deletions(-) (limited to 'streaming') diff --git a/assembly/pom.xml b/assembly/pom.xml index 28b0692dff..8103534796 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-assembly_${scala-short.version} + spark-assembly_2.10 Spark Project Assembly http://spark.incubator.apache.org/ diff --git a/bagel/pom.xml b/bagel/pom.xml index c8b9c4f4cd..461e76a753 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 jar Spark Project Bagel http://spark.incubator.apache.org/ diff --git a/core/pom.xml b/core/pom.xml index e2033c9912..af605e1837 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 jar Spark Project Core http://spark.incubator.apache.org/ diff --git a/examples/pom.xml b/examples/pom.xml index a10dee7847..464ad82e33 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-examples_${scala-short.version} + spark-examples_2.10 jar Spark Project Examples http://spark.incubator.apache.org/ diff --git a/mllib/pom.xml b/mllib/pom.xml index a57bddeff3..fce5b19bb2 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 jar Spark Project ML Library http://spark.incubator.apache.org/ diff --git a/pom.xml b/pom.xml index 8700a4828f..4c11459088 100644 --- a/pom.xml +++ b/pom.xml @@ -104,7 +104,7 @@ 2.10 2.10.3 0.13.0 - 2.2.1 + 2.2.3 1.7.2 1.2.17 1.0.4 @@ -116,10 +116,10 @@ - - typesafe-repo - Typesafe Repository - http://repo.typesafe.com/typesafe/releases/ + + maven-repo + Maven Repository + http://repo.maven.apache.org/maven2/ true @@ -138,17 +138,6 @@ false - - akka-repo - Akka Repository - http://repo.akka.io/releases/ - - true - - - false - - mqtt-repo MQTT Repository @@ -161,41 +150,6 @@ - - - oss-sonatype-releases - OSS Sonatype - https://oss.sonatype.org/content/repositories/releases - - true - - - false - - - - oss-sonatype-snapshots - OSS Sonatype - https://oss.sonatype.org/content/repositories/snapshots - - false - - - true - - - - oss-sonatype - OSS Sonatype - https://oss.sonatype.org/content/groups/public - - true - - - true - - - diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index c983ea5dfb..f8a17d93cc 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-repl-bin_${scala-short.version} + spark-repl-bin_2.10 pom Spark Project REPL binary packaging http://spark.incubator.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index ff66493229..2f27e76760 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 jar Spark Project REPL http://spark.incubator.apache.org/ diff --git a/streaming/pom.xml b/streaming/pom.xml index fb15681e25..ff95591b9f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -26,7 +26,7 @@ org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 jar Spark Project Streaming http://spark.incubator.apache.org/ diff --git a/tools/pom.xml b/tools/pom.xml index db87b54dec..353d201f90 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -25,7 +25,7 @@ org.apache.spark - spark-tools_${scala-short.version} + spark-tools_2.10 jar Spark Project Tools http://spark.incubator.apache.org/ diff --git a/yarn/pom.xml b/yarn/pom.xml index 12bc97da8a..5cf81575c9 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -25,7 +25,7 @@ org.apache.spark - spark-yarn_${scala-short.version} + spark-yarn_2.10 jar Spark Project YARN Support http://spark.incubator.apache.org/ -- cgit v1.2.3 From 17db6a9041d5e83d7b6fe47f9c36758d0613fcd6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 10 Dec 2013 11:34:10 +0530 Subject: Style fixes and addressed review comments at #221 --- assembly/pom.xml | 14 +++++++------- bagel/pom.xml | 10 +++++----- core/pom.xml | 18 +++++++++--------- .../scala/org/apache/spark/MapOutputTracker.scala | 9 ++++----- .../main/scala/org/apache/spark/Partitioner.scala | 6 +++--- .../apache/spark/deploy/worker/ui/WorkerWebUI.scala | 14 +++++--------- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 11 ++++------- .../main/scala/org/apache/spark/rdd/JdbcRDD.scala | 1 + .../main/scala/org/apache/spark/rdd/MappedRDD.scala | 3 ++- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 3 ++- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 3 ++- .../apache/spark/storage/BlockManagerMaster.scala | 7 +++---- .../spark/util/IndestructibleActorSystem.scala | 17 +++++++++++++++-- examples/pom.xml | 20 ++++++++++---------- mllib/pom.xml | 10 +++++----- pom.xml | 17 ++++++++--------- repl-bin/pom.xml | 6 +++--- repl/pom.xml | 14 +++++++------- streaming/pom.xml | 16 ++++++++-------- .../spark/streaming/receivers/ZeroMQReceiver.scala | 4 ++-- tools/pom.xml | 10 +++++----- yarn/pom.xml | 8 ++++---- 22 files changed, 114 insertions(+), 107 deletions(-) (limited to 'streaming') diff --git a/assembly/pom.xml b/assembly/pom.xml index 8103534796..c2cda41c6d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -41,27 +41,27 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 ${project.version} org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} @@ -79,7 +79,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala-short.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + ${project.build.directory}/scala-2.10/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar *:* @@ -128,7 +128,7 @@ org.apache.spark - spark-yarn_${scala-short.version} + spark-yarn_2.10 ${project.version} diff --git a/bagel/pom.xml b/bagel/pom.xml index 461e76a753..0f550d70d6 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -43,18 +43,18 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/core/pom.xml b/core/pom.xml index af605e1837..71bf15c491 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -86,7 +86,7 @@ com.twitter - chill_${scala-short.version} + chill_2.10 0.3.1 @@ -96,11 +96,11 @@ com.typesafe.akka - akka-remote_${scala-short.version} + akka-remote_2.10 com.typesafe.akka - akka-slf4j_${scala-short.version} + akka-slf4j_2.10 org.scala-lang @@ -108,7 +108,7 @@ net.liftweb - lift-json_${scala-short.version} + lift-json_2.10 it.unimi.dsi @@ -120,7 +120,7 @@ com.github.scala-incubator.io - scala-io-file_${scala-short.version} + scala-io-file_2.10 org.apache.mesos @@ -166,12 +166,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -191,8 +191,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 88a7f24884..d36e1b13a6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,12 +72,11 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { - val future = if (trackerActor.isLeft ) { - trackerActor.left.get.ask(message)(timeout) - } else { - trackerActor.right.get.ask(message)(timeout) + val future = trackerActor match { + case Left(a: ActorRef) => a.ask(message)(timeout) + case Right(b: ActorSelection) => b.ask(message)(timeout) } - return Await.result(future, timeout) + Await.result(future, timeout) } catch { case e: Exception => throw new SparkException("Error communicating with MapOutputTracker", e) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 62b608c088..bcec41c439 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -17,11 +17,11 @@ package org.apache.spark -import org.apache.spark.util.Utils -import org.apache.spark.rdd.RDD - 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. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. 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 a38e32b339..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,23 +17,19 @@ package org.apache.spark.deploy.worker.ui -import akka.actor.ActorRef -import akka.util.Timeout +import java.io.File import scala.concurrent.duration._ -import java.io.{FileInputStream, File} - +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/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 99ea6e8ee8..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,16 +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 {} 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 e72f86fb13..8df8718f3b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag + import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.util.NextIterator 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 eb3b19907d..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,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} + private[spark] class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) extends RDD[U](prev) { 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 4a465840c6..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,9 +17,10 @@ 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 * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to 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 1d109a2496..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,9 +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 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 e5de16fc01..e05b842476 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -157,10 +157,9 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, Actor while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = if (driverActor.isLeft ) { - driverActor.left.get.ask(message)(timeout) - } else { - driverActor.right.get.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) { diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala index 69519860c6..bf71882ef7 100644 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala @@ -1,5 +1,18 @@ -/** - * Copyright (C) 2009-2013 Typesafe Inc. +/* + * 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]. diff --git a/examples/pom.xml b/examples/pom.xml index 464ad82e33..97f6dfea66 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -49,25 +49,25 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} provided org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} provided org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} provided org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} provided @@ -88,7 +88,7 @@ org.apache.kafka - kafka_2.9.2 + kafka_2.10 0.8.0-beta1 @@ -107,17 +107,17 @@ com.twitter - algebird-core_${scala-short.version} + algebird-core_2.10 0.1.11 org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -166,8 +166,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/mllib/pom.xml b/mllib/pom.xml index fce5b19bb2..228f8c029b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -48,12 +48,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -63,8 +63,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/pom.xml b/pom.xml index 4c11459088..979fd0c287 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,6 @@ 1.6 - 2.10 2.10.3 0.13.0 2.2.3 @@ -205,7 +204,7 @@ com.twitter - chill_${scala-short.version} + chill_2.10 0.3.1 @@ -215,7 +214,7 @@ com.typesafe.akka - akka-actor_${scala-short.version} + akka-actor_2.10 ${akka.version} @@ -226,7 +225,7 @@ com.typesafe.akka - akka-remote_${scala-short.version} + akka-remote_2.10 ${akka.version} @@ -237,7 +236,7 @@ com.typesafe.akka - akka-slf4j_${scala-short.version} + akka-slf4j_2.10 ${akka.version} @@ -258,7 +257,7 @@ com.github.scala-incubator.io - scala-io-file_${scala-short.version} + scala-io-file_2.10 0.4.1 @@ -279,7 +278,7 @@ net.liftweb - lift-json_${scala-short.version} + lift-json_2.10 2.5.1 @@ -335,7 +334,7 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 1.9.1 test @@ -358,7 +357,7 @@ org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 1.10.0 test diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index f8a17d93cc..c2a4efa36c 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -40,18 +40,18 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} runtime org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 ${project.version} runtime diff --git a/repl/pom.xml b/repl/pom.xml index 2f27e76760..bf06d730d4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -39,18 +39,18 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} runtime org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} runtime @@ -78,18 +78,18 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/streaming/pom.xml b/streaming/pom.xml index ff95591b9f..298bc83161 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -61,8 +61,8 @@ 1.9.11 - org.apache.kafka - kafka_2.9.2 + com.sksamuel.kafka + kafka_2.10 0.8.0-beta1 @@ -111,7 +111,7 @@ com.typesafe.akka - akka-zeromq_${scala-short.version} + akka-zeromq_2.10 ${akka.version} @@ -122,12 +122,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -151,8 +151,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest 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 ce8c56fa8a..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,14 +17,14 @@ 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 -import scala.reflect.ClassTag - /** * A receiver to subscribe to ZeroMQ stream. */ diff --git a/tools/pom.xml b/tools/pom.xml index 353d201f90..2bad494fc5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -33,24 +33,24 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/yarn/pom.xml b/yarn/pom.xml index 5cf81575c9..d18ac3736d 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -33,7 +33,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -63,7 +63,7 @@ org.scalatest - scalatest_2.9.3 + scalatest_2.10 test @@ -74,8 +74,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins -- cgit v1.2.3 From 09ed7ddfa0ef8dde12304a26be776232cadc8b1e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 15 Dec 2013 12:39:58 -0800 Subject: Use scala.binary.version in POMs --- assembly/pom.xml | 14 +++++++------- bagel/pom.xml | 10 +++++----- core/pom.xml | 18 +++++++++--------- examples/pom.xml | 20 ++++++++++---------- mllib/pom.xml | 10 +++++----- pom.xml | 17 +++++++++-------- repl-bin/pom.xml | 6 +++--- repl/pom.xml | 14 +++++++------- streaming/pom.xml | 14 +++++++------- tools/pom.xml | 10 +++++----- yarn/pom.xml | 8 ++++---- 11 files changed, 71 insertions(+), 70 deletions(-) (limited to 'streaming') diff --git a/assembly/pom.xml b/assembly/pom.xml index c2cda41c6d..fc2adc1fbb 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -41,27 +41,27 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} org.apache.spark - spark-bagel_2.10 + spark-bagel_${scala.binary.version} ${project.version} org.apache.spark - spark-mllib_2.10 + spark-mllib_${scala.binary.version} ${project.version} org.apache.spark - spark-repl_2.10 + spark-repl_${scala.binary.version} ${project.version} org.apache.spark - spark-streaming_2.10 + spark-streaming_${scala.binary.version} ${project.version} @@ -79,7 +79,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-2.10/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar *:* @@ -128,7 +128,7 @@ org.apache.spark - spark-yarn_2.10 + spark-yarn_${scala.binary.version} ${project.version} diff --git a/bagel/pom.xml b/bagel/pom.xml index 0f550d70d6..cb8e79f225 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} @@ -43,18 +43,18 @@ org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} test - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.scalatest diff --git a/core/pom.xml b/core/pom.xml index 3fe48fd2af..cdbaa52731 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -86,7 +86,7 @@ com.twitter - chill_2.10 + chill_${scala.binary.version} 0.3.1 @@ -96,15 +96,15 @@ ${akka.group} - akka-actor_2.10 + akka-actor_${scala.binary.version} ${akka.group} - akka-remote_2.10 + akka-remote_${scala.binary.version} ${akka.group} - akka-slf4j_2.10 + akka-slf4j_${scala.binary.version} org.scala-lang @@ -112,7 +112,7 @@ net.liftweb - lift-json_2.10 + lift-json_${scala.binary.version} it.unimi.dsi @@ -166,12 +166,12 @@ org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} test @@ -191,8 +191,8 @@ - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.apache.maven.plugins diff --git a/examples/pom.xml b/examples/pom.xml index e16b1c80aa..7a7032c319 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -49,25 +49,25 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} provided org.apache.spark - spark-streaming_2.10 + spark-streaming_${scala.binary.version} ${project.version} provided org.apache.spark - spark-mllib_2.10 + spark-mllib_${scala.binary.version} ${project.version} provided org.apache.spark - spark-bagel_2.10 + spark-bagel_${scala.binary.version} ${project.version} provided @@ -88,7 +88,7 @@ com.sksamuel.kafka - kafka_2.10 + kafka_${scala.binary.version} 0.8.0-beta1 @@ -107,17 +107,17 @@ com.twitter - algebird-core_2.10 + algebird-core_${scala.binary.version} 0.1.11 org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} test @@ -166,8 +166,8 @@ - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.apache.maven.plugins diff --git a/mllib/pom.xml b/mllib/pom.xml index 228f8c029b..dda3900afe 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} @@ -48,12 +48,12 @@ org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} test @@ -63,8 +63,8 @@ - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.scalatest diff --git a/pom.xml b/pom.xml index 58fc02e029..fd99fabc15 100644 --- a/pom.xml +++ b/pom.xml @@ -102,6 +102,7 @@ 1.6 2.10.3 + 2.10 0.13.0 2.2.3 com.typesafe.akka @@ -206,7 +207,7 @@ com.twitter - chill_2.10 + chill_${scala.binary.version} 0.3.1 @@ -216,7 +217,7 @@ ${akka.group} - akka-actor_2.10 + akka-actor_${scala.binary.version} ${akka.version} @@ -227,7 +228,7 @@ ${akka.group} - akka-remote_2.10 + akka-remote_${scala.binary.version} ${akka.version} @@ -238,7 +239,7 @@ ${akka.group} - akka-slf4j_2.10 + akka-slf4j_${scala.binary.version} ${akka.version} @@ -249,7 +250,7 @@ ${akka.group} - akka-zeromq_2.10 + akka-zeromq_${scala.binary.version} ${akka.version} @@ -286,7 +287,7 @@ net.liftweb - lift-json_2.10 + lift-json_${scala.binary.version} 2.5.1 @@ -342,7 +343,7 @@ org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} 1.9.1 test @@ -365,7 +366,7 @@ org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} 1.10.0 test diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index c2a4efa36c..869dbdb9b0 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -40,18 +40,18 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} org.apache.spark - spark-bagel_2.10 + spark-bagel_${scala.binary.version} ${project.version} runtime org.apache.spark - spark-repl_2.10 + spark-repl_${scala.binary.version} ${project.version} runtime diff --git a/repl/pom.xml b/repl/pom.xml index bf06d730d4..b0e7877bbb 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -39,18 +39,18 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} org.apache.spark - spark-bagel_2.10 + spark-bagel_${scala.binary.version} ${project.version} runtime org.apache.spark - spark-mllib_2.10 + spark-mllib_${scala.binary.version} ${project.version} runtime @@ -78,18 +78,18 @@ org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} test - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.apache.maven.plugins diff --git a/streaming/pom.xml b/streaming/pom.xml index e27b437aae..e3b6fee9b2 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} @@ -62,7 +62,7 @@ com.sksamuel.kafka - kafka_2.10 + kafka_${scala.binary.version} 0.8.0-beta1 @@ -111,16 +111,16 @@ ${akka.group} - akka-zeromq_2.10 + akka-zeromq_${scala.binary.version} org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_2.10 + scalacheck_${scala.binary.version} test @@ -144,8 +144,8 @@ - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.scalatest diff --git a/tools/pom.xml b/tools/pom.xml index 2bad494fc5..28f5ef14b1 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -33,24 +33,24 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} org.apache.spark - spark-streaming_2.10 + spark-streaming_${scala.binary.version} ${project.version} org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.apache.maven.plugins diff --git a/yarn/pom.xml b/yarn/pom.xml index d18ac3736d..bc64a190fd 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -33,7 +33,7 @@ org.apache.spark - spark-core_2.10 + spark-core_${scala.binary.version} ${project.version} @@ -63,7 +63,7 @@ org.scalatest - scalatest_2.10 + scalatest_${scala.binary.version} test @@ -74,8 +74,8 @@ - target/scala-2.10/classes - target/scala-2.10/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.apache.maven.plugins -- cgit v1.2.3
ReleaseVersion code
CDH 4.X.X (YARN mode)2.0.0-chd4.X.X
CDH 4.X.X2.0.0-mr1-chd4.X.X
CDH 4.X.X (YARN mode)2.0.0-cdh4.X.X
CDH 4.X.X2.0.0-mr1-cdh4.X.X
CDH 3u60.20.2-cdh3u6
CDH 3u50.20.2-cdh3u5
CDH 3u40.20.2-cdh3u4